Monday, May 11, 2015

Data Locality w/ Cassandra : How to scan the local token range of a table...


I'm working on a mechanism that will allow HPCC to access data stored in Cassandra with data locality, leveraging the Java streaming capabilities from HPCC (more on this in a followup post). More specifically, we want to allow people to write functions in ECL that will execute on all nodes in an HPCC cluster, using collocated Cassandra instances as the source of their data.

To do this however, we need a couple things.   If you remember, Cassandra data is spread across multiple nodes using a token ring.  Each host is assigned one or more slices of that token ring.  Each slice has a token range, with a start and an end.  Each partition/row key hashes into a token, which determines which node gets that data.  (Typically, Murmur3 is used as the hashing algorithm.)

Thus, to scan over the data that is local to a node, we need to determine the token ranges for our local node, then we need to page over the data in that range.

First, let's look at how we determine the ranges for the local node.  You could query the system tables directly: (SELECT * FROM system.local), but if you are using a connection pool (via the Java driver), it is unclear which node will receive that query. You could also query the information from the system.peers table using your IP address in the where clause, but you may not want to configure that IP address on each node.  Instead, I was able to lean on the CQL java-driver to determine the localhost:

    public Host getLocalHost(Metadata metadata, LoadBalancingPolicy policy) {
        Set<Host> allHosts = metadata.getAllHosts();
        StringBuilder s = new StringBuilder();
        Host localHost = null;
        for (Host host : allHosts) {
            if (policy.distance(host) == HostDistance.LOCAL) {
                localHost = host;
                break;
            }
        }
        return localHost;
    }

With a Host in hand, the java-driver makes it easy to get the Token Ranges:

 Cluster cluster = Cluster.builder().addContactPoints(host).withPort(port).withLoadBalancingPolicy(policy).build();
        Metadata metadata = cluster.getMetadata();
        Host localhost = getLocalHost(metadata, policy);
        tokenRanges = unwrapTokenRanges(metadata.getTokenRanges(keyspace, localhost)).toArray(new TokenRange[0]);


The code is very straightforward, with the exception of the call to unwrapTokenRanges.   When you ask for the token ranges for a host, CQL will give you the tokens in ranges, but CQL does NOT handle wrapped ranges.  For example, let's assume we had a global token space of [-16...16].  Our host may have token ranges of [-10...-3], [12...14] and [15...-2].  You can issue the following CQL queries (Notice that token ranges are start exclusive and end inclusive.):

SELECT token(id), id, name FROM test_table WHERE token(id)>-10 AND token(id)<=-3;
SELECT token(id), id, name FROM test_table WHERE token(id)>12 AND token(id)<=14;

However, you CANNOT issue the following CQL:

SELECT token(id), id, name FROM test_table WHERE token(id)>15 AND token(id)<=-2;

That range wraps around.  To accommodate this, the java-driver provides a convenience method called unwrap().  You can then use that method, to create a set of token ranges usable in CQL queries that will account for the token range wrapping.


    Set<TokenRange> unwrapTokenRanges(Set<TokenRange> wrappedRanges) {
        HashSet<TokenRange> tokenRanges = new HashSet<TokenRange>();
        for (TokenRange tokenRange : wrappedRanges) {
            tokenRanges.addAll(tokenRange.unwrap());
        }
        return tokenRanges;
    }

Finally, we need to be able to page over the data.   Fortunately, with the 2.0 release of the java-driver, we are able to do this in a few lines of code:

     session = cluster.connect();  
     Statement select = QueryBuilder.select(columns.split(",")).from(table).where(gt(token(partitionKey), range.getStart().getValue()))
                        .and(lte(token(partitionKey), range.getEnd().getValue()));
     stmt.setFetchSize(pageSize);  
     resultSet = session.execute(stmt);   
     iterator = resultSet.iterator();    
     while (!resultSet.isFullyFetched()) {  
       resultSet.fetchMoreResults();  
       Row row = iterator.next();  
       System.out.println(row);  
     }  

The above code issues a select statement and pages over the results, scanning the portion of the table specified within the token range.

 If we throw a loop on top of all of this, to go through each token range and scan it, we have a means of executing a distributed processing job, that uses only the local portions of the Cassandra tables as input.

Stay tuned for my next post, when I show how to plug this into HPCC.

No comments: