Java tutorial
/* * Copyright 2013 Illarion Kovalchuk * <p/> * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * <p/> * http://www.apache.org/licenses/LICENSE-2.0 * <p/> * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package andromache.hadoop; import andromache.debug.MutationStringifier; import andromache.output.CassandraClientFactory; import andromache.output.CassandraTokensRangeThread; import andromache.output.RangeThreadsCache; import andromache.config.CassandraConfigHelper; import org.apache.cassandra.thrift.ConsistencyLevel; import org.apache.cassandra.thrift.Mutation; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.util.Progressable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.CharacterCodingException; import java.util.List; /** * The <code>ColumnFamilyRecordWriter</code> maps the output <key, value> * pairs to a Cassandra column family. In particular, it applies all mutations * in the value, which it associates with the key, and in turn the responsible * endpoint. * <p/> * <p> * Furthermore, this writer groups the mutations by the endpoint responsible for * the rows being affected. This allows the mutations to be executed in parallel, * directly to a responsible endpoint. * </p> * */ class CassandraRecordWriter extends RecordWriter<ByteBuffer, List<WritableMutation>> { private static final Logger log = LoggerFactory.getLogger(CassandraRecordWriter.class); public static final String BATCH_SIZE = "mapreduce.output.columnfamilyoutputformat.batch.size"; public static final String QUEUE_SIZE = "mapreduce.output.columnfamilyoutputformat.queue.size"; public static final String BATCH_DELAY_MS = "mapreduce.output.batch.delay_ms"; private final RangeThreadsCache rangeThreadsCache; private Progressable progressable; CassandraRecordWriter(TaskAttemptContext context) throws IOException { this.progressable = context; Configuration conf = context.getConfiguration(); int queueSize = conf.getInt(QUEUE_SIZE, 256); int batchSize = conf.getInt(BATCH_SIZE, 32); long batchDelay = conf.getLong(BATCH_DELAY_MS, 200); ConsistencyLevel consistencyLevel = CassandraConfigHelper.getWriteConsistencyLevel(conf); CassandraClientFactory cassandraClientFactory = new CassandraClientFactory( CassandraConfigHelper.getOutputTransportFactory(conf), CassandraConfigHelper.getOutputRpcPort(conf)); this.rangeThreadsCache = new RangeThreadsCache(conf, cassandraClientFactory, queueSize, batchSize, batchDelay, progressable, consistencyLevel); log.info("Using consistency level of {}", consistencyLevel); } /** * If the key is to be associated with a valid value, a mutation is created * for it with the given column family and columns. * </p> * * @param keyBuff the key to write. * @param value the value to write. * @throws java.io.IOException */ @Override public void write(ByteBuffer keyBuff, List<WritableMutation> value) throws IOException { if (isKeyEmpty(keyBuff)) { throwMutationsWithEmptyKey(value); } for (WritableMutation writableMutation : value) { CassandraTokensRangeThread client = rangeThreadsCache.getClient(keyBuff, writableMutation.getKeySpace()); try { client.put(keyBuff, writableMutation); } catch (InterruptedException e) { throw new IOException(e); } } progressable.progress(); } /** * Checks for null or emptiness within a key. * * @param keyBuff * @return * @throws java.nio.charset.CharacterCodingException */ private boolean isKeyEmpty(ByteBuffer keyBuff) throws CharacterCodingException { if (keyBuff == null) { return true; } boolean isEmptyBuffer = false; try { isEmptyBuffer = ByteBufferUtil.compareUnsigned(ByteBufferUtil.EMPTY_BYTE_BUFFER, keyBuff) == 0; } catch (Exception e) { isEmptyBuffer = false; // if we could not determine it, then we don't touch the logic } return isEmptyBuffer; } private void throwMutationsWithEmptyKey(List<WritableMutation> value) throws IOException { StringBuilder result = new StringBuilder(); for (WritableMutation writableMutation : value) { result.append(writableMutation.toString()); result.append("\n"); Mutation mutation = writableMutation.getMutation(); result.append(MutationStringifier.mutationToString(mutation)); result.append("\n"); } throw new IOException("Attempted to write mutations with empty key: " + result.toString()); } /** * Close this <code>RecordWriter</code> to future operations, but not before * flushing out the batched mutations. * * @param context the context of the task * @throws java.io.IOException */ @Override public void close(TaskAttemptContext context) throws IOException, InterruptedException { // close all the clients before throwing anything IOException clientException = null; for (CassandraTokensRangeThread client : rangeThreadsCache) { try { client.close(); } catch (IOException e) { clientException = e; } } if (clientException != null) { throw clientException; } } }