|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package com.tuplejump.calliope.hadoop; |
| 19 | + |
| 20 | +import com.google.common.collect.ImmutableList; |
| 21 | +import com.google.common.collect.Lists; |
| 22 | +import org.apache.cassandra.auth.IAuthenticator; |
| 23 | +import org.apache.cassandra.dht.IPartitioner; |
| 24 | +import org.apache.cassandra.dht.Range; |
| 25 | +import org.apache.cassandra.dht.Token; |
| 26 | +import org.apache.cassandra.thrift.*; |
| 27 | +import org.apache.commons.lang3.StringUtils; |
| 28 | +import org.apache.hadoop.conf.Configuration; |
| 29 | +import org.apache.hadoop.mapred.JobConf; |
| 30 | +import org.apache.hadoop.mapreduce.*; |
| 31 | +import org.apache.thrift.TApplicationException; |
| 32 | +import org.apache.thrift.TException; |
| 33 | +import org.apache.thrift.protocol.TBinaryProtocol; |
| 34 | +import org.apache.thrift.protocol.TProtocol; |
| 35 | +import org.apache.thrift.transport.TTransport; |
| 36 | +import org.apache.thrift.transport.TTransportException; |
| 37 | +import org.slf4j.Logger; |
| 38 | +import org.slf4j.LoggerFactory; |
| 39 | + |
| 40 | +import java.io.IOException; |
| 41 | +import java.util.*; |
| 42 | +import java.util.concurrent.*; |
| 43 | + |
| 44 | + |
| 45 | +public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<K, Y> implements org.apache.hadoop.mapred.InputFormat<K, Y> { |
| 46 | + private static final Logger logger = LoggerFactory.getLogger(AbstractColumnFamilyInputFormat.class); |
| 47 | + |
| 48 | + public static final String MAPRED_TASK_ID = "mapred.task.id"; |
| 49 | + // The simple fact that we need this is because the old Hadoop API wants us to "write" |
| 50 | + // to the key and value whereas the new asks for it. |
| 51 | + // I choose 8kb as the default max key size (instanciated only once), but you can |
| 52 | + // override it in your jobConf with this setting. |
| 53 | + public static final String CASSANDRA_HADOOP_MAX_KEY_SIZE = "cassandra.hadoop.max_key_size"; |
| 54 | + public static final int CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT = 8192; |
| 55 | + |
| 56 | + private String keyspace; |
| 57 | + private String cfName; |
| 58 | + private IPartitioner partitioner; |
| 59 | + |
| 60 | + protected void validateConfiguration(Configuration conf) { |
| 61 | + if (ConfigHelper.getInputKeyspace(conf) == null || ConfigHelper.getInputColumnFamily(conf) == null) { |
| 62 | + throw new UnsupportedOperationException("you must set the keyspace and columnfamily with setInputColumnFamily()"); |
| 63 | + } |
| 64 | + if (ConfigHelper.getInputInitialAddress(conf) == null) |
| 65 | + throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node with setInputInitialAddress"); |
| 66 | + if (ConfigHelper.getInputPartitioner(conf) == null) |
| 67 | + throw new UnsupportedOperationException("You must set the Cassandra partitioner class with setInputPartitioner"); |
| 68 | + } |
| 69 | + |
| 70 | + public static Cassandra.Client createAuthenticatedClient(String location, int port, Configuration conf) throws Exception { |
| 71 | + logger.debug("Creating authenticated client for CF input format"); |
| 72 | + TTransport transport; |
| 73 | + try { |
| 74 | + transport = ConfigHelper.getClientTransportFactory(conf).openTransport(location, port); |
| 75 | + } catch (Exception e) { |
| 76 | + throw new TTransportException("Failed to open a transport to " + location + ":" + port + ".", e); |
| 77 | + } |
| 78 | + TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true); |
| 79 | + Cassandra.Client client = new Cassandra.Client(binaryProtocol); |
| 80 | + |
| 81 | + // log in |
| 82 | + client.set_keyspace(ConfigHelper.getInputKeyspace(conf)); |
| 83 | + if ((ConfigHelper.getInputKeyspaceUserName(conf) != null) && (ConfigHelper.getInputKeyspacePassword(conf) != null)) { |
| 84 | + Map<String, String> creds = new HashMap<String, String>(); |
| 85 | + creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf)); |
| 86 | + creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf)); |
| 87 | + AuthenticationRequest authRequest = new AuthenticationRequest(creds); |
| 88 | + client.login(authRequest); |
| 89 | + } |
| 90 | + logger.debug("Authenticated client for CF input format created successfully"); |
| 91 | + return client; |
| 92 | + } |
| 93 | + |
| 94 | + public List<InputSplit> getSplits(JobContext context) throws IOException { |
| 95 | + Configuration conf = HadoopCompat.getConfiguration(context); |
| 96 | + ; |
| 97 | + |
| 98 | + validateConfiguration(conf); |
| 99 | + |
| 100 | + // cannonical ranges and nodes holding replicas |
| 101 | + List<TokenRange> masterRangeNodes = getRangeMap(conf); |
| 102 | + |
| 103 | + keyspace = ConfigHelper.getInputKeyspace(conf); |
| 104 | + cfName = ConfigHelper.getInputColumnFamily(conf); |
| 105 | + partitioner = ConfigHelper.getInputPartitioner(conf); |
| 106 | + logger.debug("partitioner is " + partitioner); |
| 107 | + |
| 108 | + |
| 109 | + // cannonical ranges, split into pieces, fetching the splits in parallel |
| 110 | + ExecutorService executor = new ThreadPoolExecutor(0, 128, 60L, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>()); |
| 111 | + List<InputSplit> splits = new ArrayList<InputSplit>(); |
| 112 | + |
| 113 | + try { |
| 114 | + List<Future<List<InputSplit>>> splitfutures = new ArrayList<Future<List<InputSplit>>>(); |
| 115 | + KeyRange jobKeyRange = ConfigHelper.getInputKeyRange(conf); |
| 116 | + Range<Token> jobRange = null; |
| 117 | + if (jobKeyRange != null) { |
| 118 | + if (jobKeyRange.start_key != null) { |
| 119 | + if (!partitioner.preservesOrder()) |
| 120 | + throw new UnsupportedOperationException("KeyRange based on keys can only be used with a order preserving paritioner"); |
| 121 | + if (jobKeyRange.start_token != null) |
| 122 | + throw new IllegalArgumentException("only start_key supported"); |
| 123 | + if (jobKeyRange.end_token != null) |
| 124 | + throw new IllegalArgumentException("only start_key supported"); |
| 125 | + jobRange = new Range<>(partitioner.getToken(jobKeyRange.start_key), |
| 126 | + partitioner.getToken(jobKeyRange.end_key), |
| 127 | + partitioner); |
| 128 | + } else if (jobKeyRange.start_token != null) { |
| 129 | + jobRange = new Range<>(partitioner.getTokenFactory().fromString(jobKeyRange.start_token), |
| 130 | + partitioner.getTokenFactory().fromString(jobKeyRange.end_token), |
| 131 | + partitioner); |
| 132 | + } else { |
| 133 | + logger.warn("ignoring jobKeyRange specified without start_key or start_token"); |
| 134 | + } |
| 135 | + } |
| 136 | + |
| 137 | + for (TokenRange range : masterRangeNodes) { |
| 138 | + if (jobRange == null) { |
| 139 | + // for each range, pick a live owner and ask it to compute bite-sized splits |
| 140 | + splitfutures.add(executor.submit(new SplitCallable(range, conf))); |
| 141 | + } else { |
| 142 | + Range<Token> dhtRange = new Range<Token>(partitioner.getTokenFactory().fromString(range.start_token), |
| 143 | + partitioner.getTokenFactory().fromString(range.end_token), |
| 144 | + partitioner); |
| 145 | + |
| 146 | + if (dhtRange.intersects(jobRange)) { |
| 147 | + for (Range<Token> intersection : dhtRange.intersectionWith(jobRange)) { |
| 148 | + range.start_token = partitioner.getTokenFactory().toString(intersection.left); |
| 149 | + range.end_token = partitioner.getTokenFactory().toString(intersection.right); |
| 150 | + // for each range, pick a live owner and ask it to compute bite-sized splits |
| 151 | + splitfutures.add(executor.submit(new SplitCallable(range, conf))); |
| 152 | + } |
| 153 | + } |
| 154 | + } |
| 155 | + } |
| 156 | + |
| 157 | + // wait until we have all the results back |
| 158 | + for (Future<List<InputSplit>> futureInputSplits : splitfutures) { |
| 159 | + try { |
| 160 | + splits.addAll(futureInputSplits.get()); |
| 161 | + } catch (Exception e) { |
| 162 | + throw new IOException("Could not get input splits", e); |
| 163 | + } |
| 164 | + } |
| 165 | + } finally { |
| 166 | + executor.shutdownNow(); |
| 167 | + } |
| 168 | + |
| 169 | + assert splits.size() > 0; |
| 170 | + Collections.shuffle(splits, new Random(System.nanoTime())); |
| 171 | + return splits; |
| 172 | + } |
| 173 | + |
| 174 | + /** |
| 175 | + * Gets a token range and splits it up according to the suggested |
| 176 | + * size into input splits that Hadoop can use. |
| 177 | + */ |
| 178 | + class SplitCallable implements Callable<List<InputSplit>> { |
| 179 | + |
| 180 | + private final TokenRange range; |
| 181 | + private final Configuration conf; |
| 182 | + |
| 183 | + public SplitCallable(TokenRange tr, Configuration conf) { |
| 184 | + this.range = tr; |
| 185 | + this.conf = conf; |
| 186 | + } |
| 187 | + |
| 188 | + public List<InputSplit> call() throws Exception { |
| 189 | + ArrayList<InputSplit> splits = new ArrayList<InputSplit>(); |
| 190 | + List<CfSplit> subSplits = getSubSplits(keyspace, cfName, range, conf); |
| 191 | + assert range.rpc_endpoints.size() == range.endpoints.size() : "rpc_endpoints size must match endpoints size"; |
| 192 | + // turn the sub-ranges into InputSplits |
| 193 | + String[] endpoints = range.endpoints.toArray(new String[range.endpoints.size()]); |
| 194 | + |
| 195 | + int endpointIndex = 0; |
| 196 | + for (String endpoint : range.rpc_endpoints) { |
| 197 | + String endpoint_address = endpoint; |
| 198 | + if (endpoint_address == null || endpoint_address.equals("0.0.0.0")) |
| 199 | + endpoint_address = range.endpoints.get(endpointIndex); |
| 200 | + endpoints[endpointIndex++] = endpoint_address; |
| 201 | + } |
| 202 | + |
| 203 | + Token.TokenFactory factory = partitioner.getTokenFactory(); |
| 204 | + for (CfSplit subSplit : subSplits) { |
| 205 | + Token left = factory.fromString(subSplit.getStart_token()); |
| 206 | + Token right = factory.fromString(subSplit.getEnd_token()); |
| 207 | + Range<Token> range = new Range<Token>(left, right, partitioner); |
| 208 | + List<Range<Token>> ranges = range.isWrapAround() ? range.unwrap() : ImmutableList.of(range); |
| 209 | + for (Range<Token> subrange : ranges) { |
| 210 | + ColumnFamilySplit split = |
| 211 | + new ColumnFamilySplit( |
| 212 | + factory.toString(subrange.left), |
| 213 | + factory.toString(subrange.right), |
| 214 | + subSplit.getRow_count(), |
| 215 | + endpoints); |
| 216 | + |
| 217 | + logger.debug("adding " + split); |
| 218 | + splits.add(split); |
| 219 | + } |
| 220 | + } |
| 221 | + return splits; |
| 222 | + } |
| 223 | + } |
| 224 | + |
| 225 | + private List<CfSplit> getSubSplits(String keyspace, String cfName, TokenRange range, Configuration conf) throws IOException { |
| 226 | + int splitsize = ConfigHelper.getInputSplitSize(conf); |
| 227 | + for (int i = 0; i < range.rpc_endpoints.size(); i++) { |
| 228 | + String host = range.rpc_endpoints.get(i); |
| 229 | + |
| 230 | + if (host == null || host.equals("0.0.0.0")) |
| 231 | + host = range.endpoints.get(i); |
| 232 | + |
| 233 | + try { |
| 234 | + Cassandra.Client client = ConfigHelper.createConnection(conf, host, ConfigHelper.getInputRpcPort(conf)); |
| 235 | + client.set_keyspace(keyspace); |
| 236 | + |
| 237 | + try { |
| 238 | + return client.describe_splits_ex(cfName, range.start_token, range.end_token, splitsize); |
| 239 | + } catch (TApplicationException e) { |
| 240 | + // fallback to guessing split size if talking to a server without describe_splits_ex method |
| 241 | + if (e.getType() == TApplicationException.UNKNOWN_METHOD) { |
| 242 | + List<String> splitPoints = client.describe_splits(cfName, range.start_token, range.end_token, splitsize); |
| 243 | + return tokenListToSplits(splitPoints, splitsize); |
| 244 | + } |
| 245 | + throw e; |
| 246 | + } |
| 247 | + } catch (IOException e) { |
| 248 | + logger.debug("failed connect to endpoint " + host, e); |
| 249 | + } catch (InvalidRequestException e) { |
| 250 | + throw new RuntimeException(e); |
| 251 | + } catch (TException e) { |
| 252 | + throw new RuntimeException(e); |
| 253 | + } |
| 254 | + } |
| 255 | + throw new IOException("failed connecting to all endpoints " + StringUtils.join(range.endpoints, ",")); |
| 256 | + } |
| 257 | + |
| 258 | + private List<CfSplit> tokenListToSplits(List<String> splitTokens, int splitsize) { |
| 259 | + List<CfSplit> splits = Lists.newArrayListWithExpectedSize(splitTokens.size() - 1); |
| 260 | + for (int j = 0; j < splitTokens.size() - 1; j++) |
| 261 | + splits.add(new CfSplit(splitTokens.get(j), splitTokens.get(j + 1), splitsize)); |
| 262 | + return splits; |
| 263 | + } |
| 264 | + |
| 265 | + private List<TokenRange> getRangeMap(Configuration conf) throws IOException { |
| 266 | + Cassandra.Client client = ConfigHelper.getClientFromInputAddressList(conf); |
| 267 | + |
| 268 | + List<TokenRange> map; |
| 269 | + try { |
| 270 | + map = client.describe_local_ring(ConfigHelper.getInputKeyspace(conf)); |
| 271 | + } catch (InvalidRequestException e) { |
| 272 | + throw new RuntimeException(e); |
| 273 | + } catch (TException e) { |
| 274 | + throw new RuntimeException(e); |
| 275 | + } |
| 276 | + return map; |
| 277 | + } |
| 278 | + |
| 279 | + // |
| 280 | + // Old Hadoop API |
| 281 | + // |
| 282 | + public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException { |
| 283 | + TaskAttemptContext tac = HadoopCompat.newTaskAttemptContext(jobConf, new TaskAttemptID()); |
| 284 | + List<InputSplit> newInputSplits = this.getSplits(tac); |
| 285 | + org.apache.hadoop.mapred.InputSplit[] oldInputSplits = new org.apache.hadoop.mapred.InputSplit[newInputSplits.size()]; |
| 286 | + for (int i = 0; i < newInputSplits.size(); i++) |
| 287 | + oldInputSplits[i] = (ColumnFamilySplit) newInputSplits.get(i); |
| 288 | + return oldInputSplits; |
| 289 | + } |
| 290 | +} |
0 commit comments