/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you 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 * * http://www.apache.org/licenses/LICENSE-2.0 * * 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. */ import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; import org.apache.cassandra.avro.Column; import org.apache.cassandra.avro.ColumnOrSuperColumn; import org.apache.cassandra.avro.Mutation; import org.apache.cassandra.hadoop.ColumnFamilyOutputFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.db.IColumn; import org.apache.cassandra.hadoop.ColumnFamilyInputFormat; import org.apache.cassandra.hadoop.ConfigHelper; import org.apache.cassandra.thrift.SlicePredicate; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; /** * This counts the occurrences of words in ColumnFamily Standard1, that has a single column (that we care about) * "text" containing a sequence of words. * * For each word, we output the total number of occurrences across all texts. */ public class WordCount extends Configured implements Tool { private static final Logger logger = LoggerFactory.getLogger(WordCount.class); static final String KEYSPACE = "Keyspace1"; static final String COLUMN_FAMILY = "Standard1"; static final String OUTPUT_REDUCER_VAR = "output_reducer"; static final String OUTPUT_COLUMN_FAMILY = "Standard2"; private static final String OUTPUT_PATH_PREFIX = "/tmp/word_count"; private static final String CONF_COLUMN_NAME = "columnname"; public static void main(String[] args) throws Exception { // Let ToolRunner handle generic command-line options ToolRunner.run(new Configuration(), new WordCount(), args); System.exit(0); } public static class TokenizerMapper extends Mapper<ByteBuffer, SortedMap<ByteBuffer, IColumn>, Text, IntWritable> { private final static IntWritable one = new IntWritable(1); private Text word = new Text(); private ByteBuffer columnName; public void map(ByteBuffer key, SortedMap<ByteBuffer, IColumn> columns, Context context) throws IOException, InterruptedException { IColumn column = columns.get(columnName); if (column == null) return; String value = ByteBufferUtil.string(column.value()); logger.debug("read " + key + ":" + value + " from " + context.getInputSplit()); StringTokenizer itr = new StringTokenizer(value); while (itr.hasMoreTokens()) { word.set(itr.nextToken()); context.write(word, one); } } protected void setup(org.apache.hadoop.mapreduce.Mapper.Context context) throws IOException, InterruptedException { this.columnName = ByteBuffer.wrap(context.getConfiguration().get(CONF_COLUMN_NAME).getBytes()); } } public static class ReducerToFilesystem extends Reducer<Text, IntWritable, Text, IntWritable> { private IntWritable result = new IntWritable(); public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException { int sum = 0; for (IntWritable val : values) { sum += val.get(); } result.set(sum); context.write(key, result); } } public static class ReducerToCassandra extends Reducer<Text, IntWritable, ByteBuffer, List<Mutation>> { private List<Mutation> results = new ArrayList<Mutation>(); private String columnName; public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException { int sum = 0; for (IntWritable val : values) { sum += val.get(); } results.add(getMutation(key, sum)); context.write(ByteBuffer.wrap(columnName.getBytes()), results); results.clear(); } protected void setup(org.apache.hadoop.mapreduce.Reducer.Context context) throws IOException, InterruptedException { this.columnName = context.getConfiguration().get(CONF_COLUMN_NAME); } private static Mutation getMutation(Text key, int sum) { Mutation m = new Mutation(); m.column_or_supercolumn = getCoSC(key, sum); return m; } private static ColumnOrSuperColumn getCoSC(Text key, int sum) { // Have to convert both the key and the sum to ByteBuffers // for the generalized output format ByteBuffer name = ByteBuffer.wrap(key.getBytes()); ByteBuffer value = ByteBuffer.wrap(String.valueOf(sum).getBytes()); Column c = new Column(); c.name = name; c.value = value; c.timestamp = System.currentTimeMillis() * 1000; c.ttl = 0; ColumnOrSuperColumn cosc = new ColumnOrSuperColumn(); cosc.column = c; return cosc; } } public int run(String[] args) throws Exception { String outputReducerType = "filesystem"; if (args != null && args[0].startsWith(OUTPUT_REDUCER_VAR)) { String[] s = args[0].split("="); if (s != null && s.length == 2) outputReducerType = s[1]; } logger.info("output reducer type: " + outputReducerType); for (int i = 0; i < WordCountSetup.TEST_COUNT; i++) { String columnName = "text" + i; getConf().set(CONF_COLUMN_NAME, columnName); Job job = new Job(getConf(), "wordcount"); job.setJarByClass(WordCount.class); job.setMapperClass(TokenizerMapper.class); if (outputReducerType.equalsIgnoreCase("filesystem")) { job.setCombinerClass(ReducerToFilesystem.class); job.setReducerClass(ReducerToFilesystem.class); job.setOutputKeyClass(Text.class); job.setOutputValueClass(IntWritable.class); FileOutputFormat.setOutputPath(job, new Path(OUTPUT_PATH_PREFIX + i)); } else { job.setReducerClass(ReducerToCassandra.class); job.setMapOutputKeyClass(Text.class); job.setMapOutputValueClass(IntWritable.class); job.setOutputKeyClass(ByteBuffer.class); job.setOutputValueClass(List.class); job.setOutputFormatClass(ColumnFamilyOutputFormat.class); ConfigHelper.setOutputColumnFamily(job.getConfiguration(), KEYSPACE, OUTPUT_COLUMN_FAMILY); } job.setInputFormatClass(ColumnFamilyInputFormat.class); ConfigHelper.setRpcPort(job.getConfiguration(), "9160"); ConfigHelper.setInitialAddress(job.getConfiguration(), "localhost"); ConfigHelper.setPartitioner(job.getConfiguration(), "org.apache.cassandra.dht.RandomPartitioner"); ConfigHelper.setInputColumnFamily(job.getConfiguration(), KEYSPACE, COLUMN_FAMILY); SlicePredicate predicate = new SlicePredicate().setColumn_names(Arrays.asList(ByteBuffer.wrap(columnName.getBytes()))); ConfigHelper.setInputSlicePredicate(job.getConfiguration(), predicate); job.waitForCompletion(true); } return 0; } }