PageRenderTime 67ms CodeModel.GetById 10ms RepoModel.GetById 0ms app.codeStats 0ms

/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java

https://github.com/stephenc/cassandra
Java | 351 lines | 298 code | 34 blank | 19 comment | 41 complexity | 4b899d7314c01673d39a42f26e5796cf MD5 | raw file
  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 org.apache.cassandra.hadoop;
  19. import java.io.File;
  20. import java.io.IOException;
  21. import java.net.InetAddress;
  22. import java.nio.ByteBuffer;
  23. import java.net.UnknownHostException;
  24. import java.util.*;
  25. import java.util.concurrent.TimeUnit;
  26. import java.util.concurrent.TimeoutException;
  27. import org.apache.cassandra.auth.IAuthenticator;
  28. import org.apache.cassandra.config.Config;
  29. import org.apache.cassandra.config.DatabaseDescriptor;
  30. import org.apache.cassandra.db.marshal.AbstractType;
  31. import org.apache.cassandra.db.marshal.BytesType;
  32. import org.apache.cassandra.dht.Range;
  33. import org.apache.cassandra.dht.Token;
  34. import org.apache.cassandra.io.sstable.SSTableLoader;
  35. import org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter;
  36. import org.apache.cassandra.thrift.*;
  37. import org.apache.hadoop.conf.Configuration;
  38. import org.apache.hadoop.mapreduce.RecordWriter;
  39. import org.apache.hadoop.mapreduce.TaskAttemptContext;
  40. import org.apache.thrift.protocol.*;
  41. import org.apache.thrift.transport.TFramedTransport;
  42. import org.apache.thrift.transport.TSocket;
  43. import org.apache.thrift.transport.TTransport;
  44. import org.apache.thrift.transport.TTransportException;
  45. import org.slf4j.Logger;
  46. import org.slf4j.LoggerFactory;
  47. final class BulkRecordWriter extends RecordWriter<ByteBuffer,List<Mutation>>
  48. implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
  49. {
  50. private final static String OUTPUT_LOCATION = "mapreduce.output.bulkoutputformat.localdir";
  51. private final static String BUFFER_SIZE_IN_MB = "mapreduce.output.bulkoutputformat.buffersize";
  52. private final static String STREAM_THROTTLE_MBITS = "mapreduce.output.bulkoutputformat.streamthrottlembits";
  53. private final static String MAX_FAILED_HOSTS = "mapreduce.output.bulkoutputformat.maxfailedhosts";
  54. private final Configuration conf;
  55. private final Logger logger = LoggerFactory.getLogger(BulkRecordWriter.class);
  56. private SSTableSimpleUnsortedWriter writer;
  57. private SSTableLoader loader;
  58. private final File outputdir;
  59. private Progressable progress;
  60. private int maxFailures;
  61. private enum CFType
  62. {
  63. NORMAL,
  64. SUPER,
  65. }
  66. private enum ColType
  67. {
  68. NORMAL,
  69. COUNTER
  70. }
  71. private CFType cfType;
  72. private ColType colType;
  73. BulkRecordWriter(TaskAttemptContext context) throws IOException
  74. {
  75. this(context.getConfiguration());
  76. this.progress = new Progressable(context);
  77. }
  78. BulkRecordWriter(Configuration conf, Progressable progress) throws IOException
  79. {
  80. this(conf);
  81. this.progress = progress;
  82. }
  83. BulkRecordWriter(Configuration conf) throws IOException
  84. {
  85. Config.setLoadYaml(false);
  86. Config.setOutboundBindAny(true);
  87. this.conf = conf;
  88. DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(Integer.valueOf(conf.get(STREAM_THROTTLE_MBITS, "0")));
  89. maxFailures = Integer.valueOf(conf.get(MAX_FAILED_HOSTS, "0"));
  90. String keyspace = ConfigHelper.getOutputKeyspace(conf);
  91. outputdir = new File(getOutputLocation() + File.separator + keyspace + File.separator + ConfigHelper.getOutputColumnFamily(conf)); //dir must be named by ks/cf for the loader
  92. outputdir.mkdirs();
  93. }
  94. private String getOutputLocation() throws IOException
  95. {
  96. String dir = conf.get(OUTPUT_LOCATION, System.getProperty("java.io.tmpdir"));
  97. if (dir == null)
  98. throw new IOException("Output directory not defined, if hadoop is not setting java.io.tmpdir then define " + OUTPUT_LOCATION);
  99. return dir;
  100. }
  101. private void setTypes(Mutation mutation)
  102. {
  103. if (cfType == null)
  104. {
  105. if (mutation.getColumn_or_supercolumn().isSetSuper_column() || mutation.getColumn_or_supercolumn().isSetCounter_super_column())
  106. cfType = CFType.SUPER;
  107. else
  108. cfType = CFType.NORMAL;
  109. if (mutation.getColumn_or_supercolumn().isSetCounter_column() || mutation.getColumn_or_supercolumn().isSetCounter_super_column())
  110. colType = ColType.COUNTER;
  111. else
  112. colType = ColType.NORMAL;
  113. }
  114. }
  115. private void prepareWriter() throws IOException
  116. {
  117. if (writer == null)
  118. {
  119. AbstractType<?> subcomparator = null;
  120. ExternalClient externalClient = null;
  121. String username = ConfigHelper.getOutputKeyspaceUserName(conf);
  122. String password = ConfigHelper.getOutputKeyspacePassword(conf);
  123. if (cfType == CFType.SUPER)
  124. subcomparator = BytesType.instance;
  125. this.writer = new SSTableSimpleUnsortedWriter(
  126. outputdir,
  127. ConfigHelper.getOutputPartitioner(conf),
  128. ConfigHelper.getOutputKeyspace(conf),
  129. ConfigHelper.getOutputColumnFamily(conf),
  130. BytesType.instance,
  131. subcomparator,
  132. Integer.valueOf(conf.get(BUFFER_SIZE_IN_MB, "64")),
  133. ConfigHelper.getOutputCompressionParamaters(conf));
  134. externalClient = new ExternalClient(ConfigHelper.getOutputInitialAddress(conf),
  135. ConfigHelper.getOutputRpcPort(conf),
  136. username,
  137. password);
  138. this.loader = new SSTableLoader(outputdir, externalClient, new NullOutputHandler());
  139. }
  140. }
  141. @Override
  142. public void write(ByteBuffer keybuff, List<Mutation> value) throws IOException
  143. {
  144. setTypes(value.get(0));
  145. prepareWriter();
  146. writer.newRow(keybuff);
  147. for (Mutation mut : value)
  148. {
  149. if (cfType == CFType.SUPER)
  150. {
  151. writer.newSuperColumn(mut.getColumn_or_supercolumn().getSuper_column().name);
  152. if (colType == ColType.COUNTER)
  153. for (CounterColumn column : mut.getColumn_or_supercolumn().getCounter_super_column().columns)
  154. writer.addCounterColumn(column.name, column.value);
  155. else
  156. {
  157. for (Column column : mut.getColumn_or_supercolumn().getSuper_column().columns)
  158. {
  159. if(column.ttl == 0)
  160. writer.addColumn(column.name, column.value, column.timestamp);
  161. else
  162. writer.addExpiringColumn(column.name, column.value, column.timestamp, column.ttl, System.currentTimeMillis() + ((long)column.ttl * 1000));
  163. }
  164. }
  165. }
  166. else
  167. {
  168. if (colType == ColType.COUNTER)
  169. writer.addCounterColumn(mut.getColumn_or_supercolumn().counter_column.name, mut.getColumn_or_supercolumn().counter_column.value);
  170. else
  171. {
  172. if(mut.getColumn_or_supercolumn().column.ttl == 0)
  173. writer.addColumn(mut.getColumn_or_supercolumn().column.name, mut.getColumn_or_supercolumn().column.value, mut.getColumn_or_supercolumn().column.timestamp);
  174. else
  175. writer.addExpiringColumn(mut.getColumn_or_supercolumn().column.name, mut.getColumn_or_supercolumn().column.value, mut.getColumn_or_supercolumn().column.timestamp, mut.getColumn_or_supercolumn().column.ttl, System.currentTimeMillis() + ((long)(mut.getColumn_or_supercolumn().column.ttl) * 1000));
  176. }
  177. }
  178. progress.progress();
  179. }
  180. }
  181. @Override
  182. public void close(TaskAttemptContext context) throws IOException, InterruptedException
  183. {
  184. close();
  185. }
  186. /** Fills the deprecated RecordWriter interface for streaming. */
  187. @Deprecated
  188. public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
  189. {
  190. close();
  191. }
  192. private void close() throws IOException
  193. {
  194. if (writer != null)
  195. {
  196. writer.close();
  197. SSTableLoader.LoaderFuture future = loader.stream();
  198. while (true)
  199. {
  200. try
  201. {
  202. future.get(1000, TimeUnit.MILLISECONDS);
  203. break;
  204. }
  205. catch (TimeoutException te)
  206. {
  207. progress.progress();
  208. }
  209. catch (InterruptedException e)
  210. {
  211. throw new IOException(e);
  212. }
  213. }
  214. if (future.hadFailures())
  215. {
  216. if (future.getFailedHosts().size() > maxFailures)
  217. throw new IOException("Too many hosts failed: " + future.getFailedHosts());
  218. else
  219. logger.warn("Some hosts failed: " + future.getFailedHosts());
  220. }
  221. }
  222. }
  223. static class ExternalClient extends SSTableLoader.Client
  224. {
  225. private final Map<String, Set<String>> knownCfs = new HashMap<String, Set<String>>();
  226. private final String hostlist;
  227. private final int rpcPort;
  228. private final String username;
  229. private final String password;
  230. public ExternalClient(String hostlist, int port, String username, String password)
  231. {
  232. super();
  233. this.hostlist = hostlist;
  234. this.rpcPort = port;
  235. this.username = username;
  236. this.password = password;
  237. }
  238. public void init(String keyspace)
  239. {
  240. Set<InetAddress> hosts = new HashSet<InetAddress>();
  241. String[] nodes = hostlist.split(",");
  242. for (String node : nodes)
  243. {
  244. try
  245. {
  246. hosts.add(InetAddress.getByName(node));
  247. }
  248. catch (UnknownHostException e)
  249. {
  250. throw new RuntimeException(e);
  251. }
  252. }
  253. Iterator<InetAddress> hostiter = hosts.iterator();
  254. while (hostiter.hasNext())
  255. {
  256. try
  257. {
  258. InetAddress host = hostiter.next();
  259. Cassandra.Client client = createThriftClient(host.getHostAddress(), rpcPort);
  260. // log in
  261. client.set_keyspace(keyspace);
  262. if (username != null)
  263. {
  264. Map<String, String> creds = new HashMap<String, String>();
  265. creds.put(IAuthenticator.USERNAME_KEY, username);
  266. creds.put(IAuthenticator.PASSWORD_KEY, password);
  267. AuthenticationRequest authRequest = new AuthenticationRequest(creds);
  268. client.login(authRequest);
  269. }
  270. List<TokenRange> tokenRanges = client.describe_ring(keyspace);
  271. List<KsDef> ksDefs = client.describe_keyspaces();
  272. setPartitioner(client.describe_partitioner());
  273. Token.TokenFactory tkFactory = getPartitioner().getTokenFactory();
  274. for (TokenRange tr : tokenRanges)
  275. {
  276. Range<Token> range = new Range<Token>(tkFactory.fromString(tr.start_token), tkFactory.fromString(tr.end_token));
  277. for (String ep : tr.endpoints)
  278. {
  279. addRangeForEndpoint(range, InetAddress.getByName(ep));
  280. }
  281. }
  282. for (KsDef ksDef : ksDefs)
  283. {
  284. Set<String> cfs = new HashSet<String>();
  285. for (CfDef cfDef : ksDef.cf_defs)
  286. cfs.add(cfDef.name);
  287. knownCfs.put(ksDef.name, cfs);
  288. }
  289. break;
  290. }
  291. catch (Exception e)
  292. {
  293. if (!hostiter.hasNext())
  294. throw new RuntimeException("Could not retrieve endpoint ranges: ", e);
  295. }
  296. }
  297. }
  298. public boolean validateColumnFamily(String keyspace, String cfName)
  299. {
  300. Set<String> cfs = knownCfs.get(keyspace);
  301. return cfs != null && cfs.contains(cfName);
  302. }
  303. private static Cassandra.Client createThriftClient(String host, int port) throws TTransportException
  304. {
  305. TSocket socket = new TSocket(host, port);
  306. TTransport trans = new TFramedTransport(socket);
  307. trans.open();
  308. TProtocol protocol = new org.apache.thrift.protocol.TBinaryProtocol(trans);
  309. return new Cassandra.Client(protocol);
  310. }
  311. }
  312. static class NullOutputHandler implements SSTableLoader.OutputHandler
  313. {
  314. public void output(String msg) {}
  315. public void debug(String msg) {}
  316. }
  317. }