You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2013/06/06 21:19:13 UTC
[01/10] Add CQL3 input/output formats patch by Alex Liu;
reviewed by jbellis and Mike Schrag for CASSANDRA-4421
Updated Branches:
refs/heads/cassandra-1.2 6850cdc3e -> 56e0ad1be
refs/heads/trunk deac4cc17 -> 90052d5a4
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java
new file mode 100644
index 0000000..3939e0b
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java
@@ -0,0 +1,386 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.hadoop.AbstractColumnFamilyRecordWriter;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.Progressable;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The <code>ColumnFamilyRecordWriter</code> maps the output <key, value>
+ * pairs to a Cassandra column family. In particular, it applies the binded variables
+ * in the value to the prepared statement, which it associates with the key, and in
+ * turn the responsible endpoint.
+ *
+ * <p>
+ * Furthermore, this writer groups the cql queries by the endpoint responsible for
+ * the rows being affected. This allows the cql queries to be executed in parallel,
+ * directly to a responsible endpoint.
+ * </p>
+ *
+ * @see ColumnFamilyOutputFormat
+ */
+final class ColumnFamilyRecordWriter extends AbstractColumnFamilyRecordWriter<Map<String, ByteBuffer>, List<ByteBuffer>>
+{
+ private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyRecordWriter.class);
+
+ // handles for clients for each range running in the threadpool
+ private final Map<Range, RangeClient> clients;
+
+ // host to prepared statement id mappings
+ private ConcurrentHashMap<Cassandra.Client, Integer> preparedStatements = new ConcurrentHashMap<Cassandra.Client, Integer>();
+
+ private final String cql;
+
+ private AbstractType<?> keyValidator;
+ private String [] partitionkeys;
+
+ /**
+ * Upon construction, obtain the map that this writer will use to collect
+ * mutations, and the ring cache for the given keyspace.
+ *
+ * @param context the task attempt context
+ * @throws IOException
+ */
+ ColumnFamilyRecordWriter(TaskAttemptContext context) throws IOException
+ {
+ this(context.getConfiguration());
+ this.progressable = new Progressable(context);
+ }
+
+ ColumnFamilyRecordWriter(Configuration conf, Progressable progressable) throws IOException
+ {
+ this(conf);
+ this.progressable = progressable;
+ }
+
+ ColumnFamilyRecordWriter(Configuration conf) throws IOException
+ {
+ super(conf);
+ this.clients = new HashMap<Range, RangeClient>();
+ cql = CQLConfigHelper.getOutputCql(conf);
+
+ try
+ {
+ String host = getAnyHost();
+ int port = ConfigHelper.getOutputRpcPort(conf);
+ Cassandra.Client client = ColumnFamilyOutputFormat.createAuthenticatedClient(host, port, conf);
+ retrievePartitionKeyValidator(client);
+
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ client = null;
+ }
+ }
+ catch (Exception e)
+ {
+ throw new IOException(e);
+ }
+ }
+
+ @Override
+ public void close() throws IOException
+ {
+ // close all the clients before throwing anything
+ IOException clientException = null;
+ for (RangeClient client : clients.values())
+ {
+ try
+ {
+ client.close();
+ }
+ catch (IOException e)
+ {
+ clientException = e;
+ }
+ }
+
+ if (clientException != null)
+ throw clientException;
+ }
+
+ /**
+ * If the key is to be associated with a valid value, a mutation is created
+ * for it with the given column family and columns. In the event the value
+ * in the column is missing (i.e., null), then it is marked for
+ * {@link Deletion}. Similarly, if the entire value for a key is missing
+ * (i.e., null), then the entire key is marked for {@link Deletion}.
+ * </p>
+ *
+ * @param keybuff
+ * the key to write.
+ * @param values
+ * the values to write.
+ * @throws IOException
+ */
+ @Override
+ public void write(Map<String, ByteBuffer> keys, List<ByteBuffer> values) throws IOException
+ {
+ ByteBuffer rowKey = getRowKey(keys);
+ Range<Token> range = ringCache.getRange(rowKey);
+
+ // get the client for the given range, or create a new one
+ RangeClient client = clients.get(range);
+ if (client == null)
+ {
+ // haven't seen keys for this range: create new client
+ client = new RangeClient(ringCache.getEndpoint(range));
+ client.start();
+ clients.put(range, client);
+ }
+
+ client.put(Pair.create(rowKey, values));
+ progressable.progress();
+ }
+
+ /**
+ * A client that runs in a threadpool and connects to the list of endpoints for a particular
+ * range. Binded variable values for keys in that range are sent to this client via a queue.
+ */
+ public class RangeClient extends AbstractRangeClient<List<ByteBuffer>>
+ {
+ /**
+ * Constructs an {@link RangeClient} for the given endpoints.
+ * @param endpoints the possible endpoints to execute the mutations on
+ */
+ public RangeClient(List<InetAddress> endpoints)
+ {
+ super(endpoints);
+ }
+
+ /**
+ * Loops collecting cql binded variable values from the queue and sending to Cassandra
+ */
+ public void run()
+ {
+ outer:
+ while (run || !queue.isEmpty())
+ {
+ Pair<ByteBuffer, List<ByteBuffer>> bindVariables;
+ try
+ {
+ bindVariables = queue.take();
+ }
+ catch (InterruptedException e)
+ {
+ // re-check loop condition after interrupt
+ continue;
+ }
+
+ Iterator<InetAddress> iter = endpoints.iterator();
+ while (true)
+ {
+ // send the mutation to the last-used endpoint. first time through, this will NPE harmlessly.
+ try
+ {
+ int i = 0;
+ int itemId = preparedStatement(client);
+ while (bindVariables != null)
+ {
+ client.execute_prepared_cql3_query(itemId, bindVariables.right, ConsistencyLevel.ONE);
+ i++;
+
+ if (i >= batchThreshold)
+ break;
+
+ bindVariables = queue.poll();
+ }
+
+ break;
+ }
+ catch (Exception e)
+ {
+ closeInternal();
+ if (!iter.hasNext())
+ {
+ lastException = new IOException(e);
+ break outer;
+ }
+ }
+
+ // attempt to connect to a different endpoint
+ try
+ {
+ InetAddress address = iter.next();
+ String host = address.getHostName();
+ int port = ConfigHelper.getOutputRpcPort(conf);
+ client = ColumnFamilyOutputFormat.createAuthenticatedClient(host, port, conf);
+ }
+ catch (Exception e)
+ {
+ closeInternal();
+ // TException means something unexpected went wrong to that endpoint, so
+ // we should try again to another. Other exceptions (auth or invalid request) are fatal.
+ if ((!(e instanceof TException)) || !iter.hasNext())
+ {
+ lastException = new IOException(e);
+ break outer;
+ }
+ }
+ }
+ }
+ }
+
+ /** get prepared statement id from cache, otherwise prepare it from Cassandra server*/
+ private int preparedStatement(Cassandra.Client client)
+ {
+ Integer itemId = preparedStatements.get(client);
+ if (itemId == null)
+ {
+ CqlPreparedResult result;
+ try
+ {
+ result = client.prepare_cql3_query(ByteBufferUtil.bytes(cql), Compression.NONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ throw new RuntimeException("failed to prepare cql query " + cql, e);
+ }
+ catch (TException e)
+ {
+ throw new RuntimeException("failed to prepare cql query " + cql, e);
+ }
+
+ Integer previousId = preparedStatements.putIfAbsent(client, Integer.valueOf(result.itemId));
+ itemId = previousId == null ? result.itemId : previousId;
+ }
+ return itemId;
+ }
+ }
+
+ private ByteBuffer getRowKey(Map<String, ByteBuffer> keysMap)
+ {
+ //current row key
+ ByteBuffer rowKey;
+ if (keyValidator instanceof CompositeType)
+ {
+ ByteBuffer[] keys = new ByteBuffer[partitionkeys.length];
+ for (int i = 0; i< keys.length; i++)
+ keys[i] = keysMap.get(partitionkeys[i]);
+
+ rowKey = ((CompositeType) keyValidator).build(keys);
+ }
+ else
+ {
+ rowKey = keysMap.get(partitionkeys[0]);
+ }
+ return rowKey;
+ }
+
+ /** retrieve the key validator from system.schema_columnfamilies table */
+ private void retrievePartitionKeyValidator(Cassandra.Client client) throws Exception
+ {
+ String keyspace = ConfigHelper.getOutputKeyspace(conf);
+ String cfName = ConfigHelper.getOutputColumnFamily(conf);
+ String query = "SELECT key_validator," +
+ " key_aliases " +
+ "FROM system.schema_columnfamilies " +
+ "WHERE keyspace_name='%s' and columnfamily_name='%s'";
+ String formatted = String.format(query, keyspace, cfName);
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(formatted), Compression.NONE, ConsistencyLevel.ONE);
+
+ Column rawKeyValidator = result.rows.get(0).columns.get(0);
+ String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));
+ keyValidator = parseType(validator);
+
+ Column rawPartitionKeys = result.rows.get(0).columns.get(1);
+ String keyString = ByteBufferUtil.string(ByteBuffer.wrap(rawPartitionKeys.getValue()));
+ logger.debug("partition keys: " + keyString);
+
+ List<String> keys = FBUtilities.fromJsonList(keyString);
+ partitionkeys = new String [keys.size()];
+ int i=0;
+ for (String key: keys)
+ {
+ partitionkeys[i] = key;
+ i++;
+ }
+ }
+
+ private AbstractType<?> parseType(String type) throws IOException
+ {
+ try
+ {
+ // always treat counters like longs, specifically CCT.compose is not what we need
+ if (type != null && type.equals("org.apache.cassandra.db.marshal.CounterColumnType"))
+ return LongType.instance;
+ return TypeParser.parse(type);
+ }
+ catch (ConfigurationException e)
+ {
+ throw new IOException(e);
+ }
+ catch (SyntaxException e)
+ {
+ throw new IOException(e);
+ }
+ }
+
+ private String getAnyHost() throws IOException, InvalidRequestException, TException
+ {
+ Cassandra.Client client = ConfigHelper.getClientFromOutputAddressList(conf);
+ List<TokenRange> ring = client.describe_ring(ConfigHelper.getOutputKeyspace(conf));
+ try
+ {
+ for (TokenRange range : ring)
+ return range.endpoints.get(0);
+ }
+ finally
+ {
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ client = null;
+ }
+ }
+ throw new IOException("There are no endpoints");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java b/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java
new file mode 100644
index 0000000..0a73043
--- /dev/null
+++ b/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java
@@ -0,0 +1,70 @@
+package org.apache.cassandra.thrift;
+
+/*
+ *
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.thrift.transport.TTransport;
+
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ * Transport factory for establishing thrift connections from clients to a remote server.
+ */
+public interface TClientTransportFactory
+{
+ static final String PROPERTY_KEY = "cassandra.client.transport.factory";
+ static final String LONG_OPTION = "transport-factory";
+ static final String SHORT_OPTION = "tr";
+
+ /**
+ * Opens a client transport to a thrift server.
+ * Example:
+ *
+ * <pre>
+ * TTransport transport = clientTransportFactory.openTransport(address, port);
+ * Cassandra.Iface client = new Cassandra.Client(new BinaryProtocol(transport));
+ * </pre>
+ *
+ * @param host fully qualified hostname of the server
+ * @param port RPC port of the server
+ * @param conf Hadoop configuration
+ * @return open and ready to use transport
+ * @throws Exception implementation defined; usually throws TTransportException or IOException
+ * if the connection cannot be established
+ */
+ TTransport openTransport(String host, int port, Configuration conf) throws Exception;
+
+ /**
+ * Sets an implementation defined set of options.
+ * Keys in this map must conform to the set set returned by TClientTransportFactory#supportedOptions.
+ * @param options option map
+ */
+ void setOptions(Map<String, String> options);
+
+ /**
+ * @return set of options supported by this transport factory implementation
+ */
+ Set<String> supportedOptions();
+}
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java b/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
index 792618d..5905f4a 100644
--- a/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
+++ b/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
@@ -21,6 +21,10 @@ package org.apache.cassandra.thrift;
*
*/
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
import org.apache.cassandra.hadoop.ConfigHelper;
import org.apache.thrift.transport.TFramedTransport;
import org.apache.thrift.transport.TSocket;
@@ -29,7 +33,7 @@ import org.apache.thrift.transport.TTransportException;
import org.apache.hadoop.conf.Configuration;
-public class TFramedTransportFactory implements ITransportFactory
+public class TFramedTransportFactory implements ITransportFactory, TClientTransportFactory
{
public TTransport openTransport(TSocket socket, Configuration conf) throws TTransportException
{
@@ -37,4 +41,22 @@ public class TFramedTransportFactory implements ITransportFactory
transport.open();
return transport;
}
+
+ //
+ public TTransport openTransport(String host, int port, Configuration conf) throws TTransportException
+ {
+ TSocket socket = new TSocket(host, port);
+ TTransport transport = new TFramedTransport(socket, ConfigHelper.getThriftFramedTransportSize(conf));
+ transport.open();
+ return transport;
+ }
+
+ public void setOptions(Map<String, String> options)
+ {
+ }
+
+ public Set<String> supportedOptions()
+ {
+ return Collections.emptySet();
+ }
}
[07/10] git commit: Merge branch 'cassandra-1.2' into trunk
Posted by jb...@apache.org.
Merge branch 'cassandra-1.2' into trunk
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6a9e6d59
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6a9e6d59
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6a9e6d59
Branch: refs/heads/trunk
Commit: 6a9e6d598d92a7e9d2e1aca63cea6ead16115d19
Parents: deac4cc 56e0ad1
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Jun 6 14:18:50 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Jun 6 14:18:50 2013 -0500
----------------------------------------------------------------------
----------------------------------------------------------------------
[10/10] git commit: merge from 1.2
Posted by jb...@apache.org.
merge from 1.2
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/90052d5a
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/90052d5a
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/90052d5a
Branch: refs/heads/trunk
Commit: 90052d5a460efefd2dca7a0a5c5fbd54c9674d75
Parents: 6a9e6d5
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Jun 6 14:16:28 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Jun 6 14:18:58 2013 -0500
----------------------------------------------------------------------
examples/hadoop_cql3_word_count/README.txt | 47 +
examples/hadoop_cql3_word_count/bin/word_count | 61 ++
.../hadoop_cql3_word_count/bin/word_count_counters | 58 ++
.../hadoop_cql3_word_count/bin/word_count_setup | 61 ++
examples/hadoop_cql3_word_count/build.xml | 113 +++
.../hadoop_cql3_word_count/conf/log4j.properties | 32 +
examples/hadoop_cql3_word_count/ivy.xml | 24 +
examples/hadoop_cql3_word_count/src/WordCount.java | 236 +++++
.../src/WordCountCounters.java | 122 +++
.../hadoop_cql3_word_count/src/WordCountSetup.java | 213 ++++
.../hadoop/AbstractColumnFamilyInputFormat.java | 346 +++++++
.../hadoop/AbstractColumnFamilyOutputFormat.java | 159 +++
.../hadoop/AbstractColumnFamilyRecordWriter.java | 193 ++++
.../cassandra/hadoop/ColumnFamilyInputFormat.java | 307 +------
.../cassandra/hadoop/ColumnFamilyOutputFormat.java | 122 +---
.../cassandra/hadoop/ColumnFamilyRecordReader.java | 60 +-
.../cassandra/hadoop/ColumnFamilyRecordWriter.java | 190 +---
.../apache/cassandra/hadoop/ColumnFamilySplit.java | 4 +-
.../org/apache/cassandra/hadoop/ConfigHelper.java | 59 +-
.../org/apache/cassandra/hadoop/Progressable.java | 4 +-
.../cassandra/hadoop/cql3/CQLConfigHelper.java | 109 ++
.../hadoop/cql3/ColumnFamilyInputFormat.java | 83 ++
.../hadoop/cql3/ColumnFamilyOutputFormat.java | 78 ++
.../hadoop/cql3/ColumnFamilyRecordReader.java | 763 +++++++++++++++
.../hadoop/cql3/ColumnFamilyRecordWriter.java | 386 ++++++++
.../cassandra/thrift/TClientTransportFactory.java | 70 ++
.../cassandra/thrift/TFramedTransportFactory.java | 24 +-
27 files changed, 3272 insertions(+), 652 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/examples/hadoop_cql3_word_count/README.txt
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/README.txt b/examples/hadoop_cql3_word_count/README.txt
new file mode 100644
index 0000000..f984b02
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/README.txt
@@ -0,0 +1,47 @@
+Introduction
+============
+
+WordCount hadoop example: Inserts a bunch of words across multiple rows,
+and counts them, with RandomPartitioner. The word_count_counters example sums
+the value of counter columns for a key.
+
+The scripts in bin/ assume you are running with cwd of contrib/word_count.
+
+
+Running
+=======
+
+First build and start a Cassandra server with the default configuration*,
+then run
+
+contrib/word_count$ ant
+contrib/word_count$ bin/word_count_setup
+contrib/word_count$ bin/word_count
+contrib/word_count$ bin/word_count_counters
+
+In order to view the results in Cassandra, one can use bin/cassandra-cli and
+perform the following operations:
+$ bin/cqlsh localhost
+> use cql3_worldcount;
+> select * from output_words;
+
+The output of the word count can now be configured. In the bin/word_count
+file, you can specify the OUTPUT_REDUCER. The two options are 'filesystem'
+and 'cassandra'. The filesystem option outputs to the /tmp/word_count*
+directories. The cassandra option outputs to the 'output_words' column family
+in the 'cql3_worldcount' keyspace. 'cassandra' is the default.
+
+Read the code in src/ for more details.
+
+The word_count_counters example sums the counter columns for a row. The output
+is written to a text file in /tmp/word_count_counters.
+
+*If you want to point wordcount at a real cluster, modify the seed
+and listenaddress settings accordingly.
+
+
+Troubleshooting
+===============
+
+word_count uses conf/log4j.properties to log to wc.out.
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/examples/hadoop_cql3_word_count/bin/word_count
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/bin/word_count b/examples/hadoop_cql3_word_count/bin/word_count
new file mode 100644
index 0000000..a0c5aa0
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/bin/word_count
@@ -0,0 +1,61 @@
+#!/bin/sh
+
+# 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.
+
+cwd=`dirname $0`
+
+# Cassandra class files.
+if [ ! -d $cwd/../../../build/classes/main ]; then
+ echo "Unable to locate cassandra class files" >&2
+ exit 1
+fi
+
+# word_count Jar.
+if [ ! -e $cwd/../build/word_count.jar ]; then
+ echo "Unable to locate word_count jar" >&2
+ exit 1
+fi
+
+CLASSPATH=$CLASSPATH:$cwd/../conf
+CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/main
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/thrift
+for jar in $cwd/../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../lib/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+
+if [ -x $JAVA_HOME/bin/java ]; then
+ JAVA=$JAVA_HOME/bin/java
+else
+ JAVA=`which java`
+fi
+
+if [ "x$JAVA" = "x" ]; then
+ echo "Java executable not found (hint: set JAVA_HOME)" >&2
+ exit 1
+fi
+
+OUTPUT_REDUCER=cassandra
+
+#echo $CLASSPATH
+$JAVA -Xmx1G -ea -cp $CLASSPATH WordCount output_reducer=$OUTPUT_REDUCER
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/examples/hadoop_cql3_word_count/bin/word_count_counters
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/bin/word_count_counters b/examples/hadoop_cql3_word_count/bin/word_count_counters
new file mode 100644
index 0000000..7793477
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/bin/word_count_counters
@@ -0,0 +1,58 @@
+#!/bin/sh
+
+# 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.
+
+cwd=`dirname $0`
+
+# Cassandra class files.
+if [ ! -d $cwd/../../../build/classes/main ]; then
+ echo "Unable to locate cassandra class files" >&2
+ exit 1
+fi
+
+# word_count Jar.
+if [ ! -e $cwd/../build/word_count.jar ]; then
+ echo "Unable to locate word_count jar" >&2
+ exit 1
+fi
+
+CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/main
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/thrift
+for jar in $cwd/../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../lib/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+
+if [ -x $JAVA_HOME/bin/java ]; then
+ JAVA=$JAVA_HOME/bin/java
+else
+ JAVA=`which java`
+fi
+
+if [ "x$JAVA" = "x" ]; then
+ echo "Java executable not found (hint: set JAVA_HOME)" >&2
+ exit 1
+fi
+
+#echo $CLASSPATH
+$JAVA -Xmx1G -ea -cp $CLASSPATH WordCountCounters
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/examples/hadoop_cql3_word_count/bin/word_count_setup
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/bin/word_count_setup b/examples/hadoop_cql3_word_count/bin/word_count_setup
new file mode 100644
index 0000000..d194a45
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/bin/word_count_setup
@@ -0,0 +1,61 @@
+#!/bin/sh
+
+# 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.
+
+cwd=`dirname $0`
+
+# Cassandra class files.
+if [ ! -d $cwd/../../../build/classes/main ]; then
+ echo "Unable to locate cassandra class files" >&2
+ exit 1
+fi
+
+# word_count Jar.
+if [ ! -e $cwd/../build/word_count.jar ]; then
+ echo "Unable to locate word_count jar" >&2
+ exit 1
+fi
+
+CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
+CLASSPATH=$CLASSPATH:.:$cwd/../../../build/classes/main
+CLASSPATH=$CLASSPATH:.:$cwd/../../../build/classes/thrift
+for jar in $cwd/../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../lib/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+
+if [ -x $JAVA_HOME/bin/java ]; then
+ JAVA=$JAVA_HOME/bin/java
+else
+ JAVA=`which java`
+fi
+
+if [ "x$JAVA" = "x" ]; then
+ echo "Java executable not found (hint: set JAVA_HOME)" >&2
+ exit 1
+fi
+
+HOST=localhost
+PORT=9160
+FRAMED=true
+
+$JAVA -Xmx1G -ea -Dcassandra.host=$HOST -Dcassandra.port=$PORT -Dcassandra.framed=$FRAMED -cp $CLASSPATH WordCountSetup
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/examples/hadoop_cql3_word_count/build.xml
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/build.xml b/examples/hadoop_cql3_word_count/build.xml
new file mode 100644
index 0000000..939e1b3
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/build.xml
@@ -0,0 +1,113 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+<project default="jar" name="word_count" xmlns:ivy="antlib:org.apache.ivy.ant">
+ <property name="cassandra.dir" value="../.." />
+ <property name="cassandra.dir.lib" value="${cassandra.dir}/lib" />
+ <property name="cassandra.classes" value="${cassandra.dir}/build/classes" />
+ <property name="build.src" value="${basedir}/src" />
+ <property name="build.dir" value="${basedir}/build" />
+ <property name="ivy.lib.dir" value="${build.dir}/lib" />
+ <property name="build.classes" value="${build.dir}/classes" />
+ <property name="final.name" value="word_count" />
+ <property name="ivy.version" value="2.1.0" />
+ <property name="ivy.url"
+ value="http://repo2.maven.org/maven2/org/apache/ivy/ivy" />
+
+ <condition property="ivy.jar.exists">
+ <available file="${build.dir}/ivy-${ivy.version}.jar" />
+ </condition>
+
+ <path id="autoivy.classpath">
+ <fileset dir="${ivy.lib.dir}">
+ <include name="**/*.jar" />
+ </fileset>
+ <pathelement location="${build.dir}/ivy-${ivy.version}.jar"/>
+ </path>
+
+ <path id="wordcount.build.classpath">
+ <fileset dir="${ivy.lib.dir}">
+ <include name="**/*.jar" />
+ </fileset>
+ <!-- cassandra dependencies -->
+ <fileset dir="${cassandra.dir.lib}">
+ <include name="**/*.jar" />
+ </fileset>
+ <fileset dir="${cassandra.dir}/build/lib/jars">
+ <include name="**/*.jar" />
+ </fileset>
+ <pathelement location="${cassandra.classes}/main" />
+ <pathelement location="${cassandra.classes}/thrift" />
+ </path>
+
+ <target name="init">
+ <mkdir dir="${build.classes}" />
+ </target>
+
+ <target depends="init,ivy-retrieve-build" name="build">
+ <javac destdir="${build.classes}">
+ <src path="${build.src}" />
+ <classpath refid="wordcount.build.classpath" />
+ </javac>
+ </target>
+
+ <target name="jar" depends="build">
+ <mkdir dir="${build.classes}/META-INF" />
+ <jar jarfile="${build.dir}/${final.name}.jar">
+ <fileset dir="${build.classes}" />
+ <fileset dir="${cassandra.classes}/main" />
+ <fileset dir="${cassandra.classes}/thrift" />
+ <fileset dir="${cassandra.dir}">
+ <include name="lib/**/*.jar" />
+ </fileset>
+ <zipfileset dir="${cassandra.dir}/build/lib/jars/" prefix="lib">
+ <include name="**/*.jar" />
+ </zipfileset>
+ <fileset file="${basedir}/cassandra.yaml" />
+ </jar>
+ </target>
+
+ <target name="clean">
+ <delete dir="${build.dir}" />
+ </target>
+
+ <!--
+ Ivy Specific targets
+ to fetch Ivy and this project's dependencies
+ -->
+ <target name="ivy-download" unless="ivy.jar.exists">
+ <echo>Downloading Ivy...</echo>
+ <mkdir dir="${build.dir}" />
+ <get src="${ivy.url}/${ivy.version}/ivy-${ivy.version}.jar"
+ dest="${build.dir}/ivy-${ivy.version}.jar" usetimestamp="true" />
+ </target>
+
+ <target name="ivy-init" depends="ivy-download" unless="ivy.initialized">
+ <mkdir dir="${ivy.lib.dir}"/>
+ <taskdef resource="org/apache/ivy/ant/antlib.xml"
+ uri="antlib:org.apache.ivy.ant"
+ classpathref="autoivy.classpath"/>
+ <property name="ivy.initialized" value="true"/>
+ </target>
+
+ <target name="ivy-retrieve-build" depends="ivy-init">
+ <ivy:retrieve type="jar,source" sync="true"
+ pattern="${ivy.lib.dir}/[type]s/[artifact]-[revision].[ext]" />
+ </target>
+</project>
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/examples/hadoop_cql3_word_count/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/conf/log4j.properties b/examples/hadoop_cql3_word_count/conf/log4j.properties
new file mode 100644
index 0000000..70f7657
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/conf/log4j.properties
@@ -0,0 +1,32 @@
+# 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.
+
+log4j.rootLogger=DEBUG,stdout,F
+
+#stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%5p %d{HH:mm:ss,SSS} %m%n
+
+# log file
+log4j.appender.F=org.apache.log4j.FileAppender
+log4j.appender.F.Append=false
+log4j.appender.F.layout=org.apache.log4j.PatternLayout
+log4j.appender.F.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n
+# Edit the next line to point to your logs directory
+log4j.appender.F.File=wc.out
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/examples/hadoop_cql3_word_count/ivy.xml
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/ivy.xml b/examples/hadoop_cql3_word_count/ivy.xml
new file mode 100644
index 0000000..9d44895
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/ivy.xml
@@ -0,0 +1,24 @@
+<!--
+ ~ 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.
+ -->
+<ivy-module version="2.0">
+ <info organisation="apache-cassandra" module="word-count"/>
+ <dependencies>
+ <dependency org="org.apache.hadoop" name="hadoop-core" rev="0.20.2"/>
+ </dependencies>
+</ivy-module>
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/examples/hadoop_cql3_word_count/src/WordCount.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/src/WordCount.java b/examples/hadoop_cql3_word_count/src/WordCount.java
new file mode 100644
index 0000000..09dd9e4
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/src/WordCount.java
@@ -0,0 +1,236 @@
+/**
+ * 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 java.util.Map.Entry;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.hadoop.cql3.ColumnFamilyOutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.hadoop.cql3.CQLConfigHelper;
+import org.apache.cassandra.hadoop.cql3.ColumnFamilyInputFormat;
+import org.apache.cassandra.hadoop.ConfigHelper;
+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.Reducer.Context;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.nio.charset.CharacterCodingException;
+
+/**
+ * This counts the occurrences of words in ColumnFamily
+ * cql3_worldcount ( user_id text,
+ * category_id text,
+ * sub_category_id text,
+ * title text,
+ * body text,
+ * PRIMARY KEY (user_id, category_id, sub_category_id))
+ *
+ * For each word, we output the total number of occurrences across all body texts.
+ *
+ * When outputting to Cassandra, we write the word counts to column family
+ * output_words ( row_id1 text,
+ * row_id2 text,
+ * word text,
+ * count_num text,
+ * PRIMARY KEY ((row_id1, row_id2), word))
+ * as a {word, count} to columns: word, count_num with a row key of "word sum"
+ */
+public class WordCount extends Configured implements Tool
+{
+ private static final Logger logger = LoggerFactory.getLogger(WordCount.class);
+
+ static final String KEYSPACE = "cql3_worldcount";
+ static final String COLUMN_FAMILY = "inputs";
+
+ static final String OUTPUT_REDUCER_VAR = "output_reducer";
+ static final String OUTPUT_COLUMN_FAMILY = "output_words";
+
+ private static final String OUTPUT_PATH_PREFIX = "/tmp/word_count";
+
+ private static final String PRIMARY_KEY = "row_key";
+
+ 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<Map<String, ByteBuffer>, Map<String, ByteBuffer>, Text, IntWritable>
+ {
+ private final static IntWritable one = new IntWritable(1);
+ private Text word = new Text();
+ private ByteBuffer sourceColumn;
+
+ protected void setup(org.apache.hadoop.mapreduce.Mapper.Context context)
+ throws IOException, InterruptedException
+ {
+ }
+
+ public void map(Map<String, ByteBuffer> keys, Map<String, ByteBuffer> columns, Context context) throws IOException, InterruptedException
+ {
+ for (Entry<String, ByteBuffer> column : columns.entrySet())
+ {
+ if (!"body".equalsIgnoreCase(column.getKey()))
+ continue;
+
+ String value = ByteBufferUtil.string(column.getValue());
+
+ logger.debug("read {}:{}={} from {}",
+ new Object[] {toString(keys), column.getKey(), value, context.getInputSplit()});
+
+ StringTokenizer itr = new StringTokenizer(value);
+ while (itr.hasMoreTokens())
+ {
+ word.set(itr.nextToken());
+ context.write(word, one);
+ }
+ }
+ }
+
+ private String toString(Map<String, ByteBuffer> keys)
+ {
+ String result = "";
+ try
+ {
+ for (ByteBuffer key : keys.values())
+ result = result + ByteBufferUtil.string(key) + ":";
+ }
+ catch (CharacterCodingException e)
+ {
+ logger.error("Failed to print keys", e);
+ }
+ return result;
+ }
+ }
+
+ public static class ReducerToFilesystem extends Reducer<Text, IntWritable, Text, IntWritable>
+ {
+ public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException
+ {
+ int sum = 0;
+ for (IntWritable val : values)
+ sum += val.get();
+ context.write(key, new IntWritable(sum));
+ }
+ }
+
+ public static class ReducerToCassandra extends Reducer<Text, IntWritable, Map<String, ByteBuffer>, List<ByteBuffer>>
+ {
+ private Map<String, ByteBuffer> keys;
+ private ByteBuffer key;
+ protected void setup(org.apache.hadoop.mapreduce.Reducer.Context context)
+ throws IOException, InterruptedException
+ {
+ keys = new LinkedHashMap<String, ByteBuffer>();
+ String[] partitionKeys = context.getConfiguration().get(PRIMARY_KEY).split(",");
+ keys.put("row_id1", ByteBufferUtil.bytes(partitionKeys[0]));
+ keys.put("row_id2", ByteBufferUtil.bytes(partitionKeys[1]));
+ }
+
+ public void reduce(Text word, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException
+ {
+ int sum = 0;
+ for (IntWritable val : values)
+ sum += val.get();
+ context.write(keys, getBindVariables(word, sum));
+ }
+
+ private List<ByteBuffer> getBindVariables(Text word, int sum)
+ {
+ List<ByteBuffer> variables = new ArrayList<ByteBuffer>();
+ variables.add(keys.get("row_id1"));
+ variables.add(keys.get("row_id2"));
+ variables.add(ByteBufferUtil.bytes(word.toString()));
+ variables.add(ByteBufferUtil.bytes(String.valueOf(sum)));
+ return variables;
+ }
+ }
+
+ 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);
+
+ 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));
+ }
+ else
+ {
+ job.setReducerClass(ReducerToCassandra.class);
+
+ job.setMapOutputKeyClass(Text.class);
+ job.setMapOutputValueClass(IntWritable.class);
+ job.setOutputKeyClass(Map.class);
+ job.setOutputValueClass(List.class);
+
+ job.setOutputFormatClass(ColumnFamilyOutputFormat.class);
+
+ ConfigHelper.setOutputColumnFamily(job.getConfiguration(), KEYSPACE, OUTPUT_COLUMN_FAMILY);
+ job.getConfiguration().set(PRIMARY_KEY, "word,sum");
+ String query = "INSERT INTO " + KEYSPACE + "." + OUTPUT_COLUMN_FAMILY +
+ " (row_id1, row_id2, word, count_num) " +
+ " values (?, ?, ?, ?)";
+ CQLConfigHelper.setOutputCql(job.getConfiguration(), query);
+ ConfigHelper.setOutputInitialAddress(job.getConfiguration(), "localhost");
+ ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner");
+ }
+
+ job.setInputFormatClass(ColumnFamilyInputFormat.class);
+
+ ConfigHelper.setInputRpcPort(job.getConfiguration(), "9160");
+ ConfigHelper.setInputInitialAddress(job.getConfiguration(), "localhost");
+ ConfigHelper.setInputColumnFamily(job.getConfiguration(), KEYSPACE, COLUMN_FAMILY);
+ ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner");
+
+ CQLConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), "3");
+ //this is the user defined filter clauses, you can comment it out if you want count all titles
+ CQLConfigHelper.setInputWhereClauses(job.getConfiguration(), "title='A'");
+ job.waitForCompletion(true);
+ return 0;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/examples/hadoop_cql3_word_count/src/WordCountCounters.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/src/WordCountCounters.java b/examples/hadoop_cql3_word_count/src/WordCountCounters.java
new file mode 100644
index 0000000..1cf5539
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/src/WordCountCounters.java
@@ -0,0 +1,122 @@
+/**
+ * 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.nio.charset.CharacterCodingException;
+import java.util.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.io.LongWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.cassandra.hadoop.cql3.ColumnFamilyInputFormat;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.cql3.CQLConfigHelper;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+
+/**
+ * This sums the word count stored in the input_words_count ColumnFamily for the key "sum".
+ *
+ * Output is written to a text file.
+ */
+public class WordCountCounters extends Configured implements Tool
+{
+ private static final Logger logger = LoggerFactory.getLogger(WordCountCounters.class);
+
+ static final String COUNTER_COLUMN_FAMILY = "input_words_count";
+ private static final String OUTPUT_PATH_PREFIX = "/tmp/word_count_counters";
+
+ public static void main(String[] args) throws Exception
+ {
+ // Let ToolRunner handle generic command-line options
+ ToolRunner.run(new Configuration(), new WordCountCounters(), args);
+ System.exit(0);
+ }
+
+ public static class SumMapper extends Mapper<Map<String, ByteBuffer>, Map<String, ByteBuffer>, Text, LongWritable>
+ {
+ long sum = -1;
+ public void map(Map<String, ByteBuffer> key, Map<String, ByteBuffer> columns, Context context) throws IOException, InterruptedException
+ {
+ if (sum < 0)
+ sum = 0;
+
+ logger.debug("read " + toString(key) + ":count_num from " + context.getInputSplit());
+ sum += Long.valueOf(ByteBufferUtil.string(columns.get("count_num")));
+ }
+
+ protected void cleanup(Context context) throws IOException, InterruptedException {
+ if (sum > 0)
+ context.write(new Text("total_count"), new LongWritable(sum));
+ }
+
+ private String toString(Map<String, ByteBuffer> keys)
+ {
+ String result = "";
+ try
+ {
+ for (ByteBuffer key : keys.values())
+ result = result + ByteBufferUtil.string(key) + ":";
+ }
+ catch (CharacterCodingException e)
+ {
+ logger.error("Failed to print keys", e);
+ }
+ return result;
+ }
+ }
+
+
+ public int run(String[] args) throws Exception
+ {
+ Job job = new Job(getConf(), "wordcountcounters");
+ job.setJarByClass(WordCountCounters.class);
+ job.setMapperClass(SumMapper.class);
+
+ job.setOutputKeyClass(Text.class);
+ job.setOutputValueClass(LongWritable.class);
+ FileOutputFormat.setOutputPath(job, new Path(OUTPUT_PATH_PREFIX));
+
+ job.setInputFormatClass(ColumnFamilyInputFormat.class);
+
+ ConfigHelper.setInputRpcPort(job.getConfiguration(), "9160");
+ ConfigHelper.setInputInitialAddress(job.getConfiguration(), "localhost");
+ ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner");
+ ConfigHelper.setInputColumnFamily(job.getConfiguration(), WordCount.KEYSPACE, WordCount.OUTPUT_COLUMN_FAMILY);
+
+ CQLConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), "3");
+
+ job.waitForCompletion(true);
+ return 0;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/examples/hadoop_cql3_word_count/src/WordCountSetup.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/src/WordCountSetup.java b/examples/hadoop_cql3_word_count/src/WordCountSetup.java
new file mode 100644
index 0000000..0acb8f7
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/src/WordCountSetup.java
@@ -0,0 +1,213 @@
+/**
+ * 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.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class WordCountSetup
+{
+ private static final Logger logger = LoggerFactory.getLogger(WordCountSetup.class);
+
+ public static final int TEST_COUNT = 6;
+
+ public static void main(String[] args) throws Exception
+ {
+ Cassandra.Iface client = createConnection();
+
+ setupKeyspace(client);
+ client.set_keyspace(WordCount.KEYSPACE);
+ setupTable(client);
+ insertData(client);
+
+ System.exit(0);
+ }
+
+ private static void setupKeyspace(Cassandra.Iface client)
+ throws InvalidRequestException,
+ UnavailableException,
+ TimedOutException,
+ SchemaDisagreementException,
+ TException
+ {
+ KsDef ks;
+ try
+ {
+ ks = client.describe_keyspace(WordCount.KEYSPACE);
+ }
+ catch(NotFoundException e)
+ {
+ logger.info("set up keyspace " + WordCount.KEYSPACE);
+ String query = "CREATE KEYSPACE " + WordCount.KEYSPACE +
+ " WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1}";
+
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+
+ int magnitude = client.describe_ring(WordCount.KEYSPACE).size();
+ try
+ {
+ Thread.sleep(1000 * magnitude);
+ }
+ catch (InterruptedException ie)
+ {
+ throw new RuntimeException(ie);
+ }
+ }
+ }
+
+ private static void setupTable(Cassandra.Iface client)
+ throws InvalidRequestException,
+ UnavailableException,
+ TimedOutException,
+ SchemaDisagreementException,
+ TException
+ {
+ String query = "CREATE TABLE " + WordCount.KEYSPACE + "." + WordCount.COLUMN_FAMILY +
+ " ( user_id text," +
+ " category_id text, " +
+ " sub_category_id text," +
+ " title text," +
+ " body text," +
+ " PRIMARY KEY (user_id, category_id, sub_category_id) ) ";
+
+ try
+ {
+ logger.info("set up table " + WordCount.COLUMN_FAMILY);
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ logger.error("failed to create table " + WordCount.KEYSPACE + "." + WordCount.COLUMN_FAMILY, e);
+ }
+
+ query = "CREATE INDEX title on " + WordCount.COLUMN_FAMILY + "(title)";
+ try
+ {
+ logger.info("set up index on title column ");
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ logger.error("Failed to create index on title", e);
+ }
+
+ query = "CREATE TABLE " + WordCount.KEYSPACE + "." + WordCount.OUTPUT_COLUMN_FAMILY +
+ " ( row_id text," +
+ " word text, " +
+ " count_num text," +
+ " PRIMARY KEY (row_id, word) ) ";
+
+ try
+ {
+ logger.info("set up table " + WordCount.OUTPUT_COLUMN_FAMILY);
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ logger.error("failed to create table " + WordCount.KEYSPACE + "." + WordCount.OUTPUT_COLUMN_FAMILY, e);
+ }
+ }
+
+ private static Cassandra.Iface createConnection() throws TTransportException
+ {
+ if (System.getProperty("cassandra.host") == null || System.getProperty("cassandra.port") == null)
+ {
+ logger.warn("cassandra.host or cassandra.port is not defined, using default");
+ }
+ return createConnection(System.getProperty("cassandra.host", "localhost"),
+ Integer.valueOf(System.getProperty("cassandra.port", "9160")));
+ }
+
+ private static Cassandra.Client createConnection(String host, Integer port) throws TTransportException
+ {
+ TSocket socket = new TSocket(host, port);
+ TTransport trans = new TFramedTransport(socket);
+ trans.open();
+ TProtocol protocol = new TBinaryProtocol(trans);
+
+ return new Cassandra.Client(protocol);
+ }
+
+ private static void insertData(Cassandra.Iface client)
+ throws InvalidRequestException,
+ UnavailableException,
+ TimedOutException,
+ SchemaDisagreementException,
+ TException
+ {
+ String query = "INSERT INTO " + WordCount.COLUMN_FAMILY +
+ "(user_id, category_id, sub_category_id, title, body ) " +
+ " values (?, ?, ?, ?, ?) ";
+ CqlPreparedResult result = client.prepare_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE);
+
+ String [] title = titleData();
+ String [] body = bodyData();
+ for (int i=1; i<5; i++)
+ {
+ for (int j=1; j<444; j++)
+ {
+ for (int k=1; k<4; k++)
+ {
+ List<ByteBuffer> values = new ArrayList<ByteBuffer>();
+ values.add(ByteBufferUtil.bytes(String.valueOf(j)));
+ values.add(ByteBufferUtil.bytes(String.valueOf(i)));
+ values.add(ByteBufferUtil.bytes(String.valueOf(k)));
+ values.add(ByteBufferUtil.bytes(title[i]));
+ values.add(ByteBufferUtil.bytes(body[i]));
+ client.execute_prepared_cql3_query(result.itemId, values, ConsistencyLevel.ONE);
+ }
+ }
+ }
+ }
+
+ private static String[] bodyData()
+ { // Public domain context, source http://en.wikisource.org/wiki/If%E2%80%94
+ return new String[]{
+ "",
+ "If you can keep your head when all about you",
+ "Are losing theirs and blaming it on you",
+ "If you can trust yourself when all men doubt you,",
+ "But make allowance for their doubting too:",
+ "If you can wait and not be tired by waiting,"
+ };
+ }
+
+ private static String[] titleData()
+ { // Public domain context, source http://en.wikisource.org/wiki/If%E2%80%94
+ return new String[]{
+ "",
+ "A",
+ "B",
+ "C",
+ "D",
+ "E"
+ };
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
new file mode 100644
index 0000000..1c8fd0b
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
@@ -0,0 +1,346 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.cassandra.auth.IAuthenticator;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.thrift.AuthenticationRequest;
+import org.apache.cassandra.thrift.Cassandra;
+import org.apache.cassandra.thrift.CfSplit;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.thrift.KeyRange;
+import org.apache.cassandra.thrift.TokenRange;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<K, Y> implements org.apache.hadoop.mapred.InputFormat<K, Y>
+{
+ private static final Logger logger = LoggerFactory.getLogger(AbstractColumnFamilyInputFormat.class);
+
+ public static final String MAPRED_TASK_ID = "mapred.task.id";
+ // The simple fact that we need this is because the old Hadoop API wants us to "write"
+ // to the key and value whereas the new asks for it.
+ // I choose 8kb as the default max key size (instanciated only once), but you can
+ // override it in your jobConf with this setting.
+ public static final String CASSANDRA_HADOOP_MAX_KEY_SIZE = "cassandra.hadoop.max_key_size";
+ public static final int CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT = 8192;
+
+ private String keyspace;
+ private String cfName;
+ private IPartitioner partitioner;
+
+ protected void validateConfiguration(Configuration conf)
+ {
+ if (ConfigHelper.getInputKeyspace(conf) == null || ConfigHelper.getInputColumnFamily(conf) == null)
+ {
+ throw new UnsupportedOperationException("you must set the keyspace and columnfamily with setInputColumnFamily()");
+ }
+ if (ConfigHelper.getInputInitialAddress(conf) == null)
+ throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node with setInputInitialAddress");
+ if (ConfigHelper.getInputPartitioner(conf) == null)
+ throw new UnsupportedOperationException("You must set the Cassandra partitioner class with setInputPartitioner");
+ }
+
+ public static Cassandra.Client createAuthenticatedClient(String location, int port, Configuration conf) throws Exception
+ {
+ logger.debug("Creating authenticated client for CF input format");
+ TTransport transport = ConfigHelper.getClientTransportFactory(conf).openTransport(location, port, conf);
+ TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
+ Cassandra.Client client = new Cassandra.Client(binaryProtocol);
+
+ // log in
+ client.set_keyspace(ConfigHelper.getInputKeyspace(conf));
+ if (ConfigHelper.getInputKeyspaceUserName(conf) != null)
+ {
+ Map<String, String> creds = new HashMap<String, String>();
+ creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf));
+ creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf));
+ AuthenticationRequest authRequest = new AuthenticationRequest(creds);
+ client.login(authRequest);
+ }
+ logger.debug("Authenticated client for CF input format created successfully");
+ return client;
+ }
+
+ public List<InputSplit> getSplits(JobContext context) throws IOException
+ {
+ Configuration conf = context.getConfiguration();
+
+ validateConfiguration(conf);
+
+ // cannonical ranges and nodes holding replicas
+ List<TokenRange> masterRangeNodes = getRangeMap(conf);
+
+ keyspace = ConfigHelper.getInputKeyspace(context.getConfiguration());
+ cfName = ConfigHelper.getInputColumnFamily(context.getConfiguration());
+ partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
+ logger.debug("partitioner is " + partitioner);
+
+ // cannonical ranges, split into pieces, fetching the splits in parallel
+ ExecutorService executor = Executors.newCachedThreadPool();
+ List<InputSplit> splits = new ArrayList<InputSplit>();
+
+ try
+ {
+ List<Future<List<InputSplit>>> splitfutures = new ArrayList<Future<List<InputSplit>>>();
+ KeyRange jobKeyRange = ConfigHelper.getInputKeyRange(conf);
+ Range<Token> jobRange = null;
+ if (jobKeyRange != null)
+ {
+ if (jobKeyRange.start_key == null)
+ {
+ logger.warn("ignoring jobKeyRange specified without start_key");
+ }
+ else
+ {
+ if (!partitioner.preservesOrder())
+ throw new UnsupportedOperationException("KeyRange based on keys can only be used with a order preserving paritioner");
+ if (jobKeyRange.start_token != null)
+ throw new IllegalArgumentException("only start_key supported");
+ if (jobKeyRange.end_token != null)
+ throw new IllegalArgumentException("only start_key supported");
+ jobRange = new Range<Token>(partitioner.getToken(jobKeyRange.start_key),
+ partitioner.getToken(jobKeyRange.end_key),
+ partitioner);
+ }
+ }
+
+ for (TokenRange range : masterRangeNodes)
+ {
+ if (jobRange == null)
+ {
+ // for each range, pick a live owner and ask it to compute bite-sized splits
+ splitfutures.add(executor.submit(new SplitCallable(range, conf)));
+ }
+ else
+ {
+ Range<Token> dhtRange = new Range<Token>(partitioner.getTokenFactory().fromString(range.start_token),
+ partitioner.getTokenFactory().fromString(range.end_token),
+ partitioner);
+
+ if (dhtRange.intersects(jobRange))
+ {
+ for (Range<Token> intersection: dhtRange.intersectionWith(jobRange))
+ {
+ range.start_token = partitioner.getTokenFactory().toString(intersection.left);
+ range.end_token = partitioner.getTokenFactory().toString(intersection.right);
+ // for each range, pick a live owner and ask it to compute bite-sized splits
+ splitfutures.add(executor.submit(new SplitCallable(range, conf)));
+ }
+ }
+ }
+ }
+
+ // wait until we have all the results back
+ for (Future<List<InputSplit>> futureInputSplits : splitfutures)
+ {
+ try
+ {
+ splits.addAll(futureInputSplits.get());
+ }
+ catch (Exception e)
+ {
+ throw new IOException("Could not get input splits", e);
+ }
+ }
+ }
+ finally
+ {
+ executor.shutdownNow();
+ }
+
+ assert splits.size() > 0;
+ Collections.shuffle(splits, new Random(System.nanoTime()));
+ return splits;
+ }
+
+ /**
+ * Gets a token range and splits it up according to the suggested
+ * size into input splits that Hadoop can use.
+ */
+ class SplitCallable implements Callable<List<InputSplit>>
+ {
+
+ private final TokenRange range;
+ private final Configuration conf;
+
+ public SplitCallable(TokenRange tr, Configuration conf)
+ {
+ this.range = tr;
+ this.conf = conf;
+ }
+
+ public List<InputSplit> call() throws Exception
+ {
+ ArrayList<InputSplit> splits = new ArrayList<InputSplit>();
+ List<CfSplit> subSplits = getSubSplits(keyspace, cfName, range, conf);
+ assert range.rpc_endpoints.size() == range.endpoints.size() : "rpc_endpoints size must match endpoints size";
+ // turn the sub-ranges into InputSplits
+ String[] endpoints = range.endpoints.toArray(new String[range.endpoints.size()]);
+ // hadoop needs hostname, not ip
+ int endpointIndex = 0;
+ for (String endpoint: range.rpc_endpoints)
+ {
+ String endpoint_address = endpoint;
+ if (endpoint_address == null || endpoint_address.equals("0.0.0.0"))
+ endpoint_address = range.endpoints.get(endpointIndex);
+ endpoints[endpointIndex++] = InetAddress.getByName(endpoint_address).getHostName();
+ }
+
+ Token.TokenFactory factory = partitioner.getTokenFactory();
+ for (CfSplit subSplit : subSplits)
+ {
+ Token left = factory.fromString(subSplit.getStart_token());
+ Token right = factory.fromString(subSplit.getEnd_token());
+ Range<Token> range = new Range<Token>(left, right, partitioner);
+ List<Range<Token>> ranges = range.isWrapAround() ? range.unwrap() : ImmutableList.of(range);
+ for (Range<Token> subrange : ranges)
+ {
+ ColumnFamilySplit split =
+ new ColumnFamilySplit(
+ factory.toString(subrange.left),
+ factory.toString(subrange.right),
+ subSplit.getRow_count(),
+ endpoints);
+
+ logger.debug("adding " + split);
+ splits.add(split);
+ }
+ }
+ return splits;
+ }
+ }
+
+ private List<CfSplit> getSubSplits(String keyspace, String cfName, TokenRange range, Configuration conf) throws IOException
+ {
+ int splitsize = ConfigHelper.getInputSplitSize(conf);
+ for (int i = 0; i < range.rpc_endpoints.size(); i++)
+ {
+ String host = range.rpc_endpoints.get(i);
+
+ if (host == null || host.equals("0.0.0.0"))
+ host = range.endpoints.get(i);
+
+ try
+ {
+ Cassandra.Client client = ConfigHelper.createConnection(conf, host, ConfigHelper.getInputRpcPort(conf));
+ client.set_keyspace(keyspace);
+
+ try
+ {
+ return client.describe_splits_ex(cfName, range.start_token, range.end_token, splitsize);
+ }
+ catch (TApplicationException e)
+ {
+ // fallback to guessing split size if talking to a server without describe_splits_ex method
+ if (e.getType() == TApplicationException.UNKNOWN_METHOD)
+ {
+ List<String> splitPoints = client.describe_splits(cfName, range.start_token, range.end_token, splitsize);
+ return tokenListToSplits(splitPoints, splitsize);
+ }
+ throw e;
+ }
+ }
+ catch (IOException e)
+ {
+ logger.debug("failed connect to endpoint " + host, e);
+ }
+ catch (InvalidRequestException e)
+ {
+ throw new RuntimeException(e);
+ }
+ catch (TException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+ throw new IOException("failed connecting to all endpoints " + StringUtils.join(range.endpoints, ","));
+ }
+
+ private List<CfSplit> tokenListToSplits(List<String> splitTokens, int splitsize)
+ {
+ List<CfSplit> splits = Lists.newArrayListWithExpectedSize(splitTokens.size() - 1);
+ for (int j = 0; j < splitTokens.size() - 1; j++)
+ splits.add(new CfSplit(splitTokens.get(j), splitTokens.get(j + 1), splitsize));
+ return splits;
+ }
+
+ private List<TokenRange> getRangeMap(Configuration conf) throws IOException
+ {
+ Cassandra.Client client = ConfigHelper.getClientFromInputAddressList(conf);
+
+ List<TokenRange> map;
+ try
+ {
+ map = client.describe_ring(ConfigHelper.getInputKeyspace(conf));
+ }
+ catch (InvalidRequestException e)
+ {
+ throw new RuntimeException(e);
+ }
+ catch (TException e)
+ {
+ throw new RuntimeException(e);
+ }
+ return map;
+ }
+
+ //
+ // Old Hadoop API
+ //
+ public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException
+ {
+ TaskAttemptContext tac = new TaskAttemptContext(jobConf, new TaskAttemptID());
+ List<org.apache.hadoop.mapreduce.InputSplit> newInputSplits = this.getSplits(tac);
+ org.apache.hadoop.mapred.InputSplit[] oldInputSplits = new org.apache.hadoop.mapred.InputSplit[newInputSplits.size()];
+ for (int i = 0; i < newInputSplits.size(); i++)
+ oldInputSplits[i] = (ColumnFamilySplit)newInputSplits.get(i);
+ return oldInputSplits;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
new file mode 100644
index 0000000..5a03777
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop;
+
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.auth.IAuthenticator;
+import org.apache.cassandra.thrift.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TTransport;
+
+/**
+ * The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
+ * OutputFormat that allows reduce tasks to store keys (and corresponding
+ * values) as Cassandra rows (and respective columns) in a given
+ * ColumnFamily.
+ *
+ * <p>
+ * As is the case with the {@link ColumnFamilyInputFormat}, you need to set the
+ * Keyspace and ColumnFamily in your
+ * Hadoop job Configuration. The {@link ConfigHelper} class, through its
+ * {@link ConfigHelper#setOutputColumnFamily} method, is provided to make this
+ * simple.
+ * </p>
+ *
+ * <p>
+ * For the sake of performance, this class employs a lazy write-back caching
+ * mechanism, where its record writer batches mutations created based on the
+ * reduce's inputs (in a task-specific map), and periodically makes the changes
+ * official by sending a batch mutate request to Cassandra.
+ * </p>
+ * @param <Y>
+ */
+public abstract class AbstractColumnFamilyOutputFormat<K, Y> extends OutputFormat<K, Y> implements org.apache.hadoop.mapred.OutputFormat<K, Y>
+{
+ public static final String BATCH_THRESHOLD = "mapreduce.output.columnfamilyoutputformat.batch.threshold";
+ public static final String QUEUE_SIZE = "mapreduce.output.columnfamilyoutputformat.queue.size";
+ private static final Logger logger = LoggerFactory.getLogger(AbstractColumnFamilyOutputFormat.class);
+
+
+ /**
+ * Check for validity of the output-specification for the job.
+ *
+ * @param context
+ * information about the job
+ * @throws IOException
+ * when output should not be attempted
+ */
+ public void checkOutputSpecs(JobContext context)
+ {
+ checkOutputSpecs(context.getConfiguration());
+ }
+
+ protected void checkOutputSpecs(Configuration conf)
+ {
+ if (ConfigHelper.getOutputKeyspace(conf) == null)
+ throw new UnsupportedOperationException("You must set the keyspace with setOutputKeyspace()");
+ if (ConfigHelper.getOutputPartitioner(conf) == null)
+ throw new UnsupportedOperationException("You must set the output partitioner to the one used by your Cassandra cluster");
+ if (ConfigHelper.getOutputInitialAddress(conf) == null)
+ throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node");
+ }
+
+ /** Fills the deprecated OutputFormat interface for streaming. */
+ @Deprecated
+ public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
+ {
+ checkOutputSpecs(job);
+ }
+
+ /**
+ * The OutputCommitter for this format does not write any data to the DFS.
+ *
+ * @param context
+ * the task context
+ * @return an output committer
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
+ {
+ return new NullOutputCommitter();
+ }
+
+ /**
+ * Connects to the given server:port and returns a client based on the given socket that points to the configured
+ * keyspace, and is logged in with the configured credentials.
+ *
+ * @param host fully qualified host name to connect to
+ * @param port RPC port of the server
+ * @param conf a job configuration
+ * @return a cassandra client
+ * @throws Exception set of thrown exceptions may be implementation defined,
+ * depending on the used transport factory
+ */
+ public static Cassandra.Client createAuthenticatedClient(String host, int port, Configuration conf) throws Exception
+ {
+ logger.debug("Creating authenticated client for CF output format");
+ TTransport transport = ConfigHelper.getClientTransportFactory(conf).openTransport(host, port, conf);
+ TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
+ Cassandra.Client client = new Cassandra.Client(binaryProtocol);
+ client.set_keyspace(ConfigHelper.getOutputKeyspace(conf));
+ if (ConfigHelper.getOutputKeyspaceUserName(conf) != null)
+ {
+ Map<String, String> creds = new HashMap<String, String>();
+ creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getOutputKeyspaceUserName(conf));
+ creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getOutputKeyspacePassword(conf));
+ AuthenticationRequest authRequest = new AuthenticationRequest(creds);
+ client.login(authRequest);
+ }
+ logger.debug("Authenticated client for CF output format created successfully");
+ return client;
+ }
+
+ /**
+ * An {@link OutputCommitter} that does nothing.
+ */
+ private static class NullOutputCommitter extends OutputCommitter
+ {
+ public void abortTask(TaskAttemptContext taskContext) { }
+
+ public void cleanupJob(JobContext jobContext) { }
+
+ public void commitTask(TaskAttemptContext taskContext) { }
+
+ public boolean needsTaskCommit(TaskAttemptContext taskContext)
+ {
+ return false;
+ }
+
+ public void setupJob(JobContext jobContext) { }
+
+ public void setupTask(TaskAttemptContext taskContext) { }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
new file mode 100644
index 0000000..6428db3
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
@@ -0,0 +1,193 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop;
+
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.cassandra.client.RingCache;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.thrift.transport.TTransport;
+
+
+/**
+ * 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>
+ * 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>
+ *
+ * @see ColumnFamilyOutputFormat
+ */
+public abstract class AbstractColumnFamilyRecordWriter<K, Y> extends RecordWriter<K, Y> implements org.apache.hadoop.mapred.RecordWriter<K, Y>
+{
+ // The configuration this writer is associated with.
+ protected final Configuration conf;
+
+ // The ring cache that describes the token ranges each node in the ring is
+ // responsible for. This is what allows us to group the mutations by
+ // the endpoints they should be targeted at. The targeted endpoint
+ // essentially
+ // acts as the primary replica for the rows being affected by the mutations.
+ protected final RingCache ringCache;
+
+ // The number of mutations to buffer per endpoint
+ protected final int queueSize;
+
+ protected final long batchThreshold;
+
+ protected final ConsistencyLevel consistencyLevel;
+ protected Progressable progressable;
+
+ protected AbstractColumnFamilyRecordWriter(Configuration conf)
+ {
+ this.conf = conf;
+ this.ringCache = new RingCache(conf);
+ this.queueSize = conf.getInt(AbstractColumnFamilyOutputFormat.QUEUE_SIZE, 32 * FBUtilities.getAvailableProcessors());
+ batchThreshold = conf.getLong(AbstractColumnFamilyOutputFormat.BATCH_THRESHOLD, 32);
+ consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getWriteConsistencyLevel(conf));
+ }
+
+ /**
+ * Close this <code>RecordWriter</code> to future operations, but not before
+ * flushing out the batched mutations.
+ *
+ * @param context the context of the task
+ * @throws IOException
+ */
+ public void close(TaskAttemptContext context) throws IOException, InterruptedException
+ {
+ close();
+ }
+
+ /** Fills the deprecated RecordWriter interface for streaming. */
+ @Deprecated
+ public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
+ {
+ close();
+ }
+
+ protected abstract void close() throws IOException;
+
+ /**
+ * A client that runs in a threadpool and connects to the list of endpoints for a particular
+ * range. Mutations for keys in that range are sent to this client via a queue.
+ */
+ public abstract class AbstractRangeClient<K> extends Thread
+ {
+ // The list of endpoints for this range
+ protected final List<InetAddress> endpoints;
+ // A bounded queue of incoming mutations for this range
+ protected final BlockingQueue<Pair<ByteBuffer, K>> queue = new ArrayBlockingQueue<Pair<ByteBuffer, K>>(queueSize);
+
+ protected volatile boolean run = true;
+ // we want the caller to know if something went wrong, so we record any unrecoverable exception while writing
+ // so we can throw it on the caller's stack when he calls put() again, or if there are no more put calls,
+ // when the client is closed.
+ protected volatile IOException lastException;
+
+ protected Cassandra.Client client;
+
+ /**
+ * Constructs an {@link AbstractRangeClient} for the given endpoints.
+ * @param endpoints the possible endpoints to execute the mutations on
+ */
+ public AbstractRangeClient(List<InetAddress> endpoints)
+ {
+ super("client-" + endpoints);
+ this.endpoints = endpoints;
+ }
+
+ /**
+ * enqueues the given value to Cassandra
+ */
+ public void put(Pair<ByteBuffer, K> value) throws IOException
+ {
+ while (true)
+ {
+ if (lastException != null)
+ throw lastException;
+ try
+ {
+ if (queue.offer(value, 100, TimeUnit.MILLISECONDS))
+ break;
+ }
+ catch (InterruptedException e)
+ {
+ throw new AssertionError(e);
+ }
+ }
+ }
+
+ public void close() throws IOException
+ {
+ // stop the run loop. this will result in closeInternal being called by the time join() finishes.
+ run = false;
+ interrupt();
+ try
+ {
+ this.join();
+ }
+ catch (InterruptedException e)
+ {
+ throw new AssertionError(e);
+ }
+
+ if (lastException != null)
+ throw lastException;
+ }
+
+ protected void closeInternal()
+ {
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ }
+ }
+
+ /**
+ * Loops collecting mutations from the queue and sending to Cassandra
+ */
+ public abstract void run();
+
+ @Override
+ public String toString()
+ {
+ return "#<Client for " + endpoints.toString() + ">";
+ }
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
index 404ef60..1b5a4e2 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
@@ -18,35 +18,14 @@
package org.apache.cassandra.hadoop;
import java.io.IOException;
-import java.net.InetAddress;
import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Random;
-import java.util.SortedMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.commons.lang.StringUtils;
-import org.apache.thrift.TApplicationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.*;
import org.apache.cassandra.db.Column;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.thrift.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.*;
-import org.apache.thrift.TException;
/**
* Hadoop InputFormat allowing map/reduce against Cassandra rows within one ColumnFamily.
@@ -65,283 +44,14 @@ import org.apache.thrift.TException;
*
* The default split size is 64k rows.
*/
-public class ColumnFamilyInputFormat extends InputFormat<ByteBuffer, SortedMap<ByteBuffer, Column>>
- implements org.apache.hadoop.mapred.InputFormat<ByteBuffer, SortedMap<ByteBuffer, Column>>
+public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<ByteBuffer, SortedMap<ByteBuffer, Column>>
{
- private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyInputFormat.class);
-
- public static final String MAPRED_TASK_ID = "mapred.task.id";
- // The simple fact that we need this is because the old Hadoop API wants us to "write"
- // to the key and value whereas the new asks for it.
- // I choose 8kb as the default max key size (instanciated only once), but you can
- // override it in your jobConf with this setting.
- public static final String CASSANDRA_HADOOP_MAX_KEY_SIZE = "cassandra.hadoop.max_key_size";
- public static final int CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT = 8192;
-
- private String keyspace;
- private String cfName;
- private IPartitioner partitioner;
-
- private static void validateConfiguration(Configuration conf)
- {
- if (ConfigHelper.getInputKeyspace(conf) == null || ConfigHelper.getInputColumnFamily(conf) == null)
- {
- throw new UnsupportedOperationException("you must set the keyspace and columnfamily with setInputColumnFamily()");
- }
- if (ConfigHelper.getInputSlicePredicate(conf) == null)
- {
- throw new UnsupportedOperationException("you must set the predicate with setInputSlicePredicate");
- }
- if (ConfigHelper.getInputInitialAddress(conf) == null)
- throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node with setInputInitialAddress");
- if (ConfigHelper.getInputPartitioner(conf) == null)
- throw new UnsupportedOperationException("You must set the Cassandra partitioner class with setInputPartitioner");
- }
-
- public List<InputSplit> getSplits(JobContext context) throws IOException
- {
- Configuration conf = context.getConfiguration();
-
- validateConfiguration(conf);
-
- // cannonical ranges and nodes holding replicas
- List<TokenRange> masterRangeNodes = getRangeMap(conf);
-
- keyspace = ConfigHelper.getInputKeyspace(context.getConfiguration());
- cfName = ConfigHelper.getInputColumnFamily(context.getConfiguration());
- partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
- logger.debug("partitioner is " + partitioner);
-
- // cannonical ranges, split into pieces, fetching the splits in parallel
- ExecutorService executor = Executors.newCachedThreadPool();
- List<InputSplit> splits = new ArrayList<InputSplit>();
-
- try
- {
- List<Future<List<InputSplit>>> splitfutures = new ArrayList<Future<List<InputSplit>>>();
- KeyRange jobKeyRange = ConfigHelper.getInputKeyRange(conf);
- Range<Token> jobRange = null;
- if (jobKeyRange != null)
- {
- if (jobKeyRange.start_key == null)
- {
- logger.warn("ignoring jobKeyRange specified without start_key");
- }
- else
- {
- if (!partitioner.preservesOrder())
- throw new UnsupportedOperationException("KeyRange based on keys can only be used with a order preserving paritioner");
- if (jobKeyRange.start_token != null)
- throw new IllegalArgumentException("only start_key supported");
- if (jobKeyRange.end_token != null)
- throw new IllegalArgumentException("only start_key supported");
- jobRange = new Range<Token>(partitioner.getToken(jobKeyRange.start_key),
- partitioner.getToken(jobKeyRange.end_key),
- partitioner);
- }
- }
-
- for (TokenRange range : masterRangeNodes)
- {
- if (jobRange == null)
- {
- // for each range, pick a live owner and ask it to compute bite-sized splits
- splitfutures.add(executor.submit(new SplitCallable(range, conf)));
- }
- else
- {
- Range<Token> dhtRange = new Range<Token>(partitioner.getTokenFactory().fromString(range.start_token),
- partitioner.getTokenFactory().fromString(range.end_token),
- partitioner);
-
- if (dhtRange.intersects(jobRange))
- {
- for (Range<Token> intersection: dhtRange.intersectionWith(jobRange))
- {
- range.start_token = partitioner.getTokenFactory().toString(intersection.left);
- range.end_token = partitioner.getTokenFactory().toString(intersection.right);
- // for each range, pick a live owner and ask it to compute bite-sized splits
- splitfutures.add(executor.submit(new SplitCallable(range, conf)));
- }
- }
- }
- }
-
- // wait until we have all the results back
- for (Future<List<InputSplit>> futureInputSplits : splitfutures)
- {
- try
- {
- splits.addAll(futureInputSplits.get());
- }
- catch (Exception e)
- {
- throw new IOException("Could not get input splits", e);
- }
- }
- }
- finally
- {
- executor.shutdownNow();
- }
-
- assert splits.size() > 0;
- Collections.shuffle(splits, new Random(System.nanoTime()));
- return splits;
- }
-
- /**
- * Gets a token range and splits it up according to the suggested
- * size into input splits that Hadoop can use.
- */
- class SplitCallable implements Callable<List<InputSplit>>
- {
-
- private final TokenRange range;
- private final Configuration conf;
-
- public SplitCallable(TokenRange tr, Configuration conf)
- {
- this.range = tr;
- this.conf = conf;
- }
-
- public List<InputSplit> call() throws Exception
- {
- ArrayList<InputSplit> splits = new ArrayList<InputSplit>();
- List<CfSplit> subSplits = getSubSplits(keyspace, cfName, range, conf);
- assert range.rpc_endpoints.size() == range.endpoints.size() : "rpc_endpoints size must match endpoints size";
- // turn the sub-ranges into InputSplits
- String[] endpoints = range.endpoints.toArray(new String[range.endpoints.size()]);
- // hadoop needs hostname, not ip
- int endpointIndex = 0;
- for (String endpoint: range.rpc_endpoints)
- {
- String endpoint_address = endpoint;
- if (endpoint_address == null || endpoint_address.equals("0.0.0.0"))
- endpoint_address = range.endpoints.get(endpointIndex);
- endpoints[endpointIndex++] = InetAddress.getByName(endpoint_address).getHostName();
- }
-
- Token.TokenFactory factory = partitioner.getTokenFactory();
- for (CfSplit subSplit : subSplits)
- {
- Token left = factory.fromString(subSplit.getStart_token());
- Token right = factory.fromString(subSplit.getEnd_token());
- Range<Token> range = new Range<Token>(left, right, partitioner);
- List<Range<Token>> ranges = range.isWrapAround() ? range.unwrap() : ImmutableList.of(range);
- for (Range<Token> subrange : ranges)
- {
- ColumnFamilySplit split =
- new ColumnFamilySplit(
- factory.toString(subrange.left),
- factory.toString(subrange.right),
- subSplit.getRow_count(),
- endpoints);
-
- logger.debug("adding " + split);
- splits.add(split);
- }
- }
- return splits;
- }
- }
-
- private List<CfSplit> getSubSplits(String keyspace, String cfName, TokenRange range, Configuration conf) throws IOException
- {
- int splitsize = ConfigHelper.getInputSplitSize(conf);
- for (int i = 0; i < range.rpc_endpoints.size(); i++)
- {
- String host = range.rpc_endpoints.get(i);
-
- if (host == null || host.equals("0.0.0.0"))
- host = range.endpoints.get(i);
-
- try
- {
- Cassandra.Client client = ConfigHelper.createConnection(conf, host, ConfigHelper.getInputRpcPort(conf));
- client.set_keyspace(keyspace);
-
- try
- {
- return client.describe_splits_ex(cfName, range.start_token, range.end_token, splitsize);
- }
- catch (TApplicationException e)
- {
- // fallback to guessing split size if talking to a server without describe_splits_ex method
- if (e.getType() == TApplicationException.UNKNOWN_METHOD)
- {
- List<String> splitPoints = client.describe_splits(cfName, range.start_token, range.end_token, splitsize);
- return tokenListToSplits(splitPoints, splitsize);
- }
- throw e;
- }
- }
- catch (IOException e)
- {
- logger.debug("failed connect to endpoint " + host, e);
- }
- catch (InvalidRequestException e)
- {
- throw new RuntimeException(e);
- }
- catch (TException e)
- {
- throw new RuntimeException(e);
- }
- }
- throw new IOException("failed connecting to all endpoints " + StringUtils.join(range.endpoints, ","));
- }
-
-
- private List<CfSplit> tokenListToSplits(List<String> splitTokens, int splitsize)
- {
- List<CfSplit> splits = Lists.newArrayListWithExpectedSize(splitTokens.size() - 1);
- for (int j = 0; j < splitTokens.size() - 1; j++)
- splits.add(new CfSplit(splitTokens.get(j), splitTokens.get(j + 1), splitsize));
- return splits;
- }
-
-
- private List<TokenRange> getRangeMap(Configuration conf) throws IOException
- {
- Cassandra.Client client = ConfigHelper.getClientFromInputAddressList(conf);
-
- List<TokenRange> map;
- try
- {
- map = client.describe_ring(ConfigHelper.getInputKeyspace(conf));
- }
- catch (InvalidRequestException e)
- {
- throw new RuntimeException(e);
- }
- catch (TException e)
- {
- throw new RuntimeException(e);
- }
- return map;
- }
-
+
public RecordReader<ByteBuffer, SortedMap<ByteBuffer, Column>> createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException
{
return new ColumnFamilyRecordReader();
}
-
- //
- // Old Hadoop API
- //
- public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException
- {
- TaskAttemptContext tac = new TaskAttemptContext(jobConf, new TaskAttemptID());
- List<org.apache.hadoop.mapreduce.InputSplit> newInputSplits = this.getSplits(tac);
- org.apache.hadoop.mapred.InputSplit[] oldInputSplits = new org.apache.hadoop.mapred.InputSplit[newInputSplits.size()];
- for (int i = 0; i < newInputSplits.size(); i++)
- oldInputSplits[i] = (ColumnFamilySplit)newInputSplits.get(i);
- return oldInputSplits;
- }
-
public org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<ByteBuffer, Column>> getRecordReader(org.apache.hadoop.mapred.InputSplit split, JobConf jobConf, final Reporter reporter) throws IOException
{
TaskAttemptContext tac = new TaskAttemptContext(jobConf, TaskAttemptID.forName(jobConf.get(MAPRED_TASK_ID)))
@@ -357,5 +67,16 @@ public class ColumnFamilyInputFormat extends InputFormat<ByteBuffer, SortedMap<B
recordReader.initialize((org.apache.hadoop.mapreduce.InputSplit)split, tac);
return recordReader;
}
+
+ @Override
+ protected void validateConfiguration(Configuration conf)
+ {
+ super.validateConfiguration(conf);
+
+ if (ConfigHelper.getInputSlicePredicate(conf) == null)
+ {
+ throw new UnsupportedOperationException("you must set the predicate with setInputSlicePredicate");
+ }
+ }
}
[05/10] Add CQL3 input/output formats patch by Alex Liu;
reviewed by jbellis and Mike Schrag for CASSANDRA-4421
Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
index e95e7ad..f77352a 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
@@ -18,35 +18,14 @@
package org.apache.cassandra.hadoop;
import java.io.IOException;
-import java.net.InetAddress;
import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Random;
-import java.util.SortedMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.commons.lang.StringUtils;
-import org.apache.thrift.TApplicationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.*;
import org.apache.cassandra.db.IColumn;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.thrift.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.*;
-import org.apache.thrift.TException;
/**
* Hadoop InputFormat allowing map/reduce against Cassandra rows within one ColumnFamily.
@@ -65,283 +44,14 @@ import org.apache.thrift.TException;
*
* The default split size is 64k rows.
*/
-public class ColumnFamilyInputFormat extends InputFormat<ByteBuffer, SortedMap<ByteBuffer, IColumn>>
- implements org.apache.hadoop.mapred.InputFormat<ByteBuffer, SortedMap<ByteBuffer, IColumn>>
+public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<ByteBuffer, SortedMap<ByteBuffer, IColumn>>
{
- private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyInputFormat.class);
-
- public static final String MAPRED_TASK_ID = "mapred.task.id";
- // The simple fact that we need this is because the old Hadoop API wants us to "write"
- // to the key and value whereas the new asks for it.
- // I choose 8kb as the default max key size (instanciated only once), but you can
- // override it in your jobConf with this setting.
- public static final String CASSANDRA_HADOOP_MAX_KEY_SIZE = "cassandra.hadoop.max_key_size";
- public static final int CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT = 8192;
-
- private String keyspace;
- private String cfName;
- private IPartitioner partitioner;
-
- private static void validateConfiguration(Configuration conf)
- {
- if (ConfigHelper.getInputKeyspace(conf) == null || ConfigHelper.getInputColumnFamily(conf) == null)
- {
- throw new UnsupportedOperationException("you must set the keyspace and columnfamily with setInputColumnFamily()");
- }
- if (ConfigHelper.getInputSlicePredicate(conf) == null)
- {
- throw new UnsupportedOperationException("you must set the predicate with setInputSlicePredicate");
- }
- if (ConfigHelper.getInputInitialAddress(conf) == null)
- throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node with setInputInitialAddress");
- if (ConfigHelper.getInputPartitioner(conf) == null)
- throw new UnsupportedOperationException("You must set the Cassandra partitioner class with setInputPartitioner");
- }
-
- public List<InputSplit> getSplits(JobContext context) throws IOException
- {
- Configuration conf = context.getConfiguration();
-
- validateConfiguration(conf);
-
- // cannonical ranges and nodes holding replicas
- List<TokenRange> masterRangeNodes = getRangeMap(conf);
-
- keyspace = ConfigHelper.getInputKeyspace(context.getConfiguration());
- cfName = ConfigHelper.getInputColumnFamily(context.getConfiguration());
- partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
- logger.debug("partitioner is " + partitioner);
-
- // cannonical ranges, split into pieces, fetching the splits in parallel
- ExecutorService executor = Executors.newCachedThreadPool();
- List<InputSplit> splits = new ArrayList<InputSplit>();
-
- try
- {
- List<Future<List<InputSplit>>> splitfutures = new ArrayList<Future<List<InputSplit>>>();
- KeyRange jobKeyRange = ConfigHelper.getInputKeyRange(conf);
- Range<Token> jobRange = null;
- if (jobKeyRange != null)
- {
- if (jobKeyRange.start_key == null)
- {
- logger.warn("ignoring jobKeyRange specified without start_key");
- }
- else
- {
- if (!partitioner.preservesOrder())
- throw new UnsupportedOperationException("KeyRange based on keys can only be used with a order preserving paritioner");
- if (jobKeyRange.start_token != null)
- throw new IllegalArgumentException("only start_key supported");
- if (jobKeyRange.end_token != null)
- throw new IllegalArgumentException("only start_key supported");
- jobRange = new Range<Token>(partitioner.getToken(jobKeyRange.start_key),
- partitioner.getToken(jobKeyRange.end_key),
- partitioner);
- }
- }
-
- for (TokenRange range : masterRangeNodes)
- {
- if (jobRange == null)
- {
- // for each range, pick a live owner and ask it to compute bite-sized splits
- splitfutures.add(executor.submit(new SplitCallable(range, conf)));
- }
- else
- {
- Range<Token> dhtRange = new Range<Token>(partitioner.getTokenFactory().fromString(range.start_token),
- partitioner.getTokenFactory().fromString(range.end_token),
- partitioner);
-
- if (dhtRange.intersects(jobRange))
- {
- for (Range<Token> intersection: dhtRange.intersectionWith(jobRange))
- {
- range.start_token = partitioner.getTokenFactory().toString(intersection.left);
- range.end_token = partitioner.getTokenFactory().toString(intersection.right);
- // for each range, pick a live owner and ask it to compute bite-sized splits
- splitfutures.add(executor.submit(new SplitCallable(range, conf)));
- }
- }
- }
- }
-
- // wait until we have all the results back
- for (Future<List<InputSplit>> futureInputSplits : splitfutures)
- {
- try
- {
- splits.addAll(futureInputSplits.get());
- }
- catch (Exception e)
- {
- throw new IOException("Could not get input splits", e);
- }
- }
- }
- finally
- {
- executor.shutdownNow();
- }
-
- assert splits.size() > 0;
- Collections.shuffle(splits, new Random(System.nanoTime()));
- return splits;
- }
-
- /**
- * Gets a token range and splits it up according to the suggested
- * size into input splits that Hadoop can use.
- */
- class SplitCallable implements Callable<List<InputSplit>>
- {
-
- private final TokenRange range;
- private final Configuration conf;
-
- public SplitCallable(TokenRange tr, Configuration conf)
- {
- this.range = tr;
- this.conf = conf;
- }
-
- public List<InputSplit> call() throws Exception
- {
- ArrayList<InputSplit> splits = new ArrayList<InputSplit>();
- List<CfSplit> subSplits = getSubSplits(keyspace, cfName, range, conf);
- assert range.rpc_endpoints.size() == range.endpoints.size() : "rpc_endpoints size must match endpoints size";
- // turn the sub-ranges into InputSplits
- String[] endpoints = range.endpoints.toArray(new String[range.endpoints.size()]);
- // hadoop needs hostname, not ip
- int endpointIndex = 0;
- for (String endpoint: range.rpc_endpoints)
- {
- String endpoint_address = endpoint;
- if (endpoint_address == null || endpoint_address.equals("0.0.0.0"))
- endpoint_address = range.endpoints.get(endpointIndex);
- endpoints[endpointIndex++] = InetAddress.getByName(endpoint_address).getHostName();
- }
-
- Token.TokenFactory factory = partitioner.getTokenFactory();
- for (CfSplit subSplit : subSplits)
- {
- Token left = factory.fromString(subSplit.getStart_token());
- Token right = factory.fromString(subSplit.getEnd_token());
- Range<Token> range = new Range<Token>(left, right, partitioner);
- List<Range<Token>> ranges = range.isWrapAround() ? range.unwrap() : ImmutableList.of(range);
- for (Range<Token> subrange : ranges)
- {
- ColumnFamilySplit split =
- new ColumnFamilySplit(
- factory.toString(subrange.left),
- factory.toString(subrange.right),
- subSplit.getRow_count(),
- endpoints);
-
- logger.debug("adding " + split);
- splits.add(split);
- }
- }
- return splits;
- }
- }
-
- private List<CfSplit> getSubSplits(String keyspace, String cfName, TokenRange range, Configuration conf) throws IOException
- {
- int splitsize = ConfigHelper.getInputSplitSize(conf);
- for (int i = 0; i < range.rpc_endpoints.size(); i++)
- {
- String host = range.rpc_endpoints.get(i);
-
- if (host == null || host.equals("0.0.0.0"))
- host = range.endpoints.get(i);
-
- try
- {
- Cassandra.Client client = ConfigHelper.createConnection(conf, host, ConfigHelper.getInputRpcPort(conf));
- client.set_keyspace(keyspace);
-
- try
- {
- return client.describe_splits_ex(cfName, range.start_token, range.end_token, splitsize);
- }
- catch (TApplicationException e)
- {
- // fallback to guessing split size if talking to a server without describe_splits_ex method
- if (e.getType() == TApplicationException.UNKNOWN_METHOD)
- {
- List<String> splitPoints = client.describe_splits(cfName, range.start_token, range.end_token, splitsize);
- return tokenListToSplits(splitPoints, splitsize);
- }
- throw e;
- }
- }
- catch (IOException e)
- {
- logger.debug("failed connect to endpoint " + host, e);
- }
- catch (TException e)
- {
- throw new RuntimeException(e);
- }
- catch (InvalidRequestException e)
- {
- throw new RuntimeException(e);
- }
- }
- throw new IOException("failed connecting to all endpoints " + StringUtils.join(range.endpoints, ","));
- }
-
-
- private List<CfSplit> tokenListToSplits(List<String> splitTokens, int splitsize)
- {
- List<CfSplit> splits = Lists.newArrayListWithExpectedSize(splitTokens.size() - 1);
- for (int j = 0; j < splitTokens.size() - 1; j++)
- splits.add(new CfSplit(splitTokens.get(j), splitTokens.get(j + 1), splitsize));
- return splits;
- }
-
-
- private List<TokenRange> getRangeMap(Configuration conf) throws IOException
- {
- Cassandra.Client client = ConfigHelper.getClientFromInputAddressList(conf);
-
- List<TokenRange> map;
- try
- {
- map = client.describe_ring(ConfigHelper.getInputKeyspace(conf));
- }
- catch (TException e)
- {
- throw new RuntimeException(e);
- }
- catch (InvalidRequestException e)
- {
- throw new RuntimeException(e);
- }
- return map;
- }
-
+
public RecordReader<ByteBuffer, SortedMap<ByteBuffer, IColumn>> createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException
{
return new ColumnFamilyRecordReader();
}
-
- //
- // Old Hadoop API
- //
- public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException
- {
- TaskAttemptContext tac = new TaskAttemptContext(jobConf, new TaskAttemptID());
- List<org.apache.hadoop.mapreduce.InputSplit> newInputSplits = this.getSplits(tac);
- org.apache.hadoop.mapred.InputSplit[] oldInputSplits = new org.apache.hadoop.mapred.InputSplit[newInputSplits.size()];
- for (int i = 0; i < newInputSplits.size(); i++)
- oldInputSplits[i] = (ColumnFamilySplit)newInputSplits.get(i);
- return oldInputSplits;
- }
-
public org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<ByteBuffer, IColumn>> getRecordReader(org.apache.hadoop.mapred.InputSplit split, JobConf jobConf, final Reporter reporter) throws IOException
{
TaskAttemptContext tac = new TaskAttemptContext(jobConf, TaskAttemptID.forName(jobConf.get(MAPRED_TASK_ID)))
@@ -357,5 +67,16 @@ public class ColumnFamilyInputFormat extends InputFormat<ByteBuffer, SortedMap<B
recordReader.initialize((org.apache.hadoop.mapreduce.InputSplit)split, tac);
return recordReader;
}
+
+ @Override
+ protected void validateConfiguration(Configuration conf)
+ {
+ super.validateConfiguration(conf);
+
+ if (ConfigHelper.getInputSlicePredicate(conf) == null)
+ {
+ throw new UnsupportedOperationException("you must set the predicate with setInputSlicePredicate");
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
index b3cd516..724ba7d 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
@@ -20,22 +20,9 @@ package org.apache.cassandra.hadoop;
import java.io.IOException;
import java.nio.ByteBuffer;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.auth.IAuthenticator;
import org.apache.cassandra.thrift.*;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.*;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TSocket;
-import org.apache.thrift.transport.TTransport;
-
-import javax.security.auth.login.LoginException;
/**
* The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
@@ -58,63 +45,11 @@ import javax.security.auth.login.LoginException;
* official by sending a batch mutate request to Cassandra.
* </p>
*/
-public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutation>>
- implements org.apache.hadoop.mapred.OutputFormat<ByteBuffer,List<Mutation>>
+public class ColumnFamilyOutputFormat extends AbstractColumnFamilyOutputFormat<ByteBuffer,List<Mutation>>
{
- public static final String BATCH_THRESHOLD = "mapreduce.output.columnfamilyoutputformat.batch.threshold";
- public static final String QUEUE_SIZE = "mapreduce.output.columnfamilyoutputformat.queue.size";
- private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyOutputFormat.class);
-
-
- /**
- * Check for validity of the output-specification for the job.
- *
- * @param context
- * information about the job
- * @throws IOException
- * when output should not be attempted
- */
- @Override
- public void checkOutputSpecs(JobContext context)
- {
- checkOutputSpecs(context.getConfiguration());
- }
-
- private void checkOutputSpecs(Configuration conf)
- {
- if (ConfigHelper.getOutputKeyspace(conf) == null)
- throw new UnsupportedOperationException("You must set the keyspace with setOutputKeyspace()");
- if (ConfigHelper.getOutputPartitioner(conf) == null)
- throw new UnsupportedOperationException("You must set the output partitioner to the one used by your Cassandra cluster");
- if (ConfigHelper.getOutputInitialAddress(conf) == null)
- throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node");
- }
-
- /**
- * The OutputCommitter for this format does not write any data to the DFS.
- *
- * @param context
- * the task context
- * @return an output committer
- * @throws IOException
- * @throws InterruptedException
- */
- @Override
- public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
- {
- return new NullOutputCommitter();
- }
-
- /** Fills the deprecated OutputFormat interface for streaming. */
- @Deprecated
- public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
- {
- checkOutputSpecs(job);
- }
-
/** Fills the deprecated OutputFormat interface for streaming. */
@Deprecated
- public ColumnFamilyRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress) throws IOException
+ public ColumnFamilyRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress)
{
return new ColumnFamilyRecordWriter(job, new Progressable(progress));
}
@@ -127,62 +62,8 @@ public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutat
* @return a {@link RecordWriter} to write the output for the job.
* @throws IOException
*/
- @Override
- public ColumnFamilyRecordWriter getRecordWriter(final TaskAttemptContext context) throws IOException, InterruptedException
+ public ColumnFamilyRecordWriter getRecordWriter(final TaskAttemptContext context) throws InterruptedException
{
return new ColumnFamilyRecordWriter(context);
}
-
- /**
- * Return a client based on the given socket that points to the configured
- * keyspace, and is logged in with the configured credentials.
- *
- * @param socket a socket pointing to a particular node, seed or otherwise
- * @param conf a job configuration
- * @return a cassandra client
- * @throws InvalidRequestException
- * @throws TException
- * @throws AuthenticationException
- * @throws AuthorizationException
- */
- public static Cassandra.Client createAuthenticatedClient(TSocket socket, Configuration conf)
- throws InvalidRequestException, TException, AuthenticationException, AuthorizationException, LoginException
- {
- logger.debug("Creating authenticated client for CF output format");
- TTransport transport = ConfigHelper.getOutputTransportFactory(conf).openTransport(socket, conf);
- TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
- Cassandra.Client client = new Cassandra.Client(binaryProtocol);
- client.set_keyspace(ConfigHelper.getOutputKeyspace(conf));
- if (ConfigHelper.getOutputKeyspaceUserName(conf) != null)
- {
- Map<String, String> creds = new HashMap<String, String>();
- creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getOutputKeyspaceUserName(conf));
- creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getOutputKeyspacePassword(conf));
- AuthenticationRequest authRequest = new AuthenticationRequest(creds);
- client.login(authRequest);
- }
- logger.debug("Authenticated client for CF output format created successfully");
- return client;
- }
-
- /**
- * An {@link OutputCommitter} that does nothing.
- */
- public static class NullOutputCommitter extends OutputCommitter
- {
- public void abortTask(TaskAttemptContext taskContext) { }
-
- public void cleanupJob(JobContext jobContext) { }
-
- public void commitTask(TaskAttemptContext taskContext) { }
-
- public boolean needsTaskCommit(TaskAttemptContext taskContext)
- {
- return false;
- }
-
- public void setupJob(JobContext jobContext) { }
-
- public void setupTask(TaskAttemptContext taskContext) { }
- }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index 30abdd5..daef8ec 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -24,11 +24,9 @@ import java.nio.ByteBuffer;
import java.util.*;
import com.google.common.collect.*;
-import org.apache.thrift.transport.TTransport;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.auth.IAuthenticator;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.db.IColumn;
import org.apache.cassandra.db.marshal.AbstractType;
@@ -43,7 +41,7 @@ import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.thrift.TException;
-import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap<ByteBuffer, IColumn>>
implements org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<ByteBuffer, IColumn>>
@@ -59,14 +57,14 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
private boolean isEmptyPredicate;
private int totalRowCount; // total number of rows to fetch
private int batchSize; // fetch this many per batch
- private String cfName;
private String keyspace;
- private TSocket socket;
+ private String cfName;
private Cassandra.Client client;
private ConsistencyLevel consistencyLevel;
private int keyBufferSize = 8192;
private List<IndexExpression> filter;
+
public ColumnFamilyRecordReader()
{
this(ColumnFamilyRecordReader.CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT);
@@ -80,11 +78,11 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
public void close()
{
- if (socket != null && socket.isOpen())
+ if (client != null)
{
- socket.close();
- socket = null;
- client = null;
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
}
}
@@ -139,36 +137,25 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
predicate = ConfigHelper.getInputSlicePredicate(conf);
boolean widerows = ConfigHelper.getInputIsWide(conf);
isEmptyPredicate = isEmptyPredicate(predicate);
- totalRowCount = (int) this.split.getLength();
+ totalRowCount = (this.split.getLength() < Long.MAX_VALUE)
+ ? (int) this.split.getLength()
+ : ConfigHelper.getInputSplitSize(conf);
batchSize = ConfigHelper.getRangeBatchSize(conf);
cfName = ConfigHelper.getInputColumnFamily(conf);
consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getReadConsistencyLevel(conf));
-
keyspace = ConfigHelper.getInputKeyspace(conf);
try
{
- // only need to connect once
- if (socket != null && socket.isOpen())
+ if (client != null)
return;
// create connection using thrift
String location = getLocation();
- socket = new TSocket(location, ConfigHelper.getInputRpcPort(conf));
- TTransport transport = ConfigHelper.getInputTransportFactory(conf).openTransport(socket, conf);
- TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
- client = new Cassandra.Client(binaryProtocol);
-
- // log in
- client.set_keyspace(keyspace);
- if (ConfigHelper.getInputKeyspaceUserName(conf) != null)
- {
- Map<String, String> creds = new HashMap<String, String>();
- creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf));
- creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf));
- AuthenticationRequest authRequest = new AuthenticationRequest(creds);
- client.login(authRequest);
- }
+
+ int port = ConfigHelper.getInputRpcPort(conf);
+ client = ColumnFamilyInputFormat.createAuthenticatedClient(location, port, conf);
+
}
catch (Exception e)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
index 909c291..50ec059 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
@@ -22,21 +22,14 @@ import java.io.IOException;
import java.net.InetAddress;
import java.nio.ByteBuffer;
import java.util.*;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-import org.apache.cassandra.client.RingCache;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.thrift.TException;
-import org.apache.thrift.transport.TSocket;
/**
@@ -52,33 +45,12 @@ import org.apache.thrift.transport.TSocket;
* </p>
*
* @see ColumnFamilyOutputFormat
- * @see OutputFormat
- *
*/
-final class ColumnFamilyRecordWriter extends RecordWriter<ByteBuffer,List<Mutation>>
-implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
+final class ColumnFamilyRecordWriter extends AbstractColumnFamilyRecordWriter<ByteBuffer, List<Mutation>>
{
- // The configuration this writer is associated with.
- private final Configuration conf;
-
- // The ring cache that describes the token ranges each node in the ring is
- // responsible for. This is what allows us to group the mutations by
- // the endpoints they should be targeted at. The targeted endpoint
- // essentially
- // acts as the primary replica for the rows being affected by the mutations.
- private final RingCache ringCache;
-
- // The number of mutations to buffer per endpoint
- private final int queueSize;
-
// handles for clients for each range running in the threadpool
- private final Map<Range,RangeClient> clients;
- private final long batchThreshold;
-
- private final ConsistencyLevel consistencyLevel;
- private Progressable progressable;
-
-
+ private final Map<Range, RangeClient> clients;
+
/**
* Upon construction, obtain the map that this writer will use to collect
* mutations, and the ring cache for the given keyspace.
@@ -86,28 +58,44 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
* @param context the task attempt context
* @throws IOException
*/
- ColumnFamilyRecordWriter(TaskAttemptContext context) throws IOException
+ ColumnFamilyRecordWriter(TaskAttemptContext context)
{
this(context.getConfiguration());
this.progressable = new Progressable(context);
}
- ColumnFamilyRecordWriter(Configuration conf, Progressable progressable) throws IOException
+ ColumnFamilyRecordWriter(Configuration conf, Progressable progressable)
{
this(conf);
this.progressable = progressable;
}
- ColumnFamilyRecordWriter(Configuration conf) throws IOException
+ ColumnFamilyRecordWriter(Configuration conf)
{
- this.conf = conf;
- this.ringCache = new RingCache(conf);
- this.queueSize = conf.getInt(ColumnFamilyOutputFormat.QUEUE_SIZE, 32 * FBUtilities.getAvailableProcessors());
- this.clients = new HashMap<Range,RangeClient>();
- batchThreshold = conf.getLong(ColumnFamilyOutputFormat.BATCH_THRESHOLD, 32);
- consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getWriteConsistencyLevel(conf));
+ super(conf);
+ this.clients = new HashMap<Range, RangeClient>();
}
-
+
+ @Override
+ public void close() throws IOException
+ {
+ // close all the clients before throwing anything
+ IOException clientException = null;
+ for (RangeClient client : clients.values())
+ {
+ try
+ {
+ client.close();
+ }
+ catch (IOException e)
+ {
+ clientException = e;
+ }
+ }
+ if (clientException != null)
+ throw clientException;
+ }
+
/**
* If the key is to be associated with a valid value, a mutation is created
* for it with the given column family and columns. In the event the value
@@ -143,124 +131,22 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
}
/**
- * Close this <code>RecordWriter</code> to future operations, but not before
- * flushing out the batched mutations.
- *
- * @param context the context of the task
- * @throws IOException
- */
- @Override
- public void close(TaskAttemptContext context) throws IOException, InterruptedException
- {
- close();
- }
-
- /** Fills the deprecated RecordWriter interface for streaming. */
- @Deprecated
- public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
- {
- close();
- }
-
- private void close() throws IOException
- {
- // close all the clients before throwing anything
- IOException clientException = null;
- for (RangeClient client : clients.values())
- {
- try
- {
- client.close();
- }
- catch (IOException e)
- {
- clientException = e;
- }
- }
- if (clientException != null)
- throw clientException;
- }
-
- /**
* A client that runs in a threadpool and connects to the list of endpoints for a particular
* range. Mutations for keys in that range are sent to this client via a queue.
*/
- public class RangeClient extends Thread
+ public class RangeClient extends AbstractRangeClient<Mutation>
{
- // The list of endpoints for this range
- private final List<InetAddress> endpoints;
- private final String columnFamily = ConfigHelper.getOutputColumnFamily(conf);
- // A bounded queue of incoming mutations for this range
- private final BlockingQueue<Pair<ByteBuffer, Mutation>> queue = new ArrayBlockingQueue<Pair<ByteBuffer,Mutation>>(queueSize);
-
- private volatile boolean run = true;
- // we want the caller to know if something went wrong, so we record any unrecoverable exception while writing
- // so we can throw it on the caller's stack when he calls put() again, or if there are no more put calls,
- // when the client is closed.
- private volatile IOException lastException;
-
- private Cassandra.Client thriftClient;
- private TSocket thriftSocket;
-
+ public final String columnFamily = ConfigHelper.getOutputColumnFamily(conf);
+
/**
* Constructs an {@link RangeClient} for the given endpoints.
* @param endpoints the possible endpoints to execute the mutations on
*/
public RangeClient(List<InetAddress> endpoints)
{
- super("client-" + endpoints);
- this.endpoints = endpoints;
+ super(endpoints);
}
-
- /**
- * enqueues the given value to Cassandra
- */
- public void put(Pair<ByteBuffer,Mutation> value) throws IOException
- {
- while (true)
- {
- if (lastException != null)
- throw lastException;
- try
- {
- if (queue.offer(value, 100, TimeUnit.MILLISECONDS))
- break;
- }
- catch (InterruptedException e)
- {
- throw new AssertionError(e);
- }
- }
- }
-
- public void close() throws IOException
- {
- // stop the run loop. this will result in closeInternal being called by the time join() finishes.
- run = false;
- interrupt();
- try
- {
- this.join();
- }
- catch (InterruptedException e)
- {
- throw new AssertionError(e);
- }
-
- if (lastException != null)
- throw lastException;
- }
-
- private void closeInternal()
- {
- if (thriftSocket != null)
- {
- thriftSocket.close();
- thriftSocket = null;
- thriftClient = null;
- }
- }
-
+
/**
* Loops collecting mutations from the queue and sending to Cassandra
*/
@@ -303,7 +189,7 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
// send the mutation to the last-used endpoint. first time through, this will NPE harmlessly.
try
{
- thriftClient.batch_mutate(batch, consistencyLevel);
+ client.batch_mutate(batch, consistencyLevel);
break;
}
catch (Exception e)
@@ -320,8 +206,9 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
try
{
InetAddress address = iter.next();
- thriftSocket = new TSocket(address.getHostName(), ConfigHelper.getOutputRpcPort(conf));
- thriftClient = ColumnFamilyOutputFormat.createAuthenticatedClient(thriftSocket, conf);
+ String host = address.getHostName();
+ int port = ConfigHelper.getOutputRpcPort(conf);
+ client = ColumnFamilyOutputFormat.createAuthenticatedClient(host, port, conf);
}
catch (Exception e)
{
@@ -337,11 +224,5 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
}
}
}
-
- @Override
- public String toString()
- {
- return "#<Client for " + endpoints.toString() + ">";
- }
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
index ed1f160..69c7ddb 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.mapreduce.InputSplit;
import java.io.DataInput;
import java.io.DataOutput;
+import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
@@ -79,7 +80,6 @@ public class ColumnFamilySplit extends InputSplit implements Writable, org.apach
{
out.writeUTF(startToken);
out.writeUTF(endToken);
- out.writeLong(length);
out.writeInt(dataNodes.length);
for (String endpoint : dataNodes)
{
@@ -91,8 +91,6 @@ public class ColumnFamilySplit extends InputSplit implements Writable, org.apach
{
startToken = in.readUTF();
endToken = in.readUTF();
- length = in.readLong();
-
int numOfEndpoints = in.readInt();
dataNodes = new String[numOfEndpoints];
for(int i = 0; i < numOfEndpoints; i++)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
index 0d12812..3dcfdd7 100644
--- a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
+++ b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
@@ -21,11 +21,9 @@ package org.apache.cassandra.hadoop;
*/
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
+import com.google.common.collect.Maps;
import org.apache.cassandra.io.compress.CompressionParameters;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -40,11 +38,8 @@ import org.apache.thrift.TBase;
import org.apache.thrift.TDeserializer;
import org.apache.thrift.TException;
import org.apache.thrift.TSerializer;
-import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-
-import javax.security.auth.login.LoginException;
public class ConfigHelper
@@ -74,6 +69,7 @@ public class ConfigHelper
private static final String WRITE_CONSISTENCY_LEVEL = "cassandra.consistencylevel.write";
private static final String OUTPUT_COMPRESSION_CLASS = "cassandra.output.compression.class";
private static final String OUTPUT_COMPRESSION_CHUNK_LENGTH = "cassandra.output.compression.length";
+
private static final String INPUT_TRANSPORT_FACTORY_CLASS = "cassandra.input.transport.factory.class";
private static final String OUTPUT_TRANSPORT_FACTORY_CLASS = "cassandra.output.transport.factory.class";
private static final String THRIFT_FRAMED_TRANSPORT_SIZE_IN_MB = "cassandra.thrift.framed.size_mb";
@@ -495,19 +491,13 @@ public class ConfigHelper
/**
* @param conf The configuration to use.
- * @return Value (converts MBs to Bytes) set by {@link setThriftFramedTransportSizeInMb(Configuration, int)} or default of 15MB
+ * @return Value (converts MBs to Bytes) set by {@link #setThriftFramedTransportSizeInMb(Configuration, int)} or default of 15MB
*/
public static int getThriftFramedTransportSize(Configuration conf)
{
return conf.getInt(THRIFT_FRAMED_TRANSPORT_SIZE_IN_MB, 15) * 1024 * 1024; // 15MB is default in Cassandra
}
- @Deprecated
- public static void setThriftMaxMessageLengthInMb(Configuration conf, int maxMessageSizeInMB)
- {
- // SEE CASSANDRA-5529
- }
-
public static CompressionParameters getOutputCompressionParamaters(Configuration conf)
{
if (getOutputCompressionClass(conf) == null)
@@ -567,48 +557,44 @@ public class ConfigHelper
{
try
{
- TSocket socket = new TSocket(host, port);
- TTransport transport = getInputTransportFactory(conf).openTransport(socket, conf);
+ TTransport transport = getClientTransportFactory(conf).openTransport(host, port, conf);
return new Cassandra.Client(new TBinaryProtocol(transport, true, true));
}
- catch (LoginException e)
- {
- throw new IOException("Unable to login to server " + host + ":" + port, e);
- }
- catch (TTransportException e)
+ catch (Exception e)
{
throw new IOException("Unable to connect to server " + host + ":" + port, e);
}
}
- public static ITransportFactory getInputTransportFactory(Configuration conf)
- {
- return getTransportFactory(conf.get(INPUT_TRANSPORT_FACTORY_CLASS, TFramedTransportFactory.class.getName()));
- }
-
- public static void setInputTransportFactoryClass(Configuration conf, String classname)
- {
- conf.set(INPUT_TRANSPORT_FACTORY_CLASS, classname);
- }
-
- public static ITransportFactory getOutputTransportFactory(Configuration conf)
- {
- return getTransportFactory(conf.get(OUTPUT_TRANSPORT_FACTORY_CLASS, TFramedTransportFactory.class.getName()));
- }
-
- public static void setOutputTransportFactoryClass(Configuration conf, String classname)
+ public static TClientTransportFactory getClientTransportFactory(Configuration conf)
{
- conf.set(OUTPUT_TRANSPORT_FACTORY_CLASS, classname);
+ String factoryClassName = conf.get(
+ TClientTransportFactory.PROPERTY_KEY,
+ TFramedTransportFactory.class.getName());
+ TClientTransportFactory factory = getClientTransportFactory(factoryClassName);
+ Map<String, String> options = getOptions(conf, factory.supportedOptions());
+ factory.setOptions(options);
+ return factory;
}
- private static ITransportFactory getTransportFactory(String factoryClassName) {
+ private static TClientTransportFactory getClientTransportFactory(String factoryClassName) {
try
{
- return (ITransportFactory) Class.forName(factoryClassName).newInstance();
+ return (TClientTransportFactory) Class.forName(factoryClassName).newInstance();
}
catch (Exception e)
{
throw new RuntimeException("Failed to instantiate transport factory:" + factoryClassName, e);
}
}
+ private static Map<String, String> getOptions(Configuration conf, Set<String> supportedOptions) {
+ Map<String, String> options = Maps.newHashMap();
+ for (String optionKey : supportedOptions)
+ {
+ String optionValue = conf.get(optionKey);
+ if (optionValue != null)
+ options.put(optionKey, optionValue);
+ }
+ return options;
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/Progressable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/Progressable.java b/src/java/org/apache/cassandra/hadoop/Progressable.java
index 091a828..ac253ef 100644
--- a/src/java/org/apache/cassandra/hadoop/Progressable.java
+++ b/src/java/org/apache/cassandra/hadoop/Progressable.java
@@ -29,12 +29,12 @@ public class Progressable
private TaskAttemptContext context;
private org.apache.hadoop.util.Progressable progressable;
- Progressable(TaskAttemptContext context)
+ public Progressable(TaskAttemptContext context)
{
this.context = context;
}
- Progressable(org.apache.hadoop.util.Progressable progressable)
+ public Progressable(org.apache.hadoop.util.Progressable progressable)
{
this.progressable = progressable;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java b/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java
new file mode 100644
index 0000000..66bcfdb
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java
@@ -0,0 +1,109 @@
+package org.apache.cassandra.hadoop.cql3;
+/*
+*
+* 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 org.apache.hadoop.conf.Configuration;
+
+public class CQLConfigHelper
+{
+ private static final String INPUT_CQL_COLUMNS_CONFIG = "cassandra.input.columnfamily.columns"; // separate by colon ,
+ private static final String INPUT_CQL_PAGE_ROW_SIZE_CONFIG = "cassandra.input.page.row.size";
+ private static final String INPUT_CQL_WHERE_CLAUSE_CONFIG = "cassandra.input.where.clause";
+ private static final String OUTPUT_CQL = "cassandra.output.cql";
+
+ /**
+ * Set the CQL columns for the input of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param columns
+ */
+ public static void setInputColumns(Configuration conf, String columns)
+ {
+ if (columns == null || columns.isEmpty())
+ return;
+
+ conf.set(INPUT_CQL_COLUMNS_CONFIG, columns);
+ }
+
+ /**
+ * Set the CQL query Limit for the input of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param cqlPageRowSize
+ */
+ public static void setInputCQLPageRowSize(Configuration conf, String cqlPageRowSize)
+ {
+ if (cqlPageRowSize == null)
+ {
+ throw new UnsupportedOperationException("cql page row size may not be null");
+ }
+
+ conf.set(INPUT_CQL_PAGE_ROW_SIZE_CONFIG, cqlPageRowSize);
+ }
+
+ /**
+ * Set the CQL user defined where clauses for the input of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param clauses
+ */
+ public static void setInputWhereClauses(Configuration conf, String clauses)
+ {
+ if (clauses == null || clauses.isEmpty())
+ return;
+
+ conf.set(INPUT_CQL_WHERE_CLAUSE_CONFIG, clauses);
+ }
+
+ /**
+ * Set the CQL prepared statement for the output of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param cql
+ */
+ public static void setOutputCql(Configuration conf, String cql)
+ {
+ if (cql == null || cql.isEmpty())
+ return;
+
+ conf.set(OUTPUT_CQL, cql);
+ }
+
+
+ public static String getInputcolumns(Configuration conf)
+ {
+ return conf.get(INPUT_CQL_COLUMNS_CONFIG);
+ }
+
+ public static String getInputPageRowSize(Configuration conf)
+ {
+ return conf.get(INPUT_CQL_PAGE_ROW_SIZE_CONFIG);
+ }
+
+ public static String getInputWhereClauses(Configuration conf)
+ {
+ return conf.get(INPUT_CQL_WHERE_CLAUSE_CONFIG);
+ }
+
+ public static String getOutputCql(Configuration conf)
+ {
+ return conf.get(OUTPUT_CQL);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java
new file mode 100644
index 0000000..525ed89
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import org.apache.cassandra.hadoop.AbstractColumnFamilyInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+
+/**
+ * Hadoop InputFormat allowing map/reduce against Cassandra rows within one ColumnFamily.
+ *
+ * At minimum, you need to set the KS and CF in your Hadoop job Configuration.
+ * The ConfigHelper class is provided to make this
+ * simple:
+ * ConfigHelper.setInputColumnFamily
+ *
+ * You can also configure the number of rows per InputSplit with
+ * ConfigHelper.setInputSplitSize. The default split size is 64k rows.
+ * the number of CQL rows per page
+ *
+ * the number of CQL rows per page
+ * CQLConfigHelper.setInputCQLPageRowSize. The default page row size is 1000. You
+ * should set it to "as big as possible, but no bigger." It set the LIMIT for the CQL
+ * query, so you need set it big enough to minimize the network overhead, and also
+ * not too big to avoid out of memory issue.
+ *
+ * the column names of the select CQL query. The default is all columns
+ * CQLConfigHelper.setInputColumns
+ *
+ * the user defined the where clause
+ * CQLConfigHelper.setInputWhereClauses. The default is no user defined where clause
+ */
+public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<Map<String, ByteBuffer>, Map<String, ByteBuffer>>
+{
+ public RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>> getRecordReader(InputSplit split, JobConf jobConf, final Reporter reporter)
+ throws IOException
+ {
+ TaskAttemptContext tac = new TaskAttemptContext(jobConf, TaskAttemptID.forName(jobConf.get(MAPRED_TASK_ID)))
+ {
+ @Override
+ public void progress()
+ {
+ reporter.progress();
+ }
+ };
+
+ ColumnFamilyRecordReader recordReader = new ColumnFamilyRecordReader();
+ recordReader.initialize((org.apache.hadoop.mapreduce.InputSplit)split, tac);
+ return recordReader;
+ }
+
+ @Override
+ public org.apache.hadoop.mapreduce.RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>> createRecordReader(
+ org.apache.hadoop.mapreduce.InputSplit arg0, TaskAttemptContext arg1) throws IOException,
+ InterruptedException
+ {
+ return new ColumnFamilyRecordReader();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java
new file mode 100644
index 0000000..3f6e2af
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.hadoop.AbstractColumnFamilyOutputFormat;
+import org.apache.cassandra.hadoop.ColumnFamilyInputFormat;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.Progressable;
+import org.apache.hadoop.mapreduce.*;
+
+/**
+ * The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
+ * OutputFormat that allows reduce tasks to store keys (and corresponding
+ * binded variable values) as CQL rows (and respective columns) in a given
+ * ColumnFamily.
+ *
+ * <p>
+ * As is the case with the {@link ColumnFamilyInputFormat}, you need to set the
+ * prepared statement in your
+ * Hadoop job Configuration. The {@link CQLConfigHelper} class, through its
+ * {@link ConfigHelper#setOutputPreparedStatement} method, is provided to make this
+ * simple.
+ * you need to set the Keyspace. The {@link ConfigHelper} class, through its
+ * {@link ConfigHelper#setOutputColumnFamily} method, is provided to make this
+ * simple.
+ * </p>
+ *
+ * <p>
+ * For the sake of performance, this class employs a lazy write-back caching
+ * mechanism, where its record writer prepared statement binded variable values
+ * created based on the reduce's inputs (in a task-specific map), and periodically
+ * makes the changes official by sending a execution of prepared statement request
+ * to Cassandra.
+ * </p>
+ */
+public class ColumnFamilyOutputFormat extends AbstractColumnFamilyOutputFormat<Map<String, ByteBuffer>, List<ByteBuffer>>
+{
+ /** Fills the deprecated OutputFormat interface for streaming. */
+ @Deprecated
+ public ColumnFamilyRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress) throws IOException
+ {
+ return new ColumnFamilyRecordWriter(job, new Progressable(progress));
+ }
+
+ /**
+ * Get the {@link RecordWriter} for the given task.
+ *
+ * @param context
+ * the information about the current task.
+ * @return a {@link RecordWriter} to write the output for the job.
+ * @throws IOException
+ */
+ public ColumnFamilyRecordWriter getRecordWriter(final TaskAttemptContext context) throws IOException, InterruptedException
+ {
+ return new ColumnFamilyRecordWriter(context);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java
new file mode 100644
index 0000000..03d7af5
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java
@@ -0,0 +1,763 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.util.*;
+
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.hadoop.ColumnFamilySplit;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+
+/**
+ * Hadoop RecordReader read the values return from the CQL query
+ * It use CQL key range query to page through the wide rows.
+ * <p/>
+ * Return List<IColumn> as keys columns
+ * <p/>
+ * Map<ByteBuffer, IColumn> as column name to columns mappings
+ */
+public class ColumnFamilyRecordReader extends RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>>
+ implements org.apache.hadoop.mapred.RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>>
+{
+ private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyRecordReader.class);
+
+ public static final int DEFAULT_CQL_PAGE_LIMIT = 1000; // TODO: find the number large enough but not OOM
+
+ private ColumnFamilySplit split;
+ private RowIterator rowIterator;
+
+ private Pair<Map<String, ByteBuffer>, Map<String, ByteBuffer>> currentRow;
+ private int totalRowCount; // total number of rows to fetch
+ private String keyspace;
+ private String cfName;
+ private Cassandra.Client client;
+ private ConsistencyLevel consistencyLevel;
+
+ // partition keys -- key aliases
+ private List<BoundColumn> partitionBoundColumns = new ArrayList<BoundColumn>();
+
+ // cluster keys -- column aliases
+ private List<BoundColumn> clusterColumns = new ArrayList<BoundColumn>();
+
+ // map prepared query type to item id
+ private Map<Integer, Integer> preparedQueryIds = new HashMap<Integer, Integer>();
+
+ // cql query select columns
+ private String columns;
+
+ // the number of cql rows per page
+ private int pageRowSize;
+
+ // user defined where clauses
+ private String userDefinedWhereClauses;
+
+ private IPartitioner partitioner;
+
+ private AbstractType<?> keyValidator;
+
+ public ColumnFamilyRecordReader()
+ {
+ super();
+ }
+
+ public void initialize(InputSplit split, TaskAttemptContext context) throws IOException
+ {
+ this.split = (ColumnFamilySplit) split;
+ Configuration conf = context.getConfiguration();
+ totalRowCount = (this.split.getLength() < Long.MAX_VALUE)
+ ? (int) this.split.getLength()
+ : ConfigHelper.getInputSplitSize(conf);
+ cfName = ConfigHelper.getInputColumnFamily(conf);
+ consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getReadConsistencyLevel(conf));
+ keyspace = ConfigHelper.getInputKeyspace(conf);
+ columns = CQLConfigHelper.getInputcolumns(conf);
+ userDefinedWhereClauses = CQLConfigHelper.getInputWhereClauses(conf);
+
+ try
+ {
+ pageRowSize = Integer.parseInt(CQLConfigHelper.getInputPageRowSize(conf));
+ }
+ catch (NumberFormatException e)
+ {
+ pageRowSize = DEFAULT_CQL_PAGE_LIMIT;
+ }
+
+ partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
+
+ try
+ {
+ if (client != null)
+ return;
+
+ // create connection using thrift
+ String location = getLocation();
+
+ int port = ConfigHelper.getInputRpcPort(conf);
+ client = ColumnFamilyInputFormat.createAuthenticatedClient(location, port, conf);
+
+ // retrieve partition keys and cluster keys from system.schema_columnfamilies table
+ retrieveKeys();
+
+ client.set_keyspace(keyspace);
+ }
+ catch (Exception e)
+ {
+ throw new RuntimeException(e);
+ }
+
+ rowIterator = new RowIterator();
+
+ logger.debug("created {}", rowIterator);
+ }
+
+ public void close()
+ {
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ client = null;
+ }
+ }
+
+ public Map<String, ByteBuffer> getCurrentKey()
+ {
+ return currentRow.left;
+ }
+
+ public Map<String, ByteBuffer> getCurrentValue()
+ {
+ return currentRow.right;
+ }
+
+ public float getProgress()
+ {
+ if (!rowIterator.hasNext())
+ return 1.0F;
+
+ // the progress is likely to be reported slightly off the actual but close enough
+ float progress = ((float) rowIterator.totalRead / totalRowCount);
+ return progress > 1.0F ? 1.0F : progress;
+ }
+
+ public boolean nextKeyValue() throws IOException
+ {
+ if (!rowIterator.hasNext())
+ {
+ logger.debug("Finished scanning " + rowIterator.totalRead + " rows (estimate was: " + totalRowCount + ")");
+ return false;
+ }
+
+ try
+ {
+ currentRow = rowIterator.next();
+ }
+ catch (Exception e)
+ {
+ // throw it as IOException, so client can catch it and handle it at client side
+ IOException ioe = new IOException(e.getMessage());
+ ioe.initCause(ioe.getCause());
+ throw ioe;
+ }
+ return true;
+ }
+
+ // we don't use endpointsnitch since we are trying to support hadoop nodes that are
+ // not necessarily on Cassandra machines, too. This should be adequate for single-DC clusters, at least.
+ private String getLocation()
+ {
+ Collection<InetAddress> localAddresses = FBUtilities.getAllLocalAddresses();
+
+ for (InetAddress address : localAddresses)
+ {
+ for (String location : split.getLocations())
+ {
+ InetAddress locationAddress;
+ try
+ {
+ locationAddress = InetAddress.getByName(location);
+ }
+ catch (UnknownHostException e)
+ {
+ throw new AssertionError(e);
+ }
+ if (address.equals(locationAddress))
+ {
+ return location;
+ }
+ }
+ }
+ return split.getLocations()[0];
+ }
+
+ // Because the old Hadoop API wants us to write to the key and value
+ // and the new asks for them, we need to copy the output of the new API
+ // to the old. Thus, expect a small performance hit.
+ // And obviously this wouldn't work for wide rows. But since ColumnFamilyInputFormat
+ // and ColumnFamilyRecordReader don't support them, it should be fine for now.
+ public boolean next(Map<String, ByteBuffer> keys, Map<String, ByteBuffer> value) throws IOException
+ {
+ if (nextKeyValue())
+ {
+ value.clear();
+ value.putAll(getCurrentValue());
+
+ keys.clear();
+ keys.putAll(getCurrentKey());
+
+ return true;
+ }
+ return false;
+ }
+
+ public long getPos() throws IOException
+ {
+ return (long) rowIterator.totalRead;
+ }
+
+ public Map<String, ByteBuffer> createKey()
+ {
+ return new LinkedHashMap<String, ByteBuffer>();
+ }
+
+ public Map<String, ByteBuffer> createValue()
+ {
+ return new LinkedHashMap<String, ByteBuffer>();
+ }
+
+ /** CQL row iterator */
+ private class RowIterator extends AbstractIterator<Pair<Map<String, ByteBuffer>, Map<String, ByteBuffer>>>
+ {
+ protected int totalRead = 0; // total number of cf rows read
+ protected Iterator<CqlRow> rows;
+ private int pageRows = 0; // the number of cql rows read of this page
+ private String previousRowKey = null; // previous CF row key
+ private String partitionKeyString; // keys in <key1>, <key2>, <key3> string format
+ private String partitionKeyMarkers; // question marks in ? , ? , ? format which matches the number of keys
+
+ public RowIterator()
+ {
+ // initial page
+ executeQuery();
+ }
+
+ protected Pair<Map<String, ByteBuffer>, Map<String, ByteBuffer>> computeNext()
+ {
+ if (rows == null)
+ return endOfData();
+
+ int index = -2;
+ //check there are more page to read
+ while (!rows.hasNext())
+ {
+ // no more data
+ if (index == -1 || emptyPartitionKeyValues())
+ {
+ logger.debug("no more data.");
+ return endOfData();
+ }
+
+ index = setTailNull(clusterColumns);
+ logger.debug("set tail to null, index: " + index);
+ executeQuery();
+ pageRows = 0;
+
+ if (rows == null || !rows.hasNext() && index < 0)
+ {
+ logger.debug("no more data.");
+ return endOfData();
+ }
+ }
+
+ Map<String, ByteBuffer> valueColumns = createValue();
+ Map<String, ByteBuffer> keyColumns = createKey();
+ int i = 0;
+ CqlRow row = rows.next();
+ for (Column column : row.columns)
+ {
+ String columnName = stringValue(ByteBuffer.wrap(column.getName()));
+ logger.debug("column: " + columnName);
+
+ if (i < partitionBoundColumns.size() + clusterColumns.size())
+ keyColumns.put(stringValue(column.name), column.value);
+ else
+ valueColumns.put(stringValue(column.name), column.value);
+
+ i++;
+ }
+
+ // increase total CQL row read for this page
+ pageRows++;
+
+ // increase total CF row read
+ if (newRow(keyColumns, previousRowKey))
+ totalRead++;
+
+ // read full page
+ if (pageRows >= pageRowSize || !rows.hasNext())
+ {
+ Iterator<String> newKeys = keyColumns.keySet().iterator();
+ for (BoundColumn column : partitionBoundColumns)
+ column.value = keyColumns.get(newKeys.next());
+
+ for (BoundColumn column : clusterColumns)
+ column.value = keyColumns.get(newKeys.next());
+
+ executeQuery();
+ pageRows = 0;
+ }
+
+ return Pair.create(keyColumns, valueColumns);
+ }
+
+ /** check whether start to read a new CF row by comparing the partition keys */
+ private boolean newRow(Map<String, ByteBuffer> keyColumns, String previousRowKey)
+ {
+ if (keyColumns.isEmpty())
+ return false;
+
+ String rowKey = "";
+ if (keyColumns.size() == 1)
+ {
+ rowKey = partitionBoundColumns.get(0).validator.getString(keyColumns.get(partitionBoundColumns.get(0).name));
+ }
+ else
+ {
+ Iterator<ByteBuffer> iter = keyColumns.values().iterator();
+ for (BoundColumn column : partitionBoundColumns)
+ rowKey = rowKey + column.validator.getString(ByteBufferUtil.clone(iter.next())) + ":";
+ }
+
+ logger.debug("previous RowKey: " + previousRowKey + ", new row key: " + rowKey);
+ if (previousRowKey == null)
+ {
+ this.previousRowKey = rowKey;
+ return true;
+ }
+
+ if (rowKey.equals(previousRowKey))
+ return false;
+
+ this.previousRowKey = rowKey;
+ return true;
+ }
+
+ /** set the last non-null key value to null, and return the previous index */
+ private int setTailNull(List<BoundColumn> values)
+ {
+ if (values.isEmpty())
+ return -1;
+
+ Iterator<BoundColumn> iterator = values.iterator();
+ int previousIndex = -1;
+ BoundColumn current;
+ while (iterator.hasNext())
+ {
+ current = iterator.next();
+ if (current.value == null)
+ {
+ int index = previousIndex > 0 ? previousIndex : 0;
+ BoundColumn column = values.get(index);
+ logger.debug("set key " + column.name + " value to null");
+ column.value = null;
+ return previousIndex - 1;
+ }
+
+ previousIndex++;
+ }
+
+ BoundColumn column = values.get(previousIndex);
+ logger.debug("set key " + column.name + " value to null");
+ column.value = null;
+ return previousIndex - 1;
+ }
+
+ /** compose the prepared query, pair.left is query id, pair.right is query */
+ private Pair<Integer, String> composeQuery(String columns)
+ {
+ Pair<Integer, String> clause = whereClause();
+ if (columns == null)
+ {
+ columns = "*";
+ }
+ else
+ {
+ // add keys in the front in order
+ String partitionKey = keyString(partitionBoundColumns);
+ String clusterKey = keyString(clusterColumns);
+
+ columns = withoutKeyColumns(columns);
+ columns = (clusterKey == null || "".equals(clusterKey))
+ ? partitionKey + "," + columns
+ : partitionKey + "," + clusterKey + "," + columns;
+ }
+
+ return Pair.create(clause.left,
+ "SELECT " + columns
+ + " FROM " + cfName
+ + clause.right
+ + (userDefinedWhereClauses == null ? "" : " AND " + userDefinedWhereClauses)
+ + " LIMIT " + pageRowSize
+ + " ALLOW FILTERING");
+ }
+
+
+ /** remove key columns from the column string */
+ private String withoutKeyColumns(String columnString)
+ {
+ Set<String> keyNames = new HashSet<String>();
+ for (BoundColumn column : Iterables.concat(partitionBoundColumns, clusterColumns))
+ keyNames.add(column.name);
+
+ String[] columns = columnString.split(",");
+ String result = null;
+ for (String column : columns)
+ {
+ String trimmed = column.trim();
+ if (keyNames.contains(trimmed))
+ continue;
+
+ result = result == null ? trimmed : result + "," + trimmed;
+ }
+ return result;
+ }
+
+ /** compose the where clause */
+ private Pair<Integer, String> whereClause()
+ {
+ if (partitionKeyString == null)
+ partitionKeyString = keyString(partitionBoundColumns);
+
+ if (partitionKeyMarkers == null)
+ partitionKeyMarkers = partitionKeyMarkers();
+ // initial query token(k) >= start_token and token(k) <= end_token
+ if (emptyPartitionKeyValues())
+ return Pair.create(0, " WHERE token(" + partitionKeyString + ") > ? AND token(" + partitionKeyString + ") <= ?");
+
+ // query token(k) > token(pre_partition_key) and token(k) <= end_token
+ if (clusterColumns.size() == 0 || clusterColumns.get(0).value == null)
+ return Pair.create(1,
+ " WHERE token(" + partitionKeyString + ") > token(" + partitionKeyMarkers + ") "
+ + " AND token(" + partitionKeyString + ") <= ?");
+
+ // query token(k) = token(pre_partition_key) and m = pre_cluster_key_m and n > pre_cluster_key_n
+ Pair<Integer, String> clause = whereClause(clusterColumns, 0);
+ return Pair.create(clause.left,
+ " WHERE token(" + partitionKeyString + ") = token(" + partitionKeyMarkers + ") " + clause.right);
+ }
+
+ /** recursively compose the where clause */
+ private Pair<Integer, String> whereClause(List<BoundColumn> column, int position)
+ {
+ if (position == column.size() - 1 || column.get(position + 1).value == null)
+ return Pair.create(position + 2, " AND " + column.get(position).name + " > ? ");
+
+ Pair<Integer, String> clause = whereClause(column, position + 1);
+ return Pair.create(clause.left, " AND " + column.get(position).name + " = ? " + clause.right);
+ }
+
+ /** check whether all key values are null */
+ private boolean emptyPartitionKeyValues()
+ {
+ for (BoundColumn column : partitionBoundColumns)
+ {
+ if (column.value != null)
+ return false;
+ }
+ return true;
+ }
+
+ /** compose the partition key string in format of <key1>, <key2>, <key3> */
+ private String keyString(List<BoundColumn> columns)
+ {
+ String result = null;
+ for (BoundColumn column : columns)
+ result = result == null ? column.name : result + "," + column.name;
+
+ return result == null ? "" : result;
+ }
+
+ /** compose the question marks for partition key string in format of ?, ? , ? */
+ private String partitionKeyMarkers()
+ {
+ String result = null;
+ for (BoundColumn column : partitionBoundColumns)
+ result = result == null ? "?" : result + ",?";
+
+ return result;
+ }
+
+ /** compose the query binding variables, pair.left is query id, pair.right is the binding variables */
+ private Pair<Integer, List<ByteBuffer>> preparedQueryBindValues()
+ {
+ List<ByteBuffer> values = new LinkedList<ByteBuffer>();
+
+ // initial query token(k) >= start_token and token(k) <= end_token
+ if (emptyPartitionKeyValues())
+ {
+ values.add(partitioner.getTokenValidator().fromString(split.getStartToken()));
+ values.add(partitioner.getTokenValidator().fromString(split.getEndToken()));
+ return Pair.create(0, values);
+ }
+ else
+ {
+ for (BoundColumn partitionBoundColumn1 : partitionBoundColumns)
+ values.add(partitionBoundColumn1.value);
+
+ if (clusterColumns.size() == 0 || clusterColumns.get(0).value == null)
+ {
+ // query token(k) > token(pre_partition_key) and token(k) <= end_token
+ values.add(partitioner.getTokenValidator().fromString(split.getEndToken()));
+ return Pair.create(1, values);
+ }
+ else
+ {
+ // query token(k) = token(pre_partition_key) and m = pre_cluster_key_m and n > pre_cluster_key_n
+ int type = preparedQueryBindValues(clusterColumns, 0, values);
+ return Pair.create(type, values);
+ }
+ }
+ }
+
+ /** recursively compose the query binding variables */
+ private int preparedQueryBindValues(List<BoundColumn> column, int position, List<ByteBuffer> bindValues)
+ {
+ if (position == column.size() - 1 || column.get(position + 1).value == null)
+ {
+ bindValues.add(column.get(position).value);
+ return position + 2;
+ }
+ else
+ {
+ bindValues.add(column.get(position).value);
+ return preparedQueryBindValues(column, position + 1, bindValues);
+ }
+ }
+
+ /** get the prepared query item Id */
+ private int prepareQuery(int type) throws InvalidRequestException, TException
+ {
+ Integer itemId = preparedQueryIds.get(type);
+ if (itemId != null)
+ return itemId;
+
+ Pair<Integer, String> query = null;
+ query = composeQuery(columns);
+ logger.debug("type:" + query.left + ", query: " + query.right);
+ CqlPreparedResult cqlPreparedResult = client.prepare_cql3_query(ByteBufferUtil.bytes(query.right), Compression.NONE);
+ preparedQueryIds.put(query.left, cqlPreparedResult.itemId);
+ return cqlPreparedResult.itemId;
+ }
+
+ /** execute the prepared query */
+ private void executeQuery()
+ {
+ Pair<Integer, List<ByteBuffer>> bindValues = preparedQueryBindValues();
+ logger.debug("query type: " + bindValues.left);
+
+ // check whether it reach end of range for type 1 query CASSANDRA-5573
+ if (bindValues.left == 1 && reachEndRange())
+ {
+ rows = null;
+ return;
+ }
+
+ int retries = 0;
+ // only try three times for TimedOutException and UnavailableException
+ while (retries < 3)
+ {
+ try
+ {
+ CqlResult cqlResult = client.execute_prepared_cql3_query(prepareQuery(bindValues.left), bindValues.right, consistencyLevel);
+ if (cqlResult != null && cqlResult.rows != null)
+ rows = cqlResult.rows.iterator();
+ return;
+ }
+ catch (TimedOutException e)
+ {
+ retries++;
+ if (retries >= 3)
+ {
+ rows = null;
+ RuntimeException rte = new RuntimeException(e.getMessage());
+ rte.initCause(e);
+ throw rte;
+ }
+ }
+ catch (UnavailableException e)
+ {
+ retries++;
+ if (retries >= 3)
+ {
+ rows = null;
+ RuntimeException rte = new RuntimeException(e.getMessage());
+ rte.initCause(e);
+ throw rte;
+ }
+ }
+ catch (Exception e)
+ {
+ rows = null;
+ RuntimeException rte = new RuntimeException(e.getMessage());
+ rte.initCause(e);
+ throw rte;
+ }
+ }
+ }
+ }
+
+ /** retrieve the partition keys and cluster keys from system.schema_columnfamilies table */
+ private void retrieveKeys() throws Exception
+ {
+ String query = "select key_aliases," +
+ "column_aliases, " +
+ "key_validator, " +
+ "comparator " +
+ "from system.schema_columnfamilies " +
+ "where keyspace_name='%s' and columnfamily_name='%s'";
+ String formatted = String.format(query, keyspace, cfName);
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(formatted), Compression.NONE, ConsistencyLevel.ONE);
+
+ CqlRow cqlRow = result.rows.get(0);
+ String keyString = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(0).getValue()));
+ logger.debug("partition keys: " + keyString);
+ List<String> keys = FBUtilities.fromJsonList(keyString);
+
+ for (String key : keys)
+ partitionBoundColumns.add(new BoundColumn(key));
+
+ keyString = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(1).getValue()));
+ logger.debug("cluster columns: " + keyString);
+ keys = FBUtilities.fromJsonList(keyString);
+
+ for (String key : keys)
+ clusterColumns.add(new BoundColumn(key));
+
+ Column rawKeyValidator = cqlRow.columns.get(2);
+ String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));
+ logger.debug("row key validator: " + validator);
+ keyValidator = parseType(validator);
+
+ if (keyValidator instanceof CompositeType)
+ {
+ List<AbstractType<?>> types = ((CompositeType) keyValidator).types;
+ for (int i = 0; i < partitionBoundColumns.size(); i++)
+ partitionBoundColumns.get(i).validator = types.get(i);
+ }
+ else
+ {
+ partitionBoundColumns.get(0).validator = keyValidator;
+ }
+ }
+
+ /** check whether current row is at the end of range */
+ private boolean reachEndRange()
+ {
+ // current row key
+ ByteBuffer rowKey;
+ if (keyValidator instanceof CompositeType)
+ {
+ ByteBuffer[] keys = new ByteBuffer[partitionBoundColumns.size()];
+ for (int i = 0; i < partitionBoundColumns.size(); i++)
+ keys[i] = partitionBoundColumns.get(i).value.duplicate();
+
+ rowKey = ((CompositeType) keyValidator).build(keys);
+ }
+ else
+ {
+ rowKey = partitionBoundColumns.get(0).value;
+ }
+
+ String endToken = split.getEndToken();
+ String currentToken = partitioner.getToken(rowKey).toString();
+ logger.debug("End token: " + endToken + ", current token: " + currentToken);
+
+ return endToken.equals(currentToken);
+ }
+
+ private static AbstractType<?> parseType(String type) throws IOException
+ {
+ try
+ {
+ // always treat counters like longs, specifically CCT.compose is not what we need
+ if (type != null && type.equals("org.apache.cassandra.db.marshal.CounterColumnType"))
+ return LongType.instance;
+ return TypeParser.parse(type);
+ }
+ catch (ConfigurationException e)
+ {
+ throw new IOException(e);
+ }
+ catch (SyntaxException e)
+ {
+ throw new IOException(e);
+ }
+ }
+
+ private class BoundColumn
+ {
+ final String name;
+ ByteBuffer value;
+ AbstractType<?> validator;
+
+ public BoundColumn(String name)
+ {
+ this.name = name;
+ }
+ }
+
+ /** get string from a ByteBuffer, catch the exception and throw it as runtime exception*/
+ private static String stringValue(ByteBuffer value)
+ {
+ try
+ {
+ return ByteBufferUtil.string(value);
+ }
+ catch (CharacterCodingException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+}
[08/10] merge from 1.2
Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java b/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
index 792618d..5905f4a 100644
--- a/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
+++ b/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
@@ -21,6 +21,10 @@ package org.apache.cassandra.thrift;
*
*/
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
import org.apache.cassandra.hadoop.ConfigHelper;
import org.apache.thrift.transport.TFramedTransport;
import org.apache.thrift.transport.TSocket;
@@ -29,7 +33,7 @@ import org.apache.thrift.transport.TTransportException;
import org.apache.hadoop.conf.Configuration;
-public class TFramedTransportFactory implements ITransportFactory
+public class TFramedTransportFactory implements ITransportFactory, TClientTransportFactory
{
public TTransport openTransport(TSocket socket, Configuration conf) throws TTransportException
{
@@ -37,4 +41,22 @@ public class TFramedTransportFactory implements ITransportFactory
transport.open();
return transport;
}
+
+ //
+ public TTransport openTransport(String host, int port, Configuration conf) throws TTransportException
+ {
+ TSocket socket = new TSocket(host, port);
+ TTransport transport = new TFramedTransport(socket, ConfigHelper.getThriftFramedTransportSize(conf));
+ transport.open();
+ return transport;
+ }
+
+ public void setOptions(Map<String, String> options)
+ {
+ }
+
+ public Set<String> supportedOptions()
+ {
+ return Collections.emptySet();
+ }
}
[03/10] git commit: Add CQL3 input/output formats patch by Alex Liu;
reviewed by jbellis and Mike Schrag for CASSANDRA-4421
Posted by jb...@apache.org.
Add CQL3 input/output formats
patch by Alex Liu; reviewed by jbellis and Mike Schrag for CASSANDRA-4421
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/56e0ad1b
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/56e0ad1b
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/56e0ad1b
Branch: refs/heads/cassandra-1.2
Commit: 56e0ad1be2c3d7a2eb56b361e021668b8f22c095
Parents: 6850cdc
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Jun 6 14:09:51 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Jun 6 14:18:23 2013 -0500
----------------------------------------------------------------------
CHANGES.txt | 1 +
examples/hadoop_cql3_word_count/README.txt | 47 +
examples/hadoop_cql3_word_count/bin/word_count | 61 ++
.../hadoop_cql3_word_count/bin/word_count_counters | 58 ++
.../hadoop_cql3_word_count/bin/word_count_setup | 61 ++
examples/hadoop_cql3_word_count/build.xml | 113 +++
.../hadoop_cql3_word_count/conf/log4j.properties | 32 +
examples/hadoop_cql3_word_count/ivy.xml | 24 +
examples/hadoop_cql3_word_count/src/WordCount.java | 236 +++++
.../src/WordCountCounters.java | 122 +++
.../hadoop_cql3_word_count/src/WordCountSetup.java | 214 ++++
.../org/apache/cassandra/client/RingCache.java | 2 +-
.../hadoop/AbstractColumnFamilyInputFormat.java | 346 +++++++
.../hadoop/AbstractColumnFamilyOutputFormat.java | 159 +++
.../hadoop/AbstractColumnFamilyRecordWriter.java | 193 ++++
.../cassandra/hadoop/ColumnFamilyInputFormat.java | 307 +------
.../cassandra/hadoop/ColumnFamilyOutputFormat.java | 125 +---
.../cassandra/hadoop/ColumnFamilyRecordReader.java | 43 +-
.../cassandra/hadoop/ColumnFamilyRecordWriter.java | 195 +---
.../apache/cassandra/hadoop/ColumnFamilySplit.java | 4 +-
.../org/apache/cassandra/hadoop/ConfigHelper.java | 68 +-
.../org/apache/cassandra/hadoop/Progressable.java | 4 +-
.../cassandra/hadoop/cql3/CQLConfigHelper.java | 109 ++
.../hadoop/cql3/ColumnFamilyInputFormat.java | 83 ++
.../hadoop/cql3/ColumnFamilyOutputFormat.java | 78 ++
.../hadoop/cql3/ColumnFamilyRecordReader.java | 763 +++++++++++++++
.../hadoop/cql3/ColumnFamilyRecordWriter.java | 386 ++++++++
.../cassandra/thrift/TClientTransportFactory.java | 70 ++
.../cassandra/thrift/TFramedTransportFactory.java | 24 +-
29 files changed, 3280 insertions(+), 648 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 75b3173..36493d6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,6 +1,7 @@
1.2.6
* Reduce SSTableLoader memory usage (CASSANDRA-5555)
* Scale hinted_handoff_throttle_in_kb to cluster size (CASSANDRA-5272)
+ * (Hadoop) Add CQL3 input/output formats (CASSANDRA-4421)
* (Hadoop) Fix InputKeyRange in CFIF (CASSANDRA-5536)
* Fix dealing with ridiculously large max sstable sizes in LCS (CASSANDRA-5589)
* Ignore pre-truncate hints (CASSANDRA-4655)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/README.txt
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/README.txt b/examples/hadoop_cql3_word_count/README.txt
new file mode 100644
index 0000000..f984b02
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/README.txt
@@ -0,0 +1,47 @@
+Introduction
+============
+
+WordCount hadoop example: Inserts a bunch of words across multiple rows,
+and counts them, with RandomPartitioner. The word_count_counters example sums
+the value of counter columns for a key.
+
+The scripts in bin/ assume you are running with cwd of contrib/word_count.
+
+
+Running
+=======
+
+First build and start a Cassandra server with the default configuration*,
+then run
+
+contrib/word_count$ ant
+contrib/word_count$ bin/word_count_setup
+contrib/word_count$ bin/word_count
+contrib/word_count$ bin/word_count_counters
+
+In order to view the results in Cassandra, one can use bin/cassandra-cli and
+perform the following operations:
+$ bin/cqlsh localhost
+> use cql3_worldcount;
+> select * from output_words;
+
+The output of the word count can now be configured. In the bin/word_count
+file, you can specify the OUTPUT_REDUCER. The two options are 'filesystem'
+and 'cassandra'. The filesystem option outputs to the /tmp/word_count*
+directories. The cassandra option outputs to the 'output_words' column family
+in the 'cql3_worldcount' keyspace. 'cassandra' is the default.
+
+Read the code in src/ for more details.
+
+The word_count_counters example sums the counter columns for a row. The output
+is written to a text file in /tmp/word_count_counters.
+
+*If you want to point wordcount at a real cluster, modify the seed
+and listenaddress settings accordingly.
+
+
+Troubleshooting
+===============
+
+word_count uses conf/log4j.properties to log to wc.out.
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/bin/word_count
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/bin/word_count b/examples/hadoop_cql3_word_count/bin/word_count
new file mode 100644
index 0000000..a0c5aa0
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/bin/word_count
@@ -0,0 +1,61 @@
+#!/bin/sh
+
+# 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.
+
+cwd=`dirname $0`
+
+# Cassandra class files.
+if [ ! -d $cwd/../../../build/classes/main ]; then
+ echo "Unable to locate cassandra class files" >&2
+ exit 1
+fi
+
+# word_count Jar.
+if [ ! -e $cwd/../build/word_count.jar ]; then
+ echo "Unable to locate word_count jar" >&2
+ exit 1
+fi
+
+CLASSPATH=$CLASSPATH:$cwd/../conf
+CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/main
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/thrift
+for jar in $cwd/../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../lib/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+
+if [ -x $JAVA_HOME/bin/java ]; then
+ JAVA=$JAVA_HOME/bin/java
+else
+ JAVA=`which java`
+fi
+
+if [ "x$JAVA" = "x" ]; then
+ echo "Java executable not found (hint: set JAVA_HOME)" >&2
+ exit 1
+fi
+
+OUTPUT_REDUCER=cassandra
+
+#echo $CLASSPATH
+$JAVA -Xmx1G -ea -cp $CLASSPATH WordCount output_reducer=$OUTPUT_REDUCER
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/bin/word_count_counters
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/bin/word_count_counters b/examples/hadoop_cql3_word_count/bin/word_count_counters
new file mode 100644
index 0000000..7793477
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/bin/word_count_counters
@@ -0,0 +1,58 @@
+#!/bin/sh
+
+# 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.
+
+cwd=`dirname $0`
+
+# Cassandra class files.
+if [ ! -d $cwd/../../../build/classes/main ]; then
+ echo "Unable to locate cassandra class files" >&2
+ exit 1
+fi
+
+# word_count Jar.
+if [ ! -e $cwd/../build/word_count.jar ]; then
+ echo "Unable to locate word_count jar" >&2
+ exit 1
+fi
+
+CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/main
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/thrift
+for jar in $cwd/../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../lib/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+
+if [ -x $JAVA_HOME/bin/java ]; then
+ JAVA=$JAVA_HOME/bin/java
+else
+ JAVA=`which java`
+fi
+
+if [ "x$JAVA" = "x" ]; then
+ echo "Java executable not found (hint: set JAVA_HOME)" >&2
+ exit 1
+fi
+
+#echo $CLASSPATH
+$JAVA -Xmx1G -ea -cp $CLASSPATH WordCountCounters
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/bin/word_count_setup
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/bin/word_count_setup b/examples/hadoop_cql3_word_count/bin/word_count_setup
new file mode 100644
index 0000000..d194a45
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/bin/word_count_setup
@@ -0,0 +1,61 @@
+#!/bin/sh
+
+# 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.
+
+cwd=`dirname $0`
+
+# Cassandra class files.
+if [ ! -d $cwd/../../../build/classes/main ]; then
+ echo "Unable to locate cassandra class files" >&2
+ exit 1
+fi
+
+# word_count Jar.
+if [ ! -e $cwd/../build/word_count.jar ]; then
+ echo "Unable to locate word_count jar" >&2
+ exit 1
+fi
+
+CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
+CLASSPATH=$CLASSPATH:.:$cwd/../../../build/classes/main
+CLASSPATH=$CLASSPATH:.:$cwd/../../../build/classes/thrift
+for jar in $cwd/../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../lib/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+
+if [ -x $JAVA_HOME/bin/java ]; then
+ JAVA=$JAVA_HOME/bin/java
+else
+ JAVA=`which java`
+fi
+
+if [ "x$JAVA" = "x" ]; then
+ echo "Java executable not found (hint: set JAVA_HOME)" >&2
+ exit 1
+fi
+
+HOST=localhost
+PORT=9160
+FRAMED=true
+
+$JAVA -Xmx1G -ea -Dcassandra.host=$HOST -Dcassandra.port=$PORT -Dcassandra.framed=$FRAMED -cp $CLASSPATH WordCountSetup
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/build.xml
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/build.xml b/examples/hadoop_cql3_word_count/build.xml
new file mode 100644
index 0000000..939e1b3
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/build.xml
@@ -0,0 +1,113 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+<project default="jar" name="word_count" xmlns:ivy="antlib:org.apache.ivy.ant">
+ <property name="cassandra.dir" value="../.." />
+ <property name="cassandra.dir.lib" value="${cassandra.dir}/lib" />
+ <property name="cassandra.classes" value="${cassandra.dir}/build/classes" />
+ <property name="build.src" value="${basedir}/src" />
+ <property name="build.dir" value="${basedir}/build" />
+ <property name="ivy.lib.dir" value="${build.dir}/lib" />
+ <property name="build.classes" value="${build.dir}/classes" />
+ <property name="final.name" value="word_count" />
+ <property name="ivy.version" value="2.1.0" />
+ <property name="ivy.url"
+ value="http://repo2.maven.org/maven2/org/apache/ivy/ivy" />
+
+ <condition property="ivy.jar.exists">
+ <available file="${build.dir}/ivy-${ivy.version}.jar" />
+ </condition>
+
+ <path id="autoivy.classpath">
+ <fileset dir="${ivy.lib.dir}">
+ <include name="**/*.jar" />
+ </fileset>
+ <pathelement location="${build.dir}/ivy-${ivy.version}.jar"/>
+ </path>
+
+ <path id="wordcount.build.classpath">
+ <fileset dir="${ivy.lib.dir}">
+ <include name="**/*.jar" />
+ </fileset>
+ <!-- cassandra dependencies -->
+ <fileset dir="${cassandra.dir.lib}">
+ <include name="**/*.jar" />
+ </fileset>
+ <fileset dir="${cassandra.dir}/build/lib/jars">
+ <include name="**/*.jar" />
+ </fileset>
+ <pathelement location="${cassandra.classes}/main" />
+ <pathelement location="${cassandra.classes}/thrift" />
+ </path>
+
+ <target name="init">
+ <mkdir dir="${build.classes}" />
+ </target>
+
+ <target depends="init,ivy-retrieve-build" name="build">
+ <javac destdir="${build.classes}">
+ <src path="${build.src}" />
+ <classpath refid="wordcount.build.classpath" />
+ </javac>
+ </target>
+
+ <target name="jar" depends="build">
+ <mkdir dir="${build.classes}/META-INF" />
+ <jar jarfile="${build.dir}/${final.name}.jar">
+ <fileset dir="${build.classes}" />
+ <fileset dir="${cassandra.classes}/main" />
+ <fileset dir="${cassandra.classes}/thrift" />
+ <fileset dir="${cassandra.dir}">
+ <include name="lib/**/*.jar" />
+ </fileset>
+ <zipfileset dir="${cassandra.dir}/build/lib/jars/" prefix="lib">
+ <include name="**/*.jar" />
+ </zipfileset>
+ <fileset file="${basedir}/cassandra.yaml" />
+ </jar>
+ </target>
+
+ <target name="clean">
+ <delete dir="${build.dir}" />
+ </target>
+
+ <!--
+ Ivy Specific targets
+ to fetch Ivy and this project's dependencies
+ -->
+ <target name="ivy-download" unless="ivy.jar.exists">
+ <echo>Downloading Ivy...</echo>
+ <mkdir dir="${build.dir}" />
+ <get src="${ivy.url}/${ivy.version}/ivy-${ivy.version}.jar"
+ dest="${build.dir}/ivy-${ivy.version}.jar" usetimestamp="true" />
+ </target>
+
+ <target name="ivy-init" depends="ivy-download" unless="ivy.initialized">
+ <mkdir dir="${ivy.lib.dir}"/>
+ <taskdef resource="org/apache/ivy/ant/antlib.xml"
+ uri="antlib:org.apache.ivy.ant"
+ classpathref="autoivy.classpath"/>
+ <property name="ivy.initialized" value="true"/>
+ </target>
+
+ <target name="ivy-retrieve-build" depends="ivy-init">
+ <ivy:retrieve type="jar,source" sync="true"
+ pattern="${ivy.lib.dir}/[type]s/[artifact]-[revision].[ext]" />
+ </target>
+</project>
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/conf/log4j.properties b/examples/hadoop_cql3_word_count/conf/log4j.properties
new file mode 100644
index 0000000..70f7657
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/conf/log4j.properties
@@ -0,0 +1,32 @@
+# 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.
+
+log4j.rootLogger=DEBUG,stdout,F
+
+#stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%5p %d{HH:mm:ss,SSS} %m%n
+
+# log file
+log4j.appender.F=org.apache.log4j.FileAppender
+log4j.appender.F.Append=false
+log4j.appender.F.layout=org.apache.log4j.PatternLayout
+log4j.appender.F.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n
+# Edit the next line to point to your logs directory
+log4j.appender.F.File=wc.out
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/ivy.xml
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/ivy.xml b/examples/hadoop_cql3_word_count/ivy.xml
new file mode 100644
index 0000000..9d44895
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/ivy.xml
@@ -0,0 +1,24 @@
+<!--
+ ~ 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.
+ -->
+<ivy-module version="2.0">
+ <info organisation="apache-cassandra" module="word-count"/>
+ <dependencies>
+ <dependency org="org.apache.hadoop" name="hadoop-core" rev="0.20.2"/>
+ </dependencies>
+</ivy-module>
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/src/WordCount.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/src/WordCount.java b/examples/hadoop_cql3_word_count/src/WordCount.java
new file mode 100644
index 0000000..09dd9e4
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/src/WordCount.java
@@ -0,0 +1,236 @@
+/**
+ * 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 java.util.Map.Entry;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.hadoop.cql3.ColumnFamilyOutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.hadoop.cql3.CQLConfigHelper;
+import org.apache.cassandra.hadoop.cql3.ColumnFamilyInputFormat;
+import org.apache.cassandra.hadoop.ConfigHelper;
+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.Reducer.Context;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.nio.charset.CharacterCodingException;
+
+/**
+ * This counts the occurrences of words in ColumnFamily
+ * cql3_worldcount ( user_id text,
+ * category_id text,
+ * sub_category_id text,
+ * title text,
+ * body text,
+ * PRIMARY KEY (user_id, category_id, sub_category_id))
+ *
+ * For each word, we output the total number of occurrences across all body texts.
+ *
+ * When outputting to Cassandra, we write the word counts to column family
+ * output_words ( row_id1 text,
+ * row_id2 text,
+ * word text,
+ * count_num text,
+ * PRIMARY KEY ((row_id1, row_id2), word))
+ * as a {word, count} to columns: word, count_num with a row key of "word sum"
+ */
+public class WordCount extends Configured implements Tool
+{
+ private static final Logger logger = LoggerFactory.getLogger(WordCount.class);
+
+ static final String KEYSPACE = "cql3_worldcount";
+ static final String COLUMN_FAMILY = "inputs";
+
+ static final String OUTPUT_REDUCER_VAR = "output_reducer";
+ static final String OUTPUT_COLUMN_FAMILY = "output_words";
+
+ private static final String OUTPUT_PATH_PREFIX = "/tmp/word_count";
+
+ private static final String PRIMARY_KEY = "row_key";
+
+ 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<Map<String, ByteBuffer>, Map<String, ByteBuffer>, Text, IntWritable>
+ {
+ private final static IntWritable one = new IntWritable(1);
+ private Text word = new Text();
+ private ByteBuffer sourceColumn;
+
+ protected void setup(org.apache.hadoop.mapreduce.Mapper.Context context)
+ throws IOException, InterruptedException
+ {
+ }
+
+ public void map(Map<String, ByteBuffer> keys, Map<String, ByteBuffer> columns, Context context) throws IOException, InterruptedException
+ {
+ for (Entry<String, ByteBuffer> column : columns.entrySet())
+ {
+ if (!"body".equalsIgnoreCase(column.getKey()))
+ continue;
+
+ String value = ByteBufferUtil.string(column.getValue());
+
+ logger.debug("read {}:{}={} from {}",
+ new Object[] {toString(keys), column.getKey(), value, context.getInputSplit()});
+
+ StringTokenizer itr = new StringTokenizer(value);
+ while (itr.hasMoreTokens())
+ {
+ word.set(itr.nextToken());
+ context.write(word, one);
+ }
+ }
+ }
+
+ private String toString(Map<String, ByteBuffer> keys)
+ {
+ String result = "";
+ try
+ {
+ for (ByteBuffer key : keys.values())
+ result = result + ByteBufferUtil.string(key) + ":";
+ }
+ catch (CharacterCodingException e)
+ {
+ logger.error("Failed to print keys", e);
+ }
+ return result;
+ }
+ }
+
+ public static class ReducerToFilesystem extends Reducer<Text, IntWritable, Text, IntWritable>
+ {
+ public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException
+ {
+ int sum = 0;
+ for (IntWritable val : values)
+ sum += val.get();
+ context.write(key, new IntWritable(sum));
+ }
+ }
+
+ public static class ReducerToCassandra extends Reducer<Text, IntWritable, Map<String, ByteBuffer>, List<ByteBuffer>>
+ {
+ private Map<String, ByteBuffer> keys;
+ private ByteBuffer key;
+ protected void setup(org.apache.hadoop.mapreduce.Reducer.Context context)
+ throws IOException, InterruptedException
+ {
+ keys = new LinkedHashMap<String, ByteBuffer>();
+ String[] partitionKeys = context.getConfiguration().get(PRIMARY_KEY).split(",");
+ keys.put("row_id1", ByteBufferUtil.bytes(partitionKeys[0]));
+ keys.put("row_id2", ByteBufferUtil.bytes(partitionKeys[1]));
+ }
+
+ public void reduce(Text word, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException
+ {
+ int sum = 0;
+ for (IntWritable val : values)
+ sum += val.get();
+ context.write(keys, getBindVariables(word, sum));
+ }
+
+ private List<ByteBuffer> getBindVariables(Text word, int sum)
+ {
+ List<ByteBuffer> variables = new ArrayList<ByteBuffer>();
+ variables.add(keys.get("row_id1"));
+ variables.add(keys.get("row_id2"));
+ variables.add(ByteBufferUtil.bytes(word.toString()));
+ variables.add(ByteBufferUtil.bytes(String.valueOf(sum)));
+ return variables;
+ }
+ }
+
+ 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);
+
+ 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));
+ }
+ else
+ {
+ job.setReducerClass(ReducerToCassandra.class);
+
+ job.setMapOutputKeyClass(Text.class);
+ job.setMapOutputValueClass(IntWritable.class);
+ job.setOutputKeyClass(Map.class);
+ job.setOutputValueClass(List.class);
+
+ job.setOutputFormatClass(ColumnFamilyOutputFormat.class);
+
+ ConfigHelper.setOutputColumnFamily(job.getConfiguration(), KEYSPACE, OUTPUT_COLUMN_FAMILY);
+ job.getConfiguration().set(PRIMARY_KEY, "word,sum");
+ String query = "INSERT INTO " + KEYSPACE + "." + OUTPUT_COLUMN_FAMILY +
+ " (row_id1, row_id2, word, count_num) " +
+ " values (?, ?, ?, ?)";
+ CQLConfigHelper.setOutputCql(job.getConfiguration(), query);
+ ConfigHelper.setOutputInitialAddress(job.getConfiguration(), "localhost");
+ ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner");
+ }
+
+ job.setInputFormatClass(ColumnFamilyInputFormat.class);
+
+ ConfigHelper.setInputRpcPort(job.getConfiguration(), "9160");
+ ConfigHelper.setInputInitialAddress(job.getConfiguration(), "localhost");
+ ConfigHelper.setInputColumnFamily(job.getConfiguration(), KEYSPACE, COLUMN_FAMILY);
+ ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner");
+
+ CQLConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), "3");
+ //this is the user defined filter clauses, you can comment it out if you want count all titles
+ CQLConfigHelper.setInputWhereClauses(job.getConfiguration(), "title='A'");
+ job.waitForCompletion(true);
+ return 0;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/src/WordCountCounters.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/src/WordCountCounters.java b/examples/hadoop_cql3_word_count/src/WordCountCounters.java
new file mode 100644
index 0000000..1cf5539
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/src/WordCountCounters.java
@@ -0,0 +1,122 @@
+/**
+ * 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.nio.charset.CharacterCodingException;
+import java.util.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.io.LongWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.cassandra.hadoop.cql3.ColumnFamilyInputFormat;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.cql3.CQLConfigHelper;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+
+/**
+ * This sums the word count stored in the input_words_count ColumnFamily for the key "sum".
+ *
+ * Output is written to a text file.
+ */
+public class WordCountCounters extends Configured implements Tool
+{
+ private static final Logger logger = LoggerFactory.getLogger(WordCountCounters.class);
+
+ static final String COUNTER_COLUMN_FAMILY = "input_words_count";
+ private static final String OUTPUT_PATH_PREFIX = "/tmp/word_count_counters";
+
+ public static void main(String[] args) throws Exception
+ {
+ // Let ToolRunner handle generic command-line options
+ ToolRunner.run(new Configuration(), new WordCountCounters(), args);
+ System.exit(0);
+ }
+
+ public static class SumMapper extends Mapper<Map<String, ByteBuffer>, Map<String, ByteBuffer>, Text, LongWritable>
+ {
+ long sum = -1;
+ public void map(Map<String, ByteBuffer> key, Map<String, ByteBuffer> columns, Context context) throws IOException, InterruptedException
+ {
+ if (sum < 0)
+ sum = 0;
+
+ logger.debug("read " + toString(key) + ":count_num from " + context.getInputSplit());
+ sum += Long.valueOf(ByteBufferUtil.string(columns.get("count_num")));
+ }
+
+ protected void cleanup(Context context) throws IOException, InterruptedException {
+ if (sum > 0)
+ context.write(new Text("total_count"), new LongWritable(sum));
+ }
+
+ private String toString(Map<String, ByteBuffer> keys)
+ {
+ String result = "";
+ try
+ {
+ for (ByteBuffer key : keys.values())
+ result = result + ByteBufferUtil.string(key) + ":";
+ }
+ catch (CharacterCodingException e)
+ {
+ logger.error("Failed to print keys", e);
+ }
+ return result;
+ }
+ }
+
+
+ public int run(String[] args) throws Exception
+ {
+ Job job = new Job(getConf(), "wordcountcounters");
+ job.setJarByClass(WordCountCounters.class);
+ job.setMapperClass(SumMapper.class);
+
+ job.setOutputKeyClass(Text.class);
+ job.setOutputValueClass(LongWritable.class);
+ FileOutputFormat.setOutputPath(job, new Path(OUTPUT_PATH_PREFIX));
+
+ job.setInputFormatClass(ColumnFamilyInputFormat.class);
+
+ ConfigHelper.setInputRpcPort(job.getConfiguration(), "9160");
+ ConfigHelper.setInputInitialAddress(job.getConfiguration(), "localhost");
+ ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner");
+ ConfigHelper.setInputColumnFamily(job.getConfiguration(), WordCount.KEYSPACE, WordCount.OUTPUT_COLUMN_FAMILY);
+
+ CQLConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), "3");
+
+ job.waitForCompletion(true);
+ return 0;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/src/WordCountSetup.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/src/WordCountSetup.java b/examples/hadoop_cql3_word_count/src/WordCountSetup.java
new file mode 100644
index 0000000..4c22e49
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/src/WordCountSetup.java
@@ -0,0 +1,214 @@
+/**
+ * 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.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class WordCountSetup
+{
+ private static final Logger logger = LoggerFactory.getLogger(WordCountSetup.class);
+
+ public static final int TEST_COUNT = 6;
+
+ public static void main(String[] args) throws Exception
+ {
+ Cassandra.Iface client = createConnection();
+
+ setupKeyspace(client);
+ client.set_keyspace(WordCount.KEYSPACE);
+ setupTable(client);
+ insertData(client);
+
+ System.exit(0);
+ }
+
+ private static void setupKeyspace(Cassandra.Iface client)
+ throws InvalidRequestException,
+ UnavailableException,
+ TimedOutException,
+ SchemaDisagreementException,
+ TException
+ {
+ KsDef ks;
+ try
+ {
+ ks = client.describe_keyspace(WordCount.KEYSPACE);
+ }
+ catch(NotFoundException e)
+ {
+ logger.info("set up keyspace " + WordCount.KEYSPACE);
+ String query = "CREATE KEYSPACE " + WordCount.KEYSPACE +
+ " WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1}";
+
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+
+ int magnitude = client.describe_ring(WordCount.KEYSPACE).size();
+ try
+ {
+ Thread.sleep(1000 * magnitude);
+ }
+ catch (InterruptedException ie)
+ {
+ throw new RuntimeException(ie);
+ }
+ }
+ }
+
+ private static void setupTable(Cassandra.Iface client)
+ throws InvalidRequestException,
+ UnavailableException,
+ TimedOutException,
+ SchemaDisagreementException,
+ TException
+ {
+ String query = "CREATE TABLE " + WordCount.KEYSPACE + "." + WordCount.COLUMN_FAMILY +
+ " ( user_id text," +
+ " category_id text, " +
+ " sub_category_id text," +
+ " title text," +
+ " body text," +
+ " PRIMARY KEY (user_id, category_id, sub_category_id) ) ";
+
+ try
+ {
+ logger.info("set up table " + WordCount.COLUMN_FAMILY);
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ logger.error("failed to create table " + WordCount.KEYSPACE + "." + WordCount.COLUMN_FAMILY, e);
+ }
+
+ query = "CREATE INDEX title on " + WordCount.COLUMN_FAMILY + "(title)";
+ try
+ {
+ logger.info("set up index on title column ");
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ logger.error("Failed to create index on title", e);
+ }
+
+ query = "CREATE TABLE " + WordCount.KEYSPACE + "." + WordCount.OUTPUT_COLUMN_FAMILY +
+ " ( row_id1 text, " +
+ " row_id2 text, " +
+ " word text, " +
+ " count_num text," +
+ " PRIMARY KEY ((row_id1, row_id2), word) ) ";
+
+ try
+ {
+ logger.info("set up table " + WordCount.OUTPUT_COLUMN_FAMILY);
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ logger.error("failed to create table " + WordCount.KEYSPACE + "." + WordCount.OUTPUT_COLUMN_FAMILY, e);
+ }
+ }
+
+ private static Cassandra.Iface createConnection() throws TTransportException
+ {
+ if (System.getProperty("cassandra.host") == null || System.getProperty("cassandra.port") == null)
+ {
+ logger.warn("cassandra.host or cassandra.port is not defined, using default");
+ }
+ return createConnection(System.getProperty("cassandra.host", "localhost"),
+ Integer.valueOf(System.getProperty("cassandra.port", "9160")));
+ }
+
+ private static Cassandra.Client createConnection(String host, Integer port) throws TTransportException
+ {
+ TSocket socket = new TSocket(host, port);
+ TTransport trans = new TFramedTransport(socket);
+ trans.open();
+ TProtocol protocol = new TBinaryProtocol(trans);
+
+ return new Cassandra.Client(protocol);
+ }
+
+ private static void insertData(Cassandra.Iface client)
+ throws InvalidRequestException,
+ UnavailableException,
+ TimedOutException,
+ SchemaDisagreementException,
+ TException
+ {
+ String query = "INSERT INTO " + WordCount.COLUMN_FAMILY +
+ "(user_id, category_id, sub_category_id, title, body ) " +
+ " values (?, ?, ?, ?, ?) ";
+ CqlPreparedResult result = client.prepare_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE);
+
+ String [] title = titleData();
+ String [] body = bodyData();
+ for (int i=1; i<5; i++)
+ {
+ for (int j=1; j<444; j++)
+ {
+ for (int k=1; k<4; k++)
+ {
+ List<ByteBuffer> values = new ArrayList<ByteBuffer>();
+ values.add(ByteBufferUtil.bytes(String.valueOf(j)));
+ values.add(ByteBufferUtil.bytes(String.valueOf(i)));
+ values.add(ByteBufferUtil.bytes(String.valueOf(k)));
+ values.add(ByteBufferUtil.bytes(title[i]));
+ values.add(ByteBufferUtil.bytes(body[i]));
+ client.execute_prepared_cql3_query(result.itemId, values, ConsistencyLevel.ONE);
+ }
+ }
+ }
+ }
+
+ private static String[] bodyData()
+ { // Public domain context, source http://en.wikisource.org/wiki/If%E2%80%94
+ return new String[]{
+ "",
+ "If you can keep your head when all about you",
+ "Are losing theirs and blaming it on you",
+ "If you can trust yourself when all men doubt you,",
+ "But make allowance for their doubting too:",
+ "If you can wait and not be tired by waiting,"
+ };
+ }
+
+ private static String[] titleData()
+ { // Public domain context, source http://en.wikisource.org/wiki/If%E2%80%94
+ return new String[]{
+ "",
+ "A",
+ "B",
+ "C",
+ "D",
+ "E"
+ };
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/client/RingCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/client/RingCache.java b/src/java/org/apache/cassandra/client/RingCache.java
index 4db4a0d..3308471 100644
--- a/src/java/org/apache/cassandra/client/RingCache.java
+++ b/src/java/org/apache/cassandra/client/RingCache.java
@@ -52,7 +52,7 @@ public class RingCache
private Multimap<Range<Token>, InetAddress> rangeMap;
- public RingCache(Configuration conf) throws IOException
+ public RingCache(Configuration conf)
{
this.conf = conf;
this.partitioner = ConfigHelper.getOutputPartitioner(conf);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
new file mode 100644
index 0000000..1c8fd0b
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
@@ -0,0 +1,346 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.cassandra.auth.IAuthenticator;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.thrift.AuthenticationRequest;
+import org.apache.cassandra.thrift.Cassandra;
+import org.apache.cassandra.thrift.CfSplit;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.thrift.KeyRange;
+import org.apache.cassandra.thrift.TokenRange;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<K, Y> implements org.apache.hadoop.mapred.InputFormat<K, Y>
+{
+ private static final Logger logger = LoggerFactory.getLogger(AbstractColumnFamilyInputFormat.class);
+
+ public static final String MAPRED_TASK_ID = "mapred.task.id";
+ // The simple fact that we need this is because the old Hadoop API wants us to "write"
+ // to the key and value whereas the new asks for it.
+ // I choose 8kb as the default max key size (instanciated only once), but you can
+ // override it in your jobConf with this setting.
+ public static final String CASSANDRA_HADOOP_MAX_KEY_SIZE = "cassandra.hadoop.max_key_size";
+ public static final int CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT = 8192;
+
+ private String keyspace;
+ private String cfName;
+ private IPartitioner partitioner;
+
+ protected void validateConfiguration(Configuration conf)
+ {
+ if (ConfigHelper.getInputKeyspace(conf) == null || ConfigHelper.getInputColumnFamily(conf) == null)
+ {
+ throw new UnsupportedOperationException("you must set the keyspace and columnfamily with setInputColumnFamily()");
+ }
+ if (ConfigHelper.getInputInitialAddress(conf) == null)
+ throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node with setInputInitialAddress");
+ if (ConfigHelper.getInputPartitioner(conf) == null)
+ throw new UnsupportedOperationException("You must set the Cassandra partitioner class with setInputPartitioner");
+ }
+
+ public static Cassandra.Client createAuthenticatedClient(String location, int port, Configuration conf) throws Exception
+ {
+ logger.debug("Creating authenticated client for CF input format");
+ TTransport transport = ConfigHelper.getClientTransportFactory(conf).openTransport(location, port, conf);
+ TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
+ Cassandra.Client client = new Cassandra.Client(binaryProtocol);
+
+ // log in
+ client.set_keyspace(ConfigHelper.getInputKeyspace(conf));
+ if (ConfigHelper.getInputKeyspaceUserName(conf) != null)
+ {
+ Map<String, String> creds = new HashMap<String, String>();
+ creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf));
+ creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf));
+ AuthenticationRequest authRequest = new AuthenticationRequest(creds);
+ client.login(authRequest);
+ }
+ logger.debug("Authenticated client for CF input format created successfully");
+ return client;
+ }
+
+ public List<InputSplit> getSplits(JobContext context) throws IOException
+ {
+ Configuration conf = context.getConfiguration();
+
+ validateConfiguration(conf);
+
+ // cannonical ranges and nodes holding replicas
+ List<TokenRange> masterRangeNodes = getRangeMap(conf);
+
+ keyspace = ConfigHelper.getInputKeyspace(context.getConfiguration());
+ cfName = ConfigHelper.getInputColumnFamily(context.getConfiguration());
+ partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
+ logger.debug("partitioner is " + partitioner);
+
+ // cannonical ranges, split into pieces, fetching the splits in parallel
+ ExecutorService executor = Executors.newCachedThreadPool();
+ List<InputSplit> splits = new ArrayList<InputSplit>();
+
+ try
+ {
+ List<Future<List<InputSplit>>> splitfutures = new ArrayList<Future<List<InputSplit>>>();
+ KeyRange jobKeyRange = ConfigHelper.getInputKeyRange(conf);
+ Range<Token> jobRange = null;
+ if (jobKeyRange != null)
+ {
+ if (jobKeyRange.start_key == null)
+ {
+ logger.warn("ignoring jobKeyRange specified without start_key");
+ }
+ else
+ {
+ if (!partitioner.preservesOrder())
+ throw new UnsupportedOperationException("KeyRange based on keys can only be used with a order preserving paritioner");
+ if (jobKeyRange.start_token != null)
+ throw new IllegalArgumentException("only start_key supported");
+ if (jobKeyRange.end_token != null)
+ throw new IllegalArgumentException("only start_key supported");
+ jobRange = new Range<Token>(partitioner.getToken(jobKeyRange.start_key),
+ partitioner.getToken(jobKeyRange.end_key),
+ partitioner);
+ }
+ }
+
+ for (TokenRange range : masterRangeNodes)
+ {
+ if (jobRange == null)
+ {
+ // for each range, pick a live owner and ask it to compute bite-sized splits
+ splitfutures.add(executor.submit(new SplitCallable(range, conf)));
+ }
+ else
+ {
+ Range<Token> dhtRange = new Range<Token>(partitioner.getTokenFactory().fromString(range.start_token),
+ partitioner.getTokenFactory().fromString(range.end_token),
+ partitioner);
+
+ if (dhtRange.intersects(jobRange))
+ {
+ for (Range<Token> intersection: dhtRange.intersectionWith(jobRange))
+ {
+ range.start_token = partitioner.getTokenFactory().toString(intersection.left);
+ range.end_token = partitioner.getTokenFactory().toString(intersection.right);
+ // for each range, pick a live owner and ask it to compute bite-sized splits
+ splitfutures.add(executor.submit(new SplitCallable(range, conf)));
+ }
+ }
+ }
+ }
+
+ // wait until we have all the results back
+ for (Future<List<InputSplit>> futureInputSplits : splitfutures)
+ {
+ try
+ {
+ splits.addAll(futureInputSplits.get());
+ }
+ catch (Exception e)
+ {
+ throw new IOException("Could not get input splits", e);
+ }
+ }
+ }
+ finally
+ {
+ executor.shutdownNow();
+ }
+
+ assert splits.size() > 0;
+ Collections.shuffle(splits, new Random(System.nanoTime()));
+ return splits;
+ }
+
+ /**
+ * Gets a token range and splits it up according to the suggested
+ * size into input splits that Hadoop can use.
+ */
+ class SplitCallable implements Callable<List<InputSplit>>
+ {
+
+ private final TokenRange range;
+ private final Configuration conf;
+
+ public SplitCallable(TokenRange tr, Configuration conf)
+ {
+ this.range = tr;
+ this.conf = conf;
+ }
+
+ public List<InputSplit> call() throws Exception
+ {
+ ArrayList<InputSplit> splits = new ArrayList<InputSplit>();
+ List<CfSplit> subSplits = getSubSplits(keyspace, cfName, range, conf);
+ assert range.rpc_endpoints.size() == range.endpoints.size() : "rpc_endpoints size must match endpoints size";
+ // turn the sub-ranges into InputSplits
+ String[] endpoints = range.endpoints.toArray(new String[range.endpoints.size()]);
+ // hadoop needs hostname, not ip
+ int endpointIndex = 0;
+ for (String endpoint: range.rpc_endpoints)
+ {
+ String endpoint_address = endpoint;
+ if (endpoint_address == null || endpoint_address.equals("0.0.0.0"))
+ endpoint_address = range.endpoints.get(endpointIndex);
+ endpoints[endpointIndex++] = InetAddress.getByName(endpoint_address).getHostName();
+ }
+
+ Token.TokenFactory factory = partitioner.getTokenFactory();
+ for (CfSplit subSplit : subSplits)
+ {
+ Token left = factory.fromString(subSplit.getStart_token());
+ Token right = factory.fromString(subSplit.getEnd_token());
+ Range<Token> range = new Range<Token>(left, right, partitioner);
+ List<Range<Token>> ranges = range.isWrapAround() ? range.unwrap() : ImmutableList.of(range);
+ for (Range<Token> subrange : ranges)
+ {
+ ColumnFamilySplit split =
+ new ColumnFamilySplit(
+ factory.toString(subrange.left),
+ factory.toString(subrange.right),
+ subSplit.getRow_count(),
+ endpoints);
+
+ logger.debug("adding " + split);
+ splits.add(split);
+ }
+ }
+ return splits;
+ }
+ }
+
+ private List<CfSplit> getSubSplits(String keyspace, String cfName, TokenRange range, Configuration conf) throws IOException
+ {
+ int splitsize = ConfigHelper.getInputSplitSize(conf);
+ for (int i = 0; i < range.rpc_endpoints.size(); i++)
+ {
+ String host = range.rpc_endpoints.get(i);
+
+ if (host == null || host.equals("0.0.0.0"))
+ host = range.endpoints.get(i);
+
+ try
+ {
+ Cassandra.Client client = ConfigHelper.createConnection(conf, host, ConfigHelper.getInputRpcPort(conf));
+ client.set_keyspace(keyspace);
+
+ try
+ {
+ return client.describe_splits_ex(cfName, range.start_token, range.end_token, splitsize);
+ }
+ catch (TApplicationException e)
+ {
+ // fallback to guessing split size if talking to a server without describe_splits_ex method
+ if (e.getType() == TApplicationException.UNKNOWN_METHOD)
+ {
+ List<String> splitPoints = client.describe_splits(cfName, range.start_token, range.end_token, splitsize);
+ return tokenListToSplits(splitPoints, splitsize);
+ }
+ throw e;
+ }
+ }
+ catch (IOException e)
+ {
+ logger.debug("failed connect to endpoint " + host, e);
+ }
+ catch (InvalidRequestException e)
+ {
+ throw new RuntimeException(e);
+ }
+ catch (TException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+ throw new IOException("failed connecting to all endpoints " + StringUtils.join(range.endpoints, ","));
+ }
+
+ private List<CfSplit> tokenListToSplits(List<String> splitTokens, int splitsize)
+ {
+ List<CfSplit> splits = Lists.newArrayListWithExpectedSize(splitTokens.size() - 1);
+ for (int j = 0; j < splitTokens.size() - 1; j++)
+ splits.add(new CfSplit(splitTokens.get(j), splitTokens.get(j + 1), splitsize));
+ return splits;
+ }
+
+ private List<TokenRange> getRangeMap(Configuration conf) throws IOException
+ {
+ Cassandra.Client client = ConfigHelper.getClientFromInputAddressList(conf);
+
+ List<TokenRange> map;
+ try
+ {
+ map = client.describe_ring(ConfigHelper.getInputKeyspace(conf));
+ }
+ catch (InvalidRequestException e)
+ {
+ throw new RuntimeException(e);
+ }
+ catch (TException e)
+ {
+ throw new RuntimeException(e);
+ }
+ return map;
+ }
+
+ //
+ // Old Hadoop API
+ //
+ public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException
+ {
+ TaskAttemptContext tac = new TaskAttemptContext(jobConf, new TaskAttemptID());
+ List<org.apache.hadoop.mapreduce.InputSplit> newInputSplits = this.getSplits(tac);
+ org.apache.hadoop.mapred.InputSplit[] oldInputSplits = new org.apache.hadoop.mapred.InputSplit[newInputSplits.size()];
+ for (int i = 0; i < newInputSplits.size(); i++)
+ oldInputSplits[i] = (ColumnFamilySplit)newInputSplits.get(i);
+ return oldInputSplits;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
new file mode 100644
index 0000000..5a03777
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop;
+
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.auth.IAuthenticator;
+import org.apache.cassandra.thrift.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TTransport;
+
+/**
+ * The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
+ * OutputFormat that allows reduce tasks to store keys (and corresponding
+ * values) as Cassandra rows (and respective columns) in a given
+ * ColumnFamily.
+ *
+ * <p>
+ * As is the case with the {@link ColumnFamilyInputFormat}, you need to set the
+ * Keyspace and ColumnFamily in your
+ * Hadoop job Configuration. The {@link ConfigHelper} class, through its
+ * {@link ConfigHelper#setOutputColumnFamily} method, is provided to make this
+ * simple.
+ * </p>
+ *
+ * <p>
+ * For the sake of performance, this class employs a lazy write-back caching
+ * mechanism, where its record writer batches mutations created based on the
+ * reduce's inputs (in a task-specific map), and periodically makes the changes
+ * official by sending a batch mutate request to Cassandra.
+ * </p>
+ * @param <Y>
+ */
+public abstract class AbstractColumnFamilyOutputFormat<K, Y> extends OutputFormat<K, Y> implements org.apache.hadoop.mapred.OutputFormat<K, Y>
+{
+ public static final String BATCH_THRESHOLD = "mapreduce.output.columnfamilyoutputformat.batch.threshold";
+ public static final String QUEUE_SIZE = "mapreduce.output.columnfamilyoutputformat.queue.size";
+ private static final Logger logger = LoggerFactory.getLogger(AbstractColumnFamilyOutputFormat.class);
+
+
+ /**
+ * Check for validity of the output-specification for the job.
+ *
+ * @param context
+ * information about the job
+ * @throws IOException
+ * when output should not be attempted
+ */
+ public void checkOutputSpecs(JobContext context)
+ {
+ checkOutputSpecs(context.getConfiguration());
+ }
+
+ protected void checkOutputSpecs(Configuration conf)
+ {
+ if (ConfigHelper.getOutputKeyspace(conf) == null)
+ throw new UnsupportedOperationException("You must set the keyspace with setOutputKeyspace()");
+ if (ConfigHelper.getOutputPartitioner(conf) == null)
+ throw new UnsupportedOperationException("You must set the output partitioner to the one used by your Cassandra cluster");
+ if (ConfigHelper.getOutputInitialAddress(conf) == null)
+ throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node");
+ }
+
+ /** Fills the deprecated OutputFormat interface for streaming. */
+ @Deprecated
+ public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
+ {
+ checkOutputSpecs(job);
+ }
+
+ /**
+ * The OutputCommitter for this format does not write any data to the DFS.
+ *
+ * @param context
+ * the task context
+ * @return an output committer
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
+ {
+ return new NullOutputCommitter();
+ }
+
+ /**
+ * Connects to the given server:port and returns a client based on the given socket that points to the configured
+ * keyspace, and is logged in with the configured credentials.
+ *
+ * @param host fully qualified host name to connect to
+ * @param port RPC port of the server
+ * @param conf a job configuration
+ * @return a cassandra client
+ * @throws Exception set of thrown exceptions may be implementation defined,
+ * depending on the used transport factory
+ */
+ public static Cassandra.Client createAuthenticatedClient(String host, int port, Configuration conf) throws Exception
+ {
+ logger.debug("Creating authenticated client for CF output format");
+ TTransport transport = ConfigHelper.getClientTransportFactory(conf).openTransport(host, port, conf);
+ TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
+ Cassandra.Client client = new Cassandra.Client(binaryProtocol);
+ client.set_keyspace(ConfigHelper.getOutputKeyspace(conf));
+ if (ConfigHelper.getOutputKeyspaceUserName(conf) != null)
+ {
+ Map<String, String> creds = new HashMap<String, String>();
+ creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getOutputKeyspaceUserName(conf));
+ creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getOutputKeyspacePassword(conf));
+ AuthenticationRequest authRequest = new AuthenticationRequest(creds);
+ client.login(authRequest);
+ }
+ logger.debug("Authenticated client for CF output format created successfully");
+ return client;
+ }
+
+ /**
+ * An {@link OutputCommitter} that does nothing.
+ */
+ private static class NullOutputCommitter extends OutputCommitter
+ {
+ public void abortTask(TaskAttemptContext taskContext) { }
+
+ public void cleanupJob(JobContext jobContext) { }
+
+ public void commitTask(TaskAttemptContext taskContext) { }
+
+ public boolean needsTaskCommit(TaskAttemptContext taskContext)
+ {
+ return false;
+ }
+
+ public void setupJob(JobContext jobContext) { }
+
+ public void setupTask(TaskAttemptContext taskContext) { }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
new file mode 100644
index 0000000..6428db3
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
@@ -0,0 +1,193 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop;
+
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.cassandra.client.RingCache;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.thrift.transport.TTransport;
+
+
+/**
+ * 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>
+ * 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>
+ *
+ * @see ColumnFamilyOutputFormat
+ */
+public abstract class AbstractColumnFamilyRecordWriter<K, Y> extends RecordWriter<K, Y> implements org.apache.hadoop.mapred.RecordWriter<K, Y>
+{
+ // The configuration this writer is associated with.
+ protected final Configuration conf;
+
+ // The ring cache that describes the token ranges each node in the ring is
+ // responsible for. This is what allows us to group the mutations by
+ // the endpoints they should be targeted at. The targeted endpoint
+ // essentially
+ // acts as the primary replica for the rows being affected by the mutations.
+ protected final RingCache ringCache;
+
+ // The number of mutations to buffer per endpoint
+ protected final int queueSize;
+
+ protected final long batchThreshold;
+
+ protected final ConsistencyLevel consistencyLevel;
+ protected Progressable progressable;
+
+ protected AbstractColumnFamilyRecordWriter(Configuration conf)
+ {
+ this.conf = conf;
+ this.ringCache = new RingCache(conf);
+ this.queueSize = conf.getInt(AbstractColumnFamilyOutputFormat.QUEUE_SIZE, 32 * FBUtilities.getAvailableProcessors());
+ batchThreshold = conf.getLong(AbstractColumnFamilyOutputFormat.BATCH_THRESHOLD, 32);
+ consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getWriteConsistencyLevel(conf));
+ }
+
+ /**
+ * Close this <code>RecordWriter</code> to future operations, but not before
+ * flushing out the batched mutations.
+ *
+ * @param context the context of the task
+ * @throws IOException
+ */
+ public void close(TaskAttemptContext context) throws IOException, InterruptedException
+ {
+ close();
+ }
+
+ /** Fills the deprecated RecordWriter interface for streaming. */
+ @Deprecated
+ public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
+ {
+ close();
+ }
+
+ protected abstract void close() throws IOException;
+
+ /**
+ * A client that runs in a threadpool and connects to the list of endpoints for a particular
+ * range. Mutations for keys in that range are sent to this client via a queue.
+ */
+ public abstract class AbstractRangeClient<K> extends Thread
+ {
+ // The list of endpoints for this range
+ protected final List<InetAddress> endpoints;
+ // A bounded queue of incoming mutations for this range
+ protected final BlockingQueue<Pair<ByteBuffer, K>> queue = new ArrayBlockingQueue<Pair<ByteBuffer, K>>(queueSize);
+
+ protected volatile boolean run = true;
+ // we want the caller to know if something went wrong, so we record any unrecoverable exception while writing
+ // so we can throw it on the caller's stack when he calls put() again, or if there are no more put calls,
+ // when the client is closed.
+ protected volatile IOException lastException;
+
+ protected Cassandra.Client client;
+
+ /**
+ * Constructs an {@link AbstractRangeClient} for the given endpoints.
+ * @param endpoints the possible endpoints to execute the mutations on
+ */
+ public AbstractRangeClient(List<InetAddress> endpoints)
+ {
+ super("client-" + endpoints);
+ this.endpoints = endpoints;
+ }
+
+ /**
+ * enqueues the given value to Cassandra
+ */
+ public void put(Pair<ByteBuffer, K> value) throws IOException
+ {
+ while (true)
+ {
+ if (lastException != null)
+ throw lastException;
+ try
+ {
+ if (queue.offer(value, 100, TimeUnit.MILLISECONDS))
+ break;
+ }
+ catch (InterruptedException e)
+ {
+ throw new AssertionError(e);
+ }
+ }
+ }
+
+ public void close() throws IOException
+ {
+ // stop the run loop. this will result in closeInternal being called by the time join() finishes.
+ run = false;
+ interrupt();
+ try
+ {
+ this.join();
+ }
+ catch (InterruptedException e)
+ {
+ throw new AssertionError(e);
+ }
+
+ if (lastException != null)
+ throw lastException;
+ }
+
+ protected void closeInternal()
+ {
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ }
+ }
+
+ /**
+ * Loops collecting mutations from the queue and sending to Cassandra
+ */
+ public abstract void run();
+
+ @Override
+ public String toString()
+ {
+ return "#<Client for " + endpoints.toString() + ">";
+ }
+ }
+}
+
[04/10] Add CQL3 input/output formats patch by Alex Liu;
reviewed by jbellis and Mike Schrag for CASSANDRA-4421
Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java
new file mode 100644
index 0000000..3939e0b
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java
@@ -0,0 +1,386 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.hadoop.AbstractColumnFamilyRecordWriter;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.Progressable;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The <code>ColumnFamilyRecordWriter</code> maps the output <key, value>
+ * pairs to a Cassandra column family. In particular, it applies the binded variables
+ * in the value to the prepared statement, which it associates with the key, and in
+ * turn the responsible endpoint.
+ *
+ * <p>
+ * Furthermore, this writer groups the cql queries by the endpoint responsible for
+ * the rows being affected. This allows the cql queries to be executed in parallel,
+ * directly to a responsible endpoint.
+ * </p>
+ *
+ * @see ColumnFamilyOutputFormat
+ */
+final class ColumnFamilyRecordWriter extends AbstractColumnFamilyRecordWriter<Map<String, ByteBuffer>, List<ByteBuffer>>
+{
+ private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyRecordWriter.class);
+
+ // handles for clients for each range running in the threadpool
+ private final Map<Range, RangeClient> clients;
+
+ // host to prepared statement id mappings
+ private ConcurrentHashMap<Cassandra.Client, Integer> preparedStatements = new ConcurrentHashMap<Cassandra.Client, Integer>();
+
+ private final String cql;
+
+ private AbstractType<?> keyValidator;
+ private String [] partitionkeys;
+
+ /**
+ * Upon construction, obtain the map that this writer will use to collect
+ * mutations, and the ring cache for the given keyspace.
+ *
+ * @param context the task attempt context
+ * @throws IOException
+ */
+ ColumnFamilyRecordWriter(TaskAttemptContext context) throws IOException
+ {
+ this(context.getConfiguration());
+ this.progressable = new Progressable(context);
+ }
+
+ ColumnFamilyRecordWriter(Configuration conf, Progressable progressable) throws IOException
+ {
+ this(conf);
+ this.progressable = progressable;
+ }
+
+ ColumnFamilyRecordWriter(Configuration conf) throws IOException
+ {
+ super(conf);
+ this.clients = new HashMap<Range, RangeClient>();
+ cql = CQLConfigHelper.getOutputCql(conf);
+
+ try
+ {
+ String host = getAnyHost();
+ int port = ConfigHelper.getOutputRpcPort(conf);
+ Cassandra.Client client = ColumnFamilyOutputFormat.createAuthenticatedClient(host, port, conf);
+ retrievePartitionKeyValidator(client);
+
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ client = null;
+ }
+ }
+ catch (Exception e)
+ {
+ throw new IOException(e);
+ }
+ }
+
+ @Override
+ public void close() throws IOException
+ {
+ // close all the clients before throwing anything
+ IOException clientException = null;
+ for (RangeClient client : clients.values())
+ {
+ try
+ {
+ client.close();
+ }
+ catch (IOException e)
+ {
+ clientException = e;
+ }
+ }
+
+ if (clientException != null)
+ throw clientException;
+ }
+
+ /**
+ * If the key is to be associated with a valid value, a mutation is created
+ * for it with the given column family and columns. In the event the value
+ * in the column is missing (i.e., null), then it is marked for
+ * {@link Deletion}. Similarly, if the entire value for a key is missing
+ * (i.e., null), then the entire key is marked for {@link Deletion}.
+ * </p>
+ *
+ * @param keybuff
+ * the key to write.
+ * @param values
+ * the values to write.
+ * @throws IOException
+ */
+ @Override
+ public void write(Map<String, ByteBuffer> keys, List<ByteBuffer> values) throws IOException
+ {
+ ByteBuffer rowKey = getRowKey(keys);
+ Range<Token> range = ringCache.getRange(rowKey);
+
+ // get the client for the given range, or create a new one
+ RangeClient client = clients.get(range);
+ if (client == null)
+ {
+ // haven't seen keys for this range: create new client
+ client = new RangeClient(ringCache.getEndpoint(range));
+ client.start();
+ clients.put(range, client);
+ }
+
+ client.put(Pair.create(rowKey, values));
+ progressable.progress();
+ }
+
+ /**
+ * A client that runs in a threadpool and connects to the list of endpoints for a particular
+ * range. Binded variable values for keys in that range are sent to this client via a queue.
+ */
+ public class RangeClient extends AbstractRangeClient<List<ByteBuffer>>
+ {
+ /**
+ * Constructs an {@link RangeClient} for the given endpoints.
+ * @param endpoints the possible endpoints to execute the mutations on
+ */
+ public RangeClient(List<InetAddress> endpoints)
+ {
+ super(endpoints);
+ }
+
+ /**
+ * Loops collecting cql binded variable values from the queue and sending to Cassandra
+ */
+ public void run()
+ {
+ outer:
+ while (run || !queue.isEmpty())
+ {
+ Pair<ByteBuffer, List<ByteBuffer>> bindVariables;
+ try
+ {
+ bindVariables = queue.take();
+ }
+ catch (InterruptedException e)
+ {
+ // re-check loop condition after interrupt
+ continue;
+ }
+
+ Iterator<InetAddress> iter = endpoints.iterator();
+ while (true)
+ {
+ // send the mutation to the last-used endpoint. first time through, this will NPE harmlessly.
+ try
+ {
+ int i = 0;
+ int itemId = preparedStatement(client);
+ while (bindVariables != null)
+ {
+ client.execute_prepared_cql3_query(itemId, bindVariables.right, ConsistencyLevel.ONE);
+ i++;
+
+ if (i >= batchThreshold)
+ break;
+
+ bindVariables = queue.poll();
+ }
+
+ break;
+ }
+ catch (Exception e)
+ {
+ closeInternal();
+ if (!iter.hasNext())
+ {
+ lastException = new IOException(e);
+ break outer;
+ }
+ }
+
+ // attempt to connect to a different endpoint
+ try
+ {
+ InetAddress address = iter.next();
+ String host = address.getHostName();
+ int port = ConfigHelper.getOutputRpcPort(conf);
+ client = ColumnFamilyOutputFormat.createAuthenticatedClient(host, port, conf);
+ }
+ catch (Exception e)
+ {
+ closeInternal();
+ // TException means something unexpected went wrong to that endpoint, so
+ // we should try again to another. Other exceptions (auth or invalid request) are fatal.
+ if ((!(e instanceof TException)) || !iter.hasNext())
+ {
+ lastException = new IOException(e);
+ break outer;
+ }
+ }
+ }
+ }
+ }
+
+ /** get prepared statement id from cache, otherwise prepare it from Cassandra server*/
+ private int preparedStatement(Cassandra.Client client)
+ {
+ Integer itemId = preparedStatements.get(client);
+ if (itemId == null)
+ {
+ CqlPreparedResult result;
+ try
+ {
+ result = client.prepare_cql3_query(ByteBufferUtil.bytes(cql), Compression.NONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ throw new RuntimeException("failed to prepare cql query " + cql, e);
+ }
+ catch (TException e)
+ {
+ throw new RuntimeException("failed to prepare cql query " + cql, e);
+ }
+
+ Integer previousId = preparedStatements.putIfAbsent(client, Integer.valueOf(result.itemId));
+ itemId = previousId == null ? result.itemId : previousId;
+ }
+ return itemId;
+ }
+ }
+
+ private ByteBuffer getRowKey(Map<String, ByteBuffer> keysMap)
+ {
+ //current row key
+ ByteBuffer rowKey;
+ if (keyValidator instanceof CompositeType)
+ {
+ ByteBuffer[] keys = new ByteBuffer[partitionkeys.length];
+ for (int i = 0; i< keys.length; i++)
+ keys[i] = keysMap.get(partitionkeys[i]);
+
+ rowKey = ((CompositeType) keyValidator).build(keys);
+ }
+ else
+ {
+ rowKey = keysMap.get(partitionkeys[0]);
+ }
+ return rowKey;
+ }
+
+ /** retrieve the key validator from system.schema_columnfamilies table */
+ private void retrievePartitionKeyValidator(Cassandra.Client client) throws Exception
+ {
+ String keyspace = ConfigHelper.getOutputKeyspace(conf);
+ String cfName = ConfigHelper.getOutputColumnFamily(conf);
+ String query = "SELECT key_validator," +
+ " key_aliases " +
+ "FROM system.schema_columnfamilies " +
+ "WHERE keyspace_name='%s' and columnfamily_name='%s'";
+ String formatted = String.format(query, keyspace, cfName);
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(formatted), Compression.NONE, ConsistencyLevel.ONE);
+
+ Column rawKeyValidator = result.rows.get(0).columns.get(0);
+ String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));
+ keyValidator = parseType(validator);
+
+ Column rawPartitionKeys = result.rows.get(0).columns.get(1);
+ String keyString = ByteBufferUtil.string(ByteBuffer.wrap(rawPartitionKeys.getValue()));
+ logger.debug("partition keys: " + keyString);
+
+ List<String> keys = FBUtilities.fromJsonList(keyString);
+ partitionkeys = new String [keys.size()];
+ int i=0;
+ for (String key: keys)
+ {
+ partitionkeys[i] = key;
+ i++;
+ }
+ }
+
+ private AbstractType<?> parseType(String type) throws IOException
+ {
+ try
+ {
+ // always treat counters like longs, specifically CCT.compose is not what we need
+ if (type != null && type.equals("org.apache.cassandra.db.marshal.CounterColumnType"))
+ return LongType.instance;
+ return TypeParser.parse(type);
+ }
+ catch (ConfigurationException e)
+ {
+ throw new IOException(e);
+ }
+ catch (SyntaxException e)
+ {
+ throw new IOException(e);
+ }
+ }
+
+ private String getAnyHost() throws IOException, InvalidRequestException, TException
+ {
+ Cassandra.Client client = ConfigHelper.getClientFromOutputAddressList(conf);
+ List<TokenRange> ring = client.describe_ring(ConfigHelper.getOutputKeyspace(conf));
+ try
+ {
+ for (TokenRange range : ring)
+ return range.endpoints.get(0);
+ }
+ finally
+ {
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ client = null;
+ }
+ }
+ throw new IOException("There are no endpoints");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java b/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java
new file mode 100644
index 0000000..0a73043
--- /dev/null
+++ b/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java
@@ -0,0 +1,70 @@
+package org.apache.cassandra.thrift;
+
+/*
+ *
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.thrift.transport.TTransport;
+
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ * Transport factory for establishing thrift connections from clients to a remote server.
+ */
+public interface TClientTransportFactory
+{
+ static final String PROPERTY_KEY = "cassandra.client.transport.factory";
+ static final String LONG_OPTION = "transport-factory";
+ static final String SHORT_OPTION = "tr";
+
+ /**
+ * Opens a client transport to a thrift server.
+ * Example:
+ *
+ * <pre>
+ * TTransport transport = clientTransportFactory.openTransport(address, port);
+ * Cassandra.Iface client = new Cassandra.Client(new BinaryProtocol(transport));
+ * </pre>
+ *
+ * @param host fully qualified hostname of the server
+ * @param port RPC port of the server
+ * @param conf Hadoop configuration
+ * @return open and ready to use transport
+ * @throws Exception implementation defined; usually throws TTransportException or IOException
+ * if the connection cannot be established
+ */
+ TTransport openTransport(String host, int port, Configuration conf) throws Exception;
+
+ /**
+ * Sets an implementation defined set of options.
+ * Keys in this map must conform to the set set returned by TClientTransportFactory#supportedOptions.
+ * @param options option map
+ */
+ void setOptions(Map<String, String> options);
+
+ /**
+ * @return set of options supported by this transport factory implementation
+ */
+ Set<String> supportedOptions();
+}
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java b/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
index 792618d..5905f4a 100644
--- a/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
+++ b/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
@@ -21,6 +21,10 @@ package org.apache.cassandra.thrift;
*
*/
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
import org.apache.cassandra.hadoop.ConfigHelper;
import org.apache.thrift.transport.TFramedTransport;
import org.apache.thrift.transport.TSocket;
@@ -29,7 +33,7 @@ import org.apache.thrift.transport.TTransportException;
import org.apache.hadoop.conf.Configuration;
-public class TFramedTransportFactory implements ITransportFactory
+public class TFramedTransportFactory implements ITransportFactory, TClientTransportFactory
{
public TTransport openTransport(TSocket socket, Configuration conf) throws TTransportException
{
@@ -37,4 +41,22 @@ public class TFramedTransportFactory implements ITransportFactory
transport.open();
return transport;
}
+
+ //
+ public TTransport openTransport(String host, int port, Configuration conf) throws TTransportException
+ {
+ TSocket socket = new TSocket(host, port);
+ TTransport transport = new TFramedTransport(socket, ConfigHelper.getThriftFramedTransportSize(conf));
+ transport.open();
+ return transport;
+ }
+
+ public void setOptions(Map<String, String> options)
+ {
+ }
+
+ public Set<String> supportedOptions()
+ {
+ return Collections.emptySet();
+ }
}
[09/10] merge from 1.2
Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
index d727a20..724ba7d 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
@@ -20,23 +20,9 @@ package org.apache.cassandra.hadoop;
import java.io.IOException;
import java.nio.ByteBuffer;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.auth.IAuthenticator;
import org.apache.cassandra.thrift.*;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.*;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TSocket;
-import org.apache.thrift.transport.TTransport;
-
-import javax.security.auth.login.LoginException;
/**
* The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
@@ -59,62 +45,10 @@ import javax.security.auth.login.LoginException;
* official by sending a batch mutate request to Cassandra.
* </p>
*/
-public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutation>>
- implements org.apache.hadoop.mapred.OutputFormat<ByteBuffer,List<Mutation>>
+public class ColumnFamilyOutputFormat extends AbstractColumnFamilyOutputFormat<ByteBuffer,List<Mutation>>
{
- public static final String BATCH_THRESHOLD = "mapreduce.output.columnfamilyoutputformat.batch.threshold";
- public static final String QUEUE_SIZE = "mapreduce.output.columnfamilyoutputformat.queue.size";
- private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyOutputFormat.class);
-
-
- /**
- * Check for validity of the output-specification for the job.
- *
- * @param context
- * information about the job
- * @throws IOException
- * when output should not be attempted
- */
- @Override
- public void checkOutputSpecs(JobContext context)
- {
- checkOutputSpecs(context.getConfiguration());
- }
-
- private void checkOutputSpecs(Configuration conf)
- {
- if (ConfigHelper.getOutputKeyspace(conf) == null)
- throw new UnsupportedOperationException("You must set the keyspace with setOutputKeyspace()");
- if (ConfigHelper.getOutputPartitioner(conf) == null)
- throw new UnsupportedOperationException("You must set the output partitioner to the one used by your Cassandra cluster");
- if (ConfigHelper.getOutputInitialAddress(conf) == null)
- throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node");
- }
-
- /**
- * The OutputCommitter for this format does not write any data to the DFS.
- *
- * @param context
- * the task context
- * @return an output committer
- * @throws IOException
- * @throws InterruptedException
- */
- @Override
- public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
- {
- return new NullOutputCommitter();
- }
-
/** Fills the deprecated OutputFormat interface for streaming. */
@Deprecated
- public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
- {
- checkOutputSpecs(job);
- }
-
- /** Fills the deprecated OutputFormat interface for streaming. */
- @Deprecated @Override
public ColumnFamilyRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress)
{
return new ColumnFamilyRecordWriter(job, new Progressable(progress));
@@ -128,62 +62,8 @@ public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutat
* @return a {@link RecordWriter} to write the output for the job.
* @throws IOException
*/
- @Override
public ColumnFamilyRecordWriter getRecordWriter(final TaskAttemptContext context) throws InterruptedException
{
return new ColumnFamilyRecordWriter(context);
}
-
- /**
- * Return a client based on the given socket that points to the configured
- * keyspace, and is logged in with the configured credentials.
- *
- * @param socket a socket pointing to a particular node, seed or otherwise
- * @param conf a job configuration
- * @return a cassandra client
- * @throws InvalidRequestException
- * @throws TException
- * @throws AuthenticationException
- * @throws AuthorizationException
- */
- public static Cassandra.Client createAuthenticatedClient(TSocket socket, Configuration conf)
- throws InvalidRequestException, TException, AuthenticationException, AuthorizationException, LoginException
- {
- logger.debug("Creating authenticated client for CF output format");
- TTransport transport = ConfigHelper.getOutputTransportFactory(conf).openTransport(socket, conf);
- TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
- Cassandra.Client client = new Cassandra.Client(binaryProtocol);
- client.set_keyspace(ConfigHelper.getOutputKeyspace(conf));
- if (ConfigHelper.getOutputKeyspaceUserName(conf) != null)
- {
- Map<String, String> creds = new HashMap<String, String>();
- creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getOutputKeyspaceUserName(conf));
- creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getOutputKeyspacePassword(conf));
- AuthenticationRequest authRequest = new AuthenticationRequest(creds);
- client.login(authRequest);
- }
- logger.debug("Authenticated client for CF output format created successfully");
- return client;
- }
-
- /**
- * An {@link OutputCommitter} that does nothing.
- */
- public static class NullOutputCommitter extends OutputCommitter
- {
- public void abortTask(TaskAttemptContext taskContext) { }
-
- public void cleanupJob(JobContext jobContext) { }
-
- public void commitTask(TaskAttemptContext taskContext) { }
-
- public boolean needsTaskCommit(TaskAttemptContext taskContext)
- {
- return false;
- }
-
- public void setupJob(JobContext jobContext) { }
-
- public void setupTask(TaskAttemptContext taskContext) { }
- }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index 2b258b2..9022202 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -24,33 +24,16 @@ import java.nio.ByteBuffer;
import java.util.*;
import com.google.common.collect.*;
-
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TTransport;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.auth.IAuthenticator;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.db.Column;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.CompositeType;
import org.apache.cassandra.db.marshal.TypeParser;
import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.thrift.AuthenticationRequest;
-import org.apache.cassandra.thrift.Cassandra;
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.ColumnOrSuperColumn;
-import org.apache.cassandra.thrift.ColumnParent;
-import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.CounterColumn;
-import org.apache.cassandra.thrift.CounterSuperColumn;
-import org.apache.cassandra.thrift.IndexExpression;
-import org.apache.cassandra.thrift.KeyRange;
-import org.apache.cassandra.thrift.KeySlice;
-import org.apache.cassandra.thrift.KsDef;
-import org.apache.cassandra.thrift.SlicePredicate;
-import org.apache.cassandra.thrift.SuperColumn;
+import org.apache.cassandra.thrift.*;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
@@ -59,7 +42,7 @@ import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.thrift.TException;
-import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap<ByteBuffer, Column>>
implements org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<ByteBuffer, Column>>
@@ -75,14 +58,14 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
private boolean isEmptyPredicate;
private int totalRowCount; // total number of rows to fetch
private int batchSize; // fetch this many per batch
- private String cfName;
private String keyspace;
- private TSocket socket;
+ private String cfName;
private Cassandra.Client client;
private ConsistencyLevel consistencyLevel;
private int keyBufferSize = 8192;
private List<IndexExpression> filter;
+
public ColumnFamilyRecordReader()
{
this(ColumnFamilyRecordReader.CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT);
@@ -96,11 +79,11 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
public void close()
{
- if (socket != null && socket.isOpen())
+ if (client != null)
{
- socket.close();
- socket = null;
- client = null;
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
}
}
@@ -155,36 +138,25 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
predicate = ConfigHelper.getInputSlicePredicate(conf);
boolean widerows = ConfigHelper.getInputIsWide(conf);
isEmptyPredicate = isEmptyPredicate(predicate);
- totalRowCount = (int) this.split.getLength();
+ totalRowCount = (this.split.getLength() < Long.MAX_VALUE)
+ ? (int) this.split.getLength()
+ : ConfigHelper.getInputSplitSize(conf);
batchSize = ConfigHelper.getRangeBatchSize(conf);
cfName = ConfigHelper.getInputColumnFamily(conf);
consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getReadConsistencyLevel(conf));
-
keyspace = ConfigHelper.getInputKeyspace(conf);
try
{
- // only need to connect once
- if (socket != null && socket.isOpen())
+ if (client != null)
return;
// create connection using thrift
String location = getLocation();
- socket = new TSocket(location, ConfigHelper.getInputRpcPort(conf));
- TTransport transport = ConfigHelper.getInputTransportFactory(conf).openTransport(socket, conf);
- TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
- client = new Cassandra.Client(binaryProtocol);
-
- // log in
- client.set_keyspace(keyspace);
- if (ConfigHelper.getInputKeyspaceUserName(conf) != null)
- {
- Map<String, String> creds = new HashMap<String, String>();
- creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf));
- creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf));
- AuthenticationRequest authRequest = new AuthenticationRequest(creds);
- client.login(authRequest);
- }
+
+ int port = ConfigHelper.getInputRpcPort(conf);
+ client = ColumnFamilyInputFormat.createAuthenticatedClient(location, port, conf);
+
}
catch (Exception e)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
index 10de8ce..50ec059 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
@@ -22,21 +22,14 @@ import java.io.IOException;
import java.net.InetAddress;
import java.nio.ByteBuffer;
import java.util.*;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-import org.apache.cassandra.client.RingCache;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.thrift.TException;
-import org.apache.thrift.transport.TSocket;
/**
@@ -52,38 +45,18 @@ import org.apache.thrift.transport.TSocket;
* </p>
*
* @see ColumnFamilyOutputFormat
- * @see OutputFormat
- *
*/
-final class ColumnFamilyRecordWriter extends RecordWriter<ByteBuffer,List<Mutation>>
-implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
+final class ColumnFamilyRecordWriter extends AbstractColumnFamilyRecordWriter<ByteBuffer, List<Mutation>>
{
- // The configuration this writer is associated with.
- private final Configuration conf;
-
- // The ring cache that describes the token ranges each node in the ring is
- // responsible for. This is what allows us to group the mutations by
- // the endpoints they should be targeted at. The targeted endpoint
- // essentially
- // acts as the primary replica for the rows being affected by the mutations.
- private final RingCache ringCache;
-
- // The number of mutations to buffer per endpoint
- private final int queueSize;
-
// handles for clients for each range running in the threadpool
- private final Map<Range,RangeClient> clients;
- private final long batchThreshold;
-
- private final ConsistencyLevel consistencyLevel;
- private Progressable progressable;
-
-
+ private final Map<Range, RangeClient> clients;
+
/**
* Upon construction, obtain the map that this writer will use to collect
* mutations, and the ring cache for the given keyspace.
*
* @param context the task attempt context
+ * @throws IOException
*/
ColumnFamilyRecordWriter(TaskAttemptContext context)
{
@@ -99,14 +72,30 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
ColumnFamilyRecordWriter(Configuration conf)
{
- this.conf = conf;
- this.ringCache = new RingCache(conf);
- this.queueSize = conf.getInt(ColumnFamilyOutputFormat.QUEUE_SIZE, 32 * FBUtilities.getAvailableProcessors());
- this.clients = new HashMap<Range,RangeClient>();
- batchThreshold = conf.getLong(ColumnFamilyOutputFormat.BATCH_THRESHOLD, 32);
- consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getWriteConsistencyLevel(conf));
+ super(conf);
+ this.clients = new HashMap<Range, RangeClient>();
}
-
+
+ @Override
+ public void close() throws IOException
+ {
+ // close all the clients before throwing anything
+ IOException clientException = null;
+ for (RangeClient client : clients.values())
+ {
+ try
+ {
+ client.close();
+ }
+ catch (IOException e)
+ {
+ clientException = e;
+ }
+ }
+ if (clientException != null)
+ throw clientException;
+ }
+
/**
* If the key is to be associated with a valid value, a mutation is created
* for it with the given column family and columns. In the event the value
@@ -142,124 +131,22 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
}
/**
- * Close this <code>RecordWriter</code> to future operations, but not before
- * flushing out the batched mutations.
- *
- * @param context the context of the task
- * @throws IOException
- */
- @Override
- public void close(TaskAttemptContext context) throws IOException, InterruptedException
- {
- close();
- }
-
- /** Fills the deprecated RecordWriter interface for streaming. */
- @Deprecated
- public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
- {
- close();
- }
-
- private void close() throws IOException
- {
- // close all the clients before throwing anything
- IOException clientException = null;
- for (RangeClient client : clients.values())
- {
- try
- {
- client.close();
- }
- catch (IOException e)
- {
- clientException = e;
- }
- }
- if (clientException != null)
- throw clientException;
- }
-
- /**
* A client that runs in a threadpool and connects to the list of endpoints for a particular
* range. Mutations for keys in that range are sent to this client via a queue.
*/
- public class RangeClient extends Thread
+ public class RangeClient extends AbstractRangeClient<Mutation>
{
- // The list of endpoints for this range
- private final List<InetAddress> endpoints;
- private final String columnFamily = ConfigHelper.getOutputColumnFamily(conf);
- // A bounded queue of incoming mutations for this range
- private final BlockingQueue<Pair<ByteBuffer, Mutation>> queue = new ArrayBlockingQueue<Pair<ByteBuffer,Mutation>>(queueSize);
-
- private volatile boolean run = true;
- // we want the caller to know if something went wrong, so we record any unrecoverable exception while writing
- // so we can throw it on the caller's stack when he calls put() again, or if there are no more put calls,
- // when the client is closed.
- private volatile IOException lastException;
-
- private Cassandra.Client thriftClient;
- private TSocket thriftSocket;
-
+ public final String columnFamily = ConfigHelper.getOutputColumnFamily(conf);
+
/**
* Constructs an {@link RangeClient} for the given endpoints.
* @param endpoints the possible endpoints to execute the mutations on
*/
public RangeClient(List<InetAddress> endpoints)
{
- super("client-" + endpoints);
- this.endpoints = endpoints;
+ super(endpoints);
}
-
- /**
- * enqueues the given value to Cassandra
- */
- public void put(Pair<ByteBuffer,Mutation> value) throws IOException
- {
- while (true)
- {
- if (lastException != null)
- throw lastException;
- try
- {
- if (queue.offer(value, 100, TimeUnit.MILLISECONDS))
- break;
- }
- catch (InterruptedException e)
- {
- throw new AssertionError(e);
- }
- }
- }
-
- public void close() throws IOException
- {
- // stop the run loop. this will result in closeInternal being called by the time join() finishes.
- run = false;
- interrupt();
- try
- {
- this.join();
- }
- catch (InterruptedException e)
- {
- throw new AssertionError(e);
- }
-
- if (lastException != null)
- throw lastException;
- }
-
- private void closeInternal()
- {
- if (thriftSocket != null)
- {
- thriftSocket.close();
- thriftSocket = null;
- thriftClient = null;
- }
- }
-
+
/**
* Loops collecting mutations from the queue and sending to Cassandra
*/
@@ -302,7 +189,7 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
// send the mutation to the last-used endpoint. first time through, this will NPE harmlessly.
try
{
- thriftClient.batch_mutate(batch, consistencyLevel);
+ client.batch_mutate(batch, consistencyLevel);
break;
}
catch (Exception e)
@@ -319,8 +206,9 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
try
{
InetAddress address = iter.next();
- thriftSocket = new TSocket(address.getHostName(), ConfigHelper.getOutputRpcPort(conf));
- thriftClient = ColumnFamilyOutputFormat.createAuthenticatedClient(thriftSocket, conf);
+ String host = address.getHostName();
+ int port = ConfigHelper.getOutputRpcPort(conf);
+ client = ColumnFamilyOutputFormat.createAuthenticatedClient(host, port, conf);
}
catch (Exception e)
{
@@ -336,11 +224,5 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
}
}
}
-
- @Override
- public String toString()
- {
- return "#<Client for " + endpoints.toString() + ">";
- }
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
index ed1f160..69c7ddb 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.mapreduce.InputSplit;
import java.io.DataInput;
import java.io.DataOutput;
+import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
@@ -79,7 +80,6 @@ public class ColumnFamilySplit extends InputSplit implements Writable, org.apach
{
out.writeUTF(startToken);
out.writeUTF(endToken);
- out.writeLong(length);
out.writeInt(dataNodes.length);
for (String endpoint : dataNodes)
{
@@ -91,8 +91,6 @@ public class ColumnFamilySplit extends InputSplit implements Writable, org.apach
{
startToken = in.readUTF();
endToken = in.readUTF();
- length = in.readLong();
-
int numOfEndpoints = in.readInt();
dataNodes = new String[numOfEndpoints];
for(int i = 0; i < numOfEndpoints; i++)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
index 90f5045..3dcfdd7 100644
--- a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
+++ b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
@@ -21,11 +21,9 @@ package org.apache.cassandra.hadoop;
*/
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
+import com.google.common.collect.Maps;
import org.apache.cassandra.io.compress.CompressionParameters;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -41,11 +39,7 @@ import org.apache.thrift.TDeserializer;
import org.apache.thrift.TException;
import org.apache.thrift.TSerializer;
import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TSocket;
import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-
-import javax.security.auth.login.LoginException;
public class ConfigHelper
@@ -75,6 +69,7 @@ public class ConfigHelper
private static final String WRITE_CONSISTENCY_LEVEL = "cassandra.consistencylevel.write";
private static final String OUTPUT_COMPRESSION_CLASS = "cassandra.output.compression.class";
private static final String OUTPUT_COMPRESSION_CHUNK_LENGTH = "cassandra.output.compression.length";
+
private static final String INPUT_TRANSPORT_FACTORY_CLASS = "cassandra.input.transport.factory.class";
private static final String OUTPUT_TRANSPORT_FACTORY_CLASS = "cassandra.output.transport.factory.class";
private static final String THRIFT_FRAMED_TRANSPORT_SIZE_IN_MB = "cassandra.thrift.framed.size_mb";
@@ -562,48 +557,44 @@ public class ConfigHelper
{
try
{
- TSocket socket = new TSocket(host, port);
- TTransport transport = getInputTransportFactory(conf).openTransport(socket, conf);
+ TTransport transport = getClientTransportFactory(conf).openTransport(host, port, conf);
return new Cassandra.Client(new TBinaryProtocol(transport, true, true));
}
- catch (LoginException e)
- {
- throw new IOException("Unable to login to server " + host + ":" + port, e);
- }
- catch (TTransportException e)
+ catch (Exception e)
{
throw new IOException("Unable to connect to server " + host + ":" + port, e);
}
}
- public static ITransportFactory getInputTransportFactory(Configuration conf)
+ public static TClientTransportFactory getClientTransportFactory(Configuration conf)
{
- return getTransportFactory(conf.get(INPUT_TRANSPORT_FACTORY_CLASS, TFramedTransportFactory.class.getName()));
+ String factoryClassName = conf.get(
+ TClientTransportFactory.PROPERTY_KEY,
+ TFramedTransportFactory.class.getName());
+ TClientTransportFactory factory = getClientTransportFactory(factoryClassName);
+ Map<String, String> options = getOptions(conf, factory.supportedOptions());
+ factory.setOptions(options);
+ return factory;
}
- public static void setInputTransportFactoryClass(Configuration conf, String classname)
- {
- conf.set(INPUT_TRANSPORT_FACTORY_CLASS, classname);
- }
-
- public static ITransportFactory getOutputTransportFactory(Configuration conf)
- {
- return getTransportFactory(conf.get(OUTPUT_TRANSPORT_FACTORY_CLASS, TFramedTransportFactory.class.getName()));
- }
-
- public static void setOutputTransportFactoryClass(Configuration conf, String classname)
- {
- conf.set(OUTPUT_TRANSPORT_FACTORY_CLASS, classname);
- }
-
- private static ITransportFactory getTransportFactory(String factoryClassName) {
+ private static TClientTransportFactory getClientTransportFactory(String factoryClassName) {
try
{
- return (ITransportFactory) Class.forName(factoryClassName).newInstance();
+ return (TClientTransportFactory) Class.forName(factoryClassName).newInstance();
}
catch (Exception e)
{
throw new RuntimeException("Failed to instantiate transport factory:" + factoryClassName, e);
}
}
+ private static Map<String, String> getOptions(Configuration conf, Set<String> supportedOptions) {
+ Map<String, String> options = Maps.newHashMap();
+ for (String optionKey : supportedOptions)
+ {
+ String optionValue = conf.get(optionKey);
+ if (optionValue != null)
+ options.put(optionKey, optionValue);
+ }
+ return options;
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/Progressable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/Progressable.java b/src/java/org/apache/cassandra/hadoop/Progressable.java
index 091a828..ac253ef 100644
--- a/src/java/org/apache/cassandra/hadoop/Progressable.java
+++ b/src/java/org/apache/cassandra/hadoop/Progressable.java
@@ -29,12 +29,12 @@ public class Progressable
private TaskAttemptContext context;
private org.apache.hadoop.util.Progressable progressable;
- Progressable(TaskAttemptContext context)
+ public Progressable(TaskAttemptContext context)
{
this.context = context;
}
- Progressable(org.apache.hadoop.util.Progressable progressable)
+ public Progressable(org.apache.hadoop.util.Progressable progressable)
{
this.progressable = progressable;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java b/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java
new file mode 100644
index 0000000..66bcfdb
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java
@@ -0,0 +1,109 @@
+package org.apache.cassandra.hadoop.cql3;
+/*
+*
+* 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 org.apache.hadoop.conf.Configuration;
+
+public class CQLConfigHelper
+{
+ private static final String INPUT_CQL_COLUMNS_CONFIG = "cassandra.input.columnfamily.columns"; // separate by colon ,
+ private static final String INPUT_CQL_PAGE_ROW_SIZE_CONFIG = "cassandra.input.page.row.size";
+ private static final String INPUT_CQL_WHERE_CLAUSE_CONFIG = "cassandra.input.where.clause";
+ private static final String OUTPUT_CQL = "cassandra.output.cql";
+
+ /**
+ * Set the CQL columns for the input of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param columns
+ */
+ public static void setInputColumns(Configuration conf, String columns)
+ {
+ if (columns == null || columns.isEmpty())
+ return;
+
+ conf.set(INPUT_CQL_COLUMNS_CONFIG, columns);
+ }
+
+ /**
+ * Set the CQL query Limit for the input of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param cqlPageRowSize
+ */
+ public static void setInputCQLPageRowSize(Configuration conf, String cqlPageRowSize)
+ {
+ if (cqlPageRowSize == null)
+ {
+ throw new UnsupportedOperationException("cql page row size may not be null");
+ }
+
+ conf.set(INPUT_CQL_PAGE_ROW_SIZE_CONFIG, cqlPageRowSize);
+ }
+
+ /**
+ * Set the CQL user defined where clauses for the input of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param clauses
+ */
+ public static void setInputWhereClauses(Configuration conf, String clauses)
+ {
+ if (clauses == null || clauses.isEmpty())
+ return;
+
+ conf.set(INPUT_CQL_WHERE_CLAUSE_CONFIG, clauses);
+ }
+
+ /**
+ * Set the CQL prepared statement for the output of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param cql
+ */
+ public static void setOutputCql(Configuration conf, String cql)
+ {
+ if (cql == null || cql.isEmpty())
+ return;
+
+ conf.set(OUTPUT_CQL, cql);
+ }
+
+
+ public static String getInputcolumns(Configuration conf)
+ {
+ return conf.get(INPUT_CQL_COLUMNS_CONFIG);
+ }
+
+ public static String getInputPageRowSize(Configuration conf)
+ {
+ return conf.get(INPUT_CQL_PAGE_ROW_SIZE_CONFIG);
+ }
+
+ public static String getInputWhereClauses(Configuration conf)
+ {
+ return conf.get(INPUT_CQL_WHERE_CLAUSE_CONFIG);
+ }
+
+ public static String getOutputCql(Configuration conf)
+ {
+ return conf.get(OUTPUT_CQL);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java
new file mode 100644
index 0000000..525ed89
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import org.apache.cassandra.hadoop.AbstractColumnFamilyInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+
+/**
+ * Hadoop InputFormat allowing map/reduce against Cassandra rows within one ColumnFamily.
+ *
+ * At minimum, you need to set the KS and CF in your Hadoop job Configuration.
+ * The ConfigHelper class is provided to make this
+ * simple:
+ * ConfigHelper.setInputColumnFamily
+ *
+ * You can also configure the number of rows per InputSplit with
+ * ConfigHelper.setInputSplitSize. The default split size is 64k rows.
+ * the number of CQL rows per page
+ *
+ * the number of CQL rows per page
+ * CQLConfigHelper.setInputCQLPageRowSize. The default page row size is 1000. You
+ * should set it to "as big as possible, but no bigger." It set the LIMIT for the CQL
+ * query, so you need set it big enough to minimize the network overhead, and also
+ * not too big to avoid out of memory issue.
+ *
+ * the column names of the select CQL query. The default is all columns
+ * CQLConfigHelper.setInputColumns
+ *
+ * the user defined the where clause
+ * CQLConfigHelper.setInputWhereClauses. The default is no user defined where clause
+ */
+public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<Map<String, ByteBuffer>, Map<String, ByteBuffer>>
+{
+ public RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>> getRecordReader(InputSplit split, JobConf jobConf, final Reporter reporter)
+ throws IOException
+ {
+ TaskAttemptContext tac = new TaskAttemptContext(jobConf, TaskAttemptID.forName(jobConf.get(MAPRED_TASK_ID)))
+ {
+ @Override
+ public void progress()
+ {
+ reporter.progress();
+ }
+ };
+
+ ColumnFamilyRecordReader recordReader = new ColumnFamilyRecordReader();
+ recordReader.initialize((org.apache.hadoop.mapreduce.InputSplit)split, tac);
+ return recordReader;
+ }
+
+ @Override
+ public org.apache.hadoop.mapreduce.RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>> createRecordReader(
+ org.apache.hadoop.mapreduce.InputSplit arg0, TaskAttemptContext arg1) throws IOException,
+ InterruptedException
+ {
+ return new ColumnFamilyRecordReader();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java
new file mode 100644
index 0000000..3f6e2af
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.hadoop.AbstractColumnFamilyOutputFormat;
+import org.apache.cassandra.hadoop.ColumnFamilyInputFormat;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.Progressable;
+import org.apache.hadoop.mapreduce.*;
+
+/**
+ * The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
+ * OutputFormat that allows reduce tasks to store keys (and corresponding
+ * binded variable values) as CQL rows (and respective columns) in a given
+ * ColumnFamily.
+ *
+ * <p>
+ * As is the case with the {@link ColumnFamilyInputFormat}, you need to set the
+ * prepared statement in your
+ * Hadoop job Configuration. The {@link CQLConfigHelper} class, through its
+ * {@link ConfigHelper#setOutputPreparedStatement} method, is provided to make this
+ * simple.
+ * you need to set the Keyspace. The {@link ConfigHelper} class, through its
+ * {@link ConfigHelper#setOutputColumnFamily} method, is provided to make this
+ * simple.
+ * </p>
+ *
+ * <p>
+ * For the sake of performance, this class employs a lazy write-back caching
+ * mechanism, where its record writer prepared statement binded variable values
+ * created based on the reduce's inputs (in a task-specific map), and periodically
+ * makes the changes official by sending a execution of prepared statement request
+ * to Cassandra.
+ * </p>
+ */
+public class ColumnFamilyOutputFormat extends AbstractColumnFamilyOutputFormat<Map<String, ByteBuffer>, List<ByteBuffer>>
+{
+ /** Fills the deprecated OutputFormat interface for streaming. */
+ @Deprecated
+ public ColumnFamilyRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress) throws IOException
+ {
+ return new ColumnFamilyRecordWriter(job, new Progressable(progress));
+ }
+
+ /**
+ * Get the {@link RecordWriter} for the given task.
+ *
+ * @param context
+ * the information about the current task.
+ * @return a {@link RecordWriter} to write the output for the job.
+ * @throws IOException
+ */
+ public ColumnFamilyRecordWriter getRecordWriter(final TaskAttemptContext context) throws IOException, InterruptedException
+ {
+ return new ColumnFamilyRecordWriter(context);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java
new file mode 100644
index 0000000..03d7af5
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java
@@ -0,0 +1,763 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.util.*;
+
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.hadoop.ColumnFamilySplit;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+
+/**
+ * Hadoop RecordReader read the values return from the CQL query
+ * It use CQL key range query to page through the wide rows.
+ * <p/>
+ * Return List<IColumn> as keys columns
+ * <p/>
+ * Map<ByteBuffer, IColumn> as column name to columns mappings
+ */
+public class ColumnFamilyRecordReader extends RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>>
+ implements org.apache.hadoop.mapred.RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>>
+{
+ private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyRecordReader.class);
+
+ public static final int DEFAULT_CQL_PAGE_LIMIT = 1000; // TODO: find the number large enough but not OOM
+
+ private ColumnFamilySplit split;
+ private RowIterator rowIterator;
+
+ private Pair<Map<String, ByteBuffer>, Map<String, ByteBuffer>> currentRow;
+ private int totalRowCount; // total number of rows to fetch
+ private String keyspace;
+ private String cfName;
+ private Cassandra.Client client;
+ private ConsistencyLevel consistencyLevel;
+
+ // partition keys -- key aliases
+ private List<BoundColumn> partitionBoundColumns = new ArrayList<BoundColumn>();
+
+ // cluster keys -- column aliases
+ private List<BoundColumn> clusterColumns = new ArrayList<BoundColumn>();
+
+ // map prepared query type to item id
+ private Map<Integer, Integer> preparedQueryIds = new HashMap<Integer, Integer>();
+
+ // cql query select columns
+ private String columns;
+
+ // the number of cql rows per page
+ private int pageRowSize;
+
+ // user defined where clauses
+ private String userDefinedWhereClauses;
+
+ private IPartitioner partitioner;
+
+ private AbstractType<?> keyValidator;
+
+ public ColumnFamilyRecordReader()
+ {
+ super();
+ }
+
+ public void initialize(InputSplit split, TaskAttemptContext context) throws IOException
+ {
+ this.split = (ColumnFamilySplit) split;
+ Configuration conf = context.getConfiguration();
+ totalRowCount = (this.split.getLength() < Long.MAX_VALUE)
+ ? (int) this.split.getLength()
+ : ConfigHelper.getInputSplitSize(conf);
+ cfName = ConfigHelper.getInputColumnFamily(conf);
+ consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getReadConsistencyLevel(conf));
+ keyspace = ConfigHelper.getInputKeyspace(conf);
+ columns = CQLConfigHelper.getInputcolumns(conf);
+ userDefinedWhereClauses = CQLConfigHelper.getInputWhereClauses(conf);
+
+ try
+ {
+ pageRowSize = Integer.parseInt(CQLConfigHelper.getInputPageRowSize(conf));
+ }
+ catch (NumberFormatException e)
+ {
+ pageRowSize = DEFAULT_CQL_PAGE_LIMIT;
+ }
+
+ partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
+
+ try
+ {
+ if (client != null)
+ return;
+
+ // create connection using thrift
+ String location = getLocation();
+
+ int port = ConfigHelper.getInputRpcPort(conf);
+ client = ColumnFamilyInputFormat.createAuthenticatedClient(location, port, conf);
+
+ // retrieve partition keys and cluster keys from system.schema_columnfamilies table
+ retrieveKeys();
+
+ client.set_keyspace(keyspace);
+ }
+ catch (Exception e)
+ {
+ throw new RuntimeException(e);
+ }
+
+ rowIterator = new RowIterator();
+
+ logger.debug("created {}", rowIterator);
+ }
+
+ public void close()
+ {
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ client = null;
+ }
+ }
+
+ public Map<String, ByteBuffer> getCurrentKey()
+ {
+ return currentRow.left;
+ }
+
+ public Map<String, ByteBuffer> getCurrentValue()
+ {
+ return currentRow.right;
+ }
+
+ public float getProgress()
+ {
+ if (!rowIterator.hasNext())
+ return 1.0F;
+
+ // the progress is likely to be reported slightly off the actual but close enough
+ float progress = ((float) rowIterator.totalRead / totalRowCount);
+ return progress > 1.0F ? 1.0F : progress;
+ }
+
+ public boolean nextKeyValue() throws IOException
+ {
+ if (!rowIterator.hasNext())
+ {
+ logger.debug("Finished scanning " + rowIterator.totalRead + " rows (estimate was: " + totalRowCount + ")");
+ return false;
+ }
+
+ try
+ {
+ currentRow = rowIterator.next();
+ }
+ catch (Exception e)
+ {
+ // throw it as IOException, so client can catch it and handle it at client side
+ IOException ioe = new IOException(e.getMessage());
+ ioe.initCause(ioe.getCause());
+ throw ioe;
+ }
+ return true;
+ }
+
+ // we don't use endpointsnitch since we are trying to support hadoop nodes that are
+ // not necessarily on Cassandra machines, too. This should be adequate for single-DC clusters, at least.
+ private String getLocation()
+ {
+ Collection<InetAddress> localAddresses = FBUtilities.getAllLocalAddresses();
+
+ for (InetAddress address : localAddresses)
+ {
+ for (String location : split.getLocations())
+ {
+ InetAddress locationAddress;
+ try
+ {
+ locationAddress = InetAddress.getByName(location);
+ }
+ catch (UnknownHostException e)
+ {
+ throw new AssertionError(e);
+ }
+ if (address.equals(locationAddress))
+ {
+ return location;
+ }
+ }
+ }
+ return split.getLocations()[0];
+ }
+
+ // Because the old Hadoop API wants us to write to the key and value
+ // and the new asks for them, we need to copy the output of the new API
+ // to the old. Thus, expect a small performance hit.
+ // And obviously this wouldn't work for wide rows. But since ColumnFamilyInputFormat
+ // and ColumnFamilyRecordReader don't support them, it should be fine for now.
+ public boolean next(Map<String, ByteBuffer> keys, Map<String, ByteBuffer> value) throws IOException
+ {
+ if (nextKeyValue())
+ {
+ value.clear();
+ value.putAll(getCurrentValue());
+
+ keys.clear();
+ keys.putAll(getCurrentKey());
+
+ return true;
+ }
+ return false;
+ }
+
+ public long getPos() throws IOException
+ {
+ return (long) rowIterator.totalRead;
+ }
+
+ public Map<String, ByteBuffer> createKey()
+ {
+ return new LinkedHashMap<String, ByteBuffer>();
+ }
+
+ public Map<String, ByteBuffer> createValue()
+ {
+ return new LinkedHashMap<String, ByteBuffer>();
+ }
+
+ /** CQL row iterator */
+ private class RowIterator extends AbstractIterator<Pair<Map<String, ByteBuffer>, Map<String, ByteBuffer>>>
+ {
+ protected int totalRead = 0; // total number of cf rows read
+ protected Iterator<CqlRow> rows;
+ private int pageRows = 0; // the number of cql rows read of this page
+ private String previousRowKey = null; // previous CF row key
+ private String partitionKeyString; // keys in <key1>, <key2>, <key3> string format
+ private String partitionKeyMarkers; // question marks in ? , ? , ? format which matches the number of keys
+
+ public RowIterator()
+ {
+ // initial page
+ executeQuery();
+ }
+
+ protected Pair<Map<String, ByteBuffer>, Map<String, ByteBuffer>> computeNext()
+ {
+ if (rows == null)
+ return endOfData();
+
+ int index = -2;
+ //check there are more page to read
+ while (!rows.hasNext())
+ {
+ // no more data
+ if (index == -1 || emptyPartitionKeyValues())
+ {
+ logger.debug("no more data.");
+ return endOfData();
+ }
+
+ index = setTailNull(clusterColumns);
+ logger.debug("set tail to null, index: " + index);
+ executeQuery();
+ pageRows = 0;
+
+ if (rows == null || !rows.hasNext() && index < 0)
+ {
+ logger.debug("no more data.");
+ return endOfData();
+ }
+ }
+
+ Map<String, ByteBuffer> valueColumns = createValue();
+ Map<String, ByteBuffer> keyColumns = createKey();
+ int i = 0;
+ CqlRow row = rows.next();
+ for (Column column : row.columns)
+ {
+ String columnName = stringValue(ByteBuffer.wrap(column.getName()));
+ logger.debug("column: " + columnName);
+
+ if (i < partitionBoundColumns.size() + clusterColumns.size())
+ keyColumns.put(stringValue(column.name), column.value);
+ else
+ valueColumns.put(stringValue(column.name), column.value);
+
+ i++;
+ }
+
+ // increase total CQL row read for this page
+ pageRows++;
+
+ // increase total CF row read
+ if (newRow(keyColumns, previousRowKey))
+ totalRead++;
+
+ // read full page
+ if (pageRows >= pageRowSize || !rows.hasNext())
+ {
+ Iterator<String> newKeys = keyColumns.keySet().iterator();
+ for (BoundColumn column : partitionBoundColumns)
+ column.value = keyColumns.get(newKeys.next());
+
+ for (BoundColumn column : clusterColumns)
+ column.value = keyColumns.get(newKeys.next());
+
+ executeQuery();
+ pageRows = 0;
+ }
+
+ return Pair.create(keyColumns, valueColumns);
+ }
+
+ /** check whether start to read a new CF row by comparing the partition keys */
+ private boolean newRow(Map<String, ByteBuffer> keyColumns, String previousRowKey)
+ {
+ if (keyColumns.isEmpty())
+ return false;
+
+ String rowKey = "";
+ if (keyColumns.size() == 1)
+ {
+ rowKey = partitionBoundColumns.get(0).validator.getString(keyColumns.get(partitionBoundColumns.get(0).name));
+ }
+ else
+ {
+ Iterator<ByteBuffer> iter = keyColumns.values().iterator();
+ for (BoundColumn column : partitionBoundColumns)
+ rowKey = rowKey + column.validator.getString(ByteBufferUtil.clone(iter.next())) + ":";
+ }
+
+ logger.debug("previous RowKey: " + previousRowKey + ", new row key: " + rowKey);
+ if (previousRowKey == null)
+ {
+ this.previousRowKey = rowKey;
+ return true;
+ }
+
+ if (rowKey.equals(previousRowKey))
+ return false;
+
+ this.previousRowKey = rowKey;
+ return true;
+ }
+
+ /** set the last non-null key value to null, and return the previous index */
+ private int setTailNull(List<BoundColumn> values)
+ {
+ if (values.isEmpty())
+ return -1;
+
+ Iterator<BoundColumn> iterator = values.iterator();
+ int previousIndex = -1;
+ BoundColumn current;
+ while (iterator.hasNext())
+ {
+ current = iterator.next();
+ if (current.value == null)
+ {
+ int index = previousIndex > 0 ? previousIndex : 0;
+ BoundColumn column = values.get(index);
+ logger.debug("set key " + column.name + " value to null");
+ column.value = null;
+ return previousIndex - 1;
+ }
+
+ previousIndex++;
+ }
+
+ BoundColumn column = values.get(previousIndex);
+ logger.debug("set key " + column.name + " value to null");
+ column.value = null;
+ return previousIndex - 1;
+ }
+
+ /** compose the prepared query, pair.left is query id, pair.right is query */
+ private Pair<Integer, String> composeQuery(String columns)
+ {
+ Pair<Integer, String> clause = whereClause();
+ if (columns == null)
+ {
+ columns = "*";
+ }
+ else
+ {
+ // add keys in the front in order
+ String partitionKey = keyString(partitionBoundColumns);
+ String clusterKey = keyString(clusterColumns);
+
+ columns = withoutKeyColumns(columns);
+ columns = (clusterKey == null || "".equals(clusterKey))
+ ? partitionKey + "," + columns
+ : partitionKey + "," + clusterKey + "," + columns;
+ }
+
+ return Pair.create(clause.left,
+ "SELECT " + columns
+ + " FROM " + cfName
+ + clause.right
+ + (userDefinedWhereClauses == null ? "" : " AND " + userDefinedWhereClauses)
+ + " LIMIT " + pageRowSize
+ + " ALLOW FILTERING");
+ }
+
+
+ /** remove key columns from the column string */
+ private String withoutKeyColumns(String columnString)
+ {
+ Set<String> keyNames = new HashSet<String>();
+ for (BoundColumn column : Iterables.concat(partitionBoundColumns, clusterColumns))
+ keyNames.add(column.name);
+
+ String[] columns = columnString.split(",");
+ String result = null;
+ for (String column : columns)
+ {
+ String trimmed = column.trim();
+ if (keyNames.contains(trimmed))
+ continue;
+
+ result = result == null ? trimmed : result + "," + trimmed;
+ }
+ return result;
+ }
+
+ /** compose the where clause */
+ private Pair<Integer, String> whereClause()
+ {
+ if (partitionKeyString == null)
+ partitionKeyString = keyString(partitionBoundColumns);
+
+ if (partitionKeyMarkers == null)
+ partitionKeyMarkers = partitionKeyMarkers();
+ // initial query token(k) >= start_token and token(k) <= end_token
+ if (emptyPartitionKeyValues())
+ return Pair.create(0, " WHERE token(" + partitionKeyString + ") > ? AND token(" + partitionKeyString + ") <= ?");
+
+ // query token(k) > token(pre_partition_key) and token(k) <= end_token
+ if (clusterColumns.size() == 0 || clusterColumns.get(0).value == null)
+ return Pair.create(1,
+ " WHERE token(" + partitionKeyString + ") > token(" + partitionKeyMarkers + ") "
+ + " AND token(" + partitionKeyString + ") <= ?");
+
+ // query token(k) = token(pre_partition_key) and m = pre_cluster_key_m and n > pre_cluster_key_n
+ Pair<Integer, String> clause = whereClause(clusterColumns, 0);
+ return Pair.create(clause.left,
+ " WHERE token(" + partitionKeyString + ") = token(" + partitionKeyMarkers + ") " + clause.right);
+ }
+
+ /** recursively compose the where clause */
+ private Pair<Integer, String> whereClause(List<BoundColumn> column, int position)
+ {
+ if (position == column.size() - 1 || column.get(position + 1).value == null)
+ return Pair.create(position + 2, " AND " + column.get(position).name + " > ? ");
+
+ Pair<Integer, String> clause = whereClause(column, position + 1);
+ return Pair.create(clause.left, " AND " + column.get(position).name + " = ? " + clause.right);
+ }
+
+ /** check whether all key values are null */
+ private boolean emptyPartitionKeyValues()
+ {
+ for (BoundColumn column : partitionBoundColumns)
+ {
+ if (column.value != null)
+ return false;
+ }
+ return true;
+ }
+
+ /** compose the partition key string in format of <key1>, <key2>, <key3> */
+ private String keyString(List<BoundColumn> columns)
+ {
+ String result = null;
+ for (BoundColumn column : columns)
+ result = result == null ? column.name : result + "," + column.name;
+
+ return result == null ? "" : result;
+ }
+
+ /** compose the question marks for partition key string in format of ?, ? , ? */
+ private String partitionKeyMarkers()
+ {
+ String result = null;
+ for (BoundColumn column : partitionBoundColumns)
+ result = result == null ? "?" : result + ",?";
+
+ return result;
+ }
+
+ /** compose the query binding variables, pair.left is query id, pair.right is the binding variables */
+ private Pair<Integer, List<ByteBuffer>> preparedQueryBindValues()
+ {
+ List<ByteBuffer> values = new LinkedList<ByteBuffer>();
+
+ // initial query token(k) >= start_token and token(k) <= end_token
+ if (emptyPartitionKeyValues())
+ {
+ values.add(partitioner.getTokenValidator().fromString(split.getStartToken()));
+ values.add(partitioner.getTokenValidator().fromString(split.getEndToken()));
+ return Pair.create(0, values);
+ }
+ else
+ {
+ for (BoundColumn partitionBoundColumn1 : partitionBoundColumns)
+ values.add(partitionBoundColumn1.value);
+
+ if (clusterColumns.size() == 0 || clusterColumns.get(0).value == null)
+ {
+ // query token(k) > token(pre_partition_key) and token(k) <= end_token
+ values.add(partitioner.getTokenValidator().fromString(split.getEndToken()));
+ return Pair.create(1, values);
+ }
+ else
+ {
+ // query token(k) = token(pre_partition_key) and m = pre_cluster_key_m and n > pre_cluster_key_n
+ int type = preparedQueryBindValues(clusterColumns, 0, values);
+ return Pair.create(type, values);
+ }
+ }
+ }
+
+ /** recursively compose the query binding variables */
+ private int preparedQueryBindValues(List<BoundColumn> column, int position, List<ByteBuffer> bindValues)
+ {
+ if (position == column.size() - 1 || column.get(position + 1).value == null)
+ {
+ bindValues.add(column.get(position).value);
+ return position + 2;
+ }
+ else
+ {
+ bindValues.add(column.get(position).value);
+ return preparedQueryBindValues(column, position + 1, bindValues);
+ }
+ }
+
+ /** get the prepared query item Id */
+ private int prepareQuery(int type) throws InvalidRequestException, TException
+ {
+ Integer itemId = preparedQueryIds.get(type);
+ if (itemId != null)
+ return itemId;
+
+ Pair<Integer, String> query = null;
+ query = composeQuery(columns);
+ logger.debug("type:" + query.left + ", query: " + query.right);
+ CqlPreparedResult cqlPreparedResult = client.prepare_cql3_query(ByteBufferUtil.bytes(query.right), Compression.NONE);
+ preparedQueryIds.put(query.left, cqlPreparedResult.itemId);
+ return cqlPreparedResult.itemId;
+ }
+
+ /** execute the prepared query */
+ private void executeQuery()
+ {
+ Pair<Integer, List<ByteBuffer>> bindValues = preparedQueryBindValues();
+ logger.debug("query type: " + bindValues.left);
+
+ // check whether it reach end of range for type 1 query CASSANDRA-5573
+ if (bindValues.left == 1 && reachEndRange())
+ {
+ rows = null;
+ return;
+ }
+
+ int retries = 0;
+ // only try three times for TimedOutException and UnavailableException
+ while (retries < 3)
+ {
+ try
+ {
+ CqlResult cqlResult = client.execute_prepared_cql3_query(prepareQuery(bindValues.left), bindValues.right, consistencyLevel);
+ if (cqlResult != null && cqlResult.rows != null)
+ rows = cqlResult.rows.iterator();
+ return;
+ }
+ catch (TimedOutException e)
+ {
+ retries++;
+ if (retries >= 3)
+ {
+ rows = null;
+ RuntimeException rte = new RuntimeException(e.getMessage());
+ rte.initCause(e);
+ throw rte;
+ }
+ }
+ catch (UnavailableException e)
+ {
+ retries++;
+ if (retries >= 3)
+ {
+ rows = null;
+ RuntimeException rte = new RuntimeException(e.getMessage());
+ rte.initCause(e);
+ throw rte;
+ }
+ }
+ catch (Exception e)
+ {
+ rows = null;
+ RuntimeException rte = new RuntimeException(e.getMessage());
+ rte.initCause(e);
+ throw rte;
+ }
+ }
+ }
+ }
+
+ /** retrieve the partition keys and cluster keys from system.schema_columnfamilies table */
+ private void retrieveKeys() throws Exception
+ {
+ String query = "select key_aliases," +
+ "column_aliases, " +
+ "key_validator, " +
+ "comparator " +
+ "from system.schema_columnfamilies " +
+ "where keyspace_name='%s' and columnfamily_name='%s'";
+ String formatted = String.format(query, keyspace, cfName);
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(formatted), Compression.NONE, ConsistencyLevel.ONE);
+
+ CqlRow cqlRow = result.rows.get(0);
+ String keyString = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(0).getValue()));
+ logger.debug("partition keys: " + keyString);
+ List<String> keys = FBUtilities.fromJsonList(keyString);
+
+ for (String key : keys)
+ partitionBoundColumns.add(new BoundColumn(key));
+
+ keyString = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(1).getValue()));
+ logger.debug("cluster columns: " + keyString);
+ keys = FBUtilities.fromJsonList(keyString);
+
+ for (String key : keys)
+ clusterColumns.add(new BoundColumn(key));
+
+ Column rawKeyValidator = cqlRow.columns.get(2);
+ String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));
+ logger.debug("row key validator: " + validator);
+ keyValidator = parseType(validator);
+
+ if (keyValidator instanceof CompositeType)
+ {
+ List<AbstractType<?>> types = ((CompositeType) keyValidator).types;
+ for (int i = 0; i < partitionBoundColumns.size(); i++)
+ partitionBoundColumns.get(i).validator = types.get(i);
+ }
+ else
+ {
+ partitionBoundColumns.get(0).validator = keyValidator;
+ }
+ }
+
+ /** check whether current row is at the end of range */
+ private boolean reachEndRange()
+ {
+ // current row key
+ ByteBuffer rowKey;
+ if (keyValidator instanceof CompositeType)
+ {
+ ByteBuffer[] keys = new ByteBuffer[partitionBoundColumns.size()];
+ for (int i = 0; i < partitionBoundColumns.size(); i++)
+ keys[i] = partitionBoundColumns.get(i).value.duplicate();
+
+ rowKey = ((CompositeType) keyValidator).build(keys);
+ }
+ else
+ {
+ rowKey = partitionBoundColumns.get(0).value;
+ }
+
+ String endToken = split.getEndToken();
+ String currentToken = partitioner.getToken(rowKey).toString();
+ logger.debug("End token: " + endToken + ", current token: " + currentToken);
+
+ return endToken.equals(currentToken);
+ }
+
+ private static AbstractType<?> parseType(String type) throws IOException
+ {
+ try
+ {
+ // always treat counters like longs, specifically CCT.compose is not what we need
+ if (type != null && type.equals("org.apache.cassandra.db.marshal.CounterColumnType"))
+ return LongType.instance;
+ return TypeParser.parse(type);
+ }
+ catch (ConfigurationException e)
+ {
+ throw new IOException(e);
+ }
+ catch (SyntaxException e)
+ {
+ throw new IOException(e);
+ }
+ }
+
+ private class BoundColumn
+ {
+ final String name;
+ ByteBuffer value;
+ AbstractType<?> validator;
+
+ public BoundColumn(String name)
+ {
+ this.name = name;
+ }
+ }
+
+ /** get string from a ByteBuffer, catch the exception and throw it as runtime exception*/
+ private static String stringValue(ByteBuffer value)
+ {
+ try
+ {
+ return ByteBufferUtil.string(value);
+ }
+ catch (CharacterCodingException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java
new file mode 100644
index 0000000..3939e0b
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordWriter.java
@@ -0,0 +1,386 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.hadoop.AbstractColumnFamilyRecordWriter;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.Progressable;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The <code>ColumnFamilyRecordWriter</code> maps the output <key, value>
+ * pairs to a Cassandra column family. In particular, it applies the binded variables
+ * in the value to the prepared statement, which it associates with the key, and in
+ * turn the responsible endpoint.
+ *
+ * <p>
+ * Furthermore, this writer groups the cql queries by the endpoint responsible for
+ * the rows being affected. This allows the cql queries to be executed in parallel,
+ * directly to a responsible endpoint.
+ * </p>
+ *
+ * @see ColumnFamilyOutputFormat
+ */
+final class ColumnFamilyRecordWriter extends AbstractColumnFamilyRecordWriter<Map<String, ByteBuffer>, List<ByteBuffer>>
+{
+ private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyRecordWriter.class);
+
+ // handles for clients for each range running in the threadpool
+ private final Map<Range, RangeClient> clients;
+
+ // host to prepared statement id mappings
+ private ConcurrentHashMap<Cassandra.Client, Integer> preparedStatements = new ConcurrentHashMap<Cassandra.Client, Integer>();
+
+ private final String cql;
+
+ private AbstractType<?> keyValidator;
+ private String [] partitionkeys;
+
+ /**
+ * Upon construction, obtain the map that this writer will use to collect
+ * mutations, and the ring cache for the given keyspace.
+ *
+ * @param context the task attempt context
+ * @throws IOException
+ */
+ ColumnFamilyRecordWriter(TaskAttemptContext context) throws IOException
+ {
+ this(context.getConfiguration());
+ this.progressable = new Progressable(context);
+ }
+
+ ColumnFamilyRecordWriter(Configuration conf, Progressable progressable) throws IOException
+ {
+ this(conf);
+ this.progressable = progressable;
+ }
+
+ ColumnFamilyRecordWriter(Configuration conf) throws IOException
+ {
+ super(conf);
+ this.clients = new HashMap<Range, RangeClient>();
+ cql = CQLConfigHelper.getOutputCql(conf);
+
+ try
+ {
+ String host = getAnyHost();
+ int port = ConfigHelper.getOutputRpcPort(conf);
+ Cassandra.Client client = ColumnFamilyOutputFormat.createAuthenticatedClient(host, port, conf);
+ retrievePartitionKeyValidator(client);
+
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ client = null;
+ }
+ }
+ catch (Exception e)
+ {
+ throw new IOException(e);
+ }
+ }
+
+ @Override
+ public void close() throws IOException
+ {
+ // close all the clients before throwing anything
+ IOException clientException = null;
+ for (RangeClient client : clients.values())
+ {
+ try
+ {
+ client.close();
+ }
+ catch (IOException e)
+ {
+ clientException = e;
+ }
+ }
+
+ if (clientException != null)
+ throw clientException;
+ }
+
+ /**
+ * If the key is to be associated with a valid value, a mutation is created
+ * for it with the given column family and columns. In the event the value
+ * in the column is missing (i.e., null), then it is marked for
+ * {@link Deletion}. Similarly, if the entire value for a key is missing
+ * (i.e., null), then the entire key is marked for {@link Deletion}.
+ * </p>
+ *
+ * @param keybuff
+ * the key to write.
+ * @param values
+ * the values to write.
+ * @throws IOException
+ */
+ @Override
+ public void write(Map<String, ByteBuffer> keys, List<ByteBuffer> values) throws IOException
+ {
+ ByteBuffer rowKey = getRowKey(keys);
+ Range<Token> range = ringCache.getRange(rowKey);
+
+ // get the client for the given range, or create a new one
+ RangeClient client = clients.get(range);
+ if (client == null)
+ {
+ // haven't seen keys for this range: create new client
+ client = new RangeClient(ringCache.getEndpoint(range));
+ client.start();
+ clients.put(range, client);
+ }
+
+ client.put(Pair.create(rowKey, values));
+ progressable.progress();
+ }
+
+ /**
+ * A client that runs in a threadpool and connects to the list of endpoints for a particular
+ * range. Binded variable values for keys in that range are sent to this client via a queue.
+ */
+ public class RangeClient extends AbstractRangeClient<List<ByteBuffer>>
+ {
+ /**
+ * Constructs an {@link RangeClient} for the given endpoints.
+ * @param endpoints the possible endpoints to execute the mutations on
+ */
+ public RangeClient(List<InetAddress> endpoints)
+ {
+ super(endpoints);
+ }
+
+ /**
+ * Loops collecting cql binded variable values from the queue and sending to Cassandra
+ */
+ public void run()
+ {
+ outer:
+ while (run || !queue.isEmpty())
+ {
+ Pair<ByteBuffer, List<ByteBuffer>> bindVariables;
+ try
+ {
+ bindVariables = queue.take();
+ }
+ catch (InterruptedException e)
+ {
+ // re-check loop condition after interrupt
+ continue;
+ }
+
+ Iterator<InetAddress> iter = endpoints.iterator();
+ while (true)
+ {
+ // send the mutation to the last-used endpoint. first time through, this will NPE harmlessly.
+ try
+ {
+ int i = 0;
+ int itemId = preparedStatement(client);
+ while (bindVariables != null)
+ {
+ client.execute_prepared_cql3_query(itemId, bindVariables.right, ConsistencyLevel.ONE);
+ i++;
+
+ if (i >= batchThreshold)
+ break;
+
+ bindVariables = queue.poll();
+ }
+
+ break;
+ }
+ catch (Exception e)
+ {
+ closeInternal();
+ if (!iter.hasNext())
+ {
+ lastException = new IOException(e);
+ break outer;
+ }
+ }
+
+ // attempt to connect to a different endpoint
+ try
+ {
+ InetAddress address = iter.next();
+ String host = address.getHostName();
+ int port = ConfigHelper.getOutputRpcPort(conf);
+ client = ColumnFamilyOutputFormat.createAuthenticatedClient(host, port, conf);
+ }
+ catch (Exception e)
+ {
+ closeInternal();
+ // TException means something unexpected went wrong to that endpoint, so
+ // we should try again to another. Other exceptions (auth or invalid request) are fatal.
+ if ((!(e instanceof TException)) || !iter.hasNext())
+ {
+ lastException = new IOException(e);
+ break outer;
+ }
+ }
+ }
+ }
+ }
+
+ /** get prepared statement id from cache, otherwise prepare it from Cassandra server*/
+ private int preparedStatement(Cassandra.Client client)
+ {
+ Integer itemId = preparedStatements.get(client);
+ if (itemId == null)
+ {
+ CqlPreparedResult result;
+ try
+ {
+ result = client.prepare_cql3_query(ByteBufferUtil.bytes(cql), Compression.NONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ throw new RuntimeException("failed to prepare cql query " + cql, e);
+ }
+ catch (TException e)
+ {
+ throw new RuntimeException("failed to prepare cql query " + cql, e);
+ }
+
+ Integer previousId = preparedStatements.putIfAbsent(client, Integer.valueOf(result.itemId));
+ itemId = previousId == null ? result.itemId : previousId;
+ }
+ return itemId;
+ }
+ }
+
+ private ByteBuffer getRowKey(Map<String, ByteBuffer> keysMap)
+ {
+ //current row key
+ ByteBuffer rowKey;
+ if (keyValidator instanceof CompositeType)
+ {
+ ByteBuffer[] keys = new ByteBuffer[partitionkeys.length];
+ for (int i = 0; i< keys.length; i++)
+ keys[i] = keysMap.get(partitionkeys[i]);
+
+ rowKey = ((CompositeType) keyValidator).build(keys);
+ }
+ else
+ {
+ rowKey = keysMap.get(partitionkeys[0]);
+ }
+ return rowKey;
+ }
+
+ /** retrieve the key validator from system.schema_columnfamilies table */
+ private void retrievePartitionKeyValidator(Cassandra.Client client) throws Exception
+ {
+ String keyspace = ConfigHelper.getOutputKeyspace(conf);
+ String cfName = ConfigHelper.getOutputColumnFamily(conf);
+ String query = "SELECT key_validator," +
+ " key_aliases " +
+ "FROM system.schema_columnfamilies " +
+ "WHERE keyspace_name='%s' and columnfamily_name='%s'";
+ String formatted = String.format(query, keyspace, cfName);
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(formatted), Compression.NONE, ConsistencyLevel.ONE);
+
+ Column rawKeyValidator = result.rows.get(0).columns.get(0);
+ String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));
+ keyValidator = parseType(validator);
+
+ Column rawPartitionKeys = result.rows.get(0).columns.get(1);
+ String keyString = ByteBufferUtil.string(ByteBuffer.wrap(rawPartitionKeys.getValue()));
+ logger.debug("partition keys: " + keyString);
+
+ List<String> keys = FBUtilities.fromJsonList(keyString);
+ partitionkeys = new String [keys.size()];
+ int i=0;
+ for (String key: keys)
+ {
+ partitionkeys[i] = key;
+ i++;
+ }
+ }
+
+ private AbstractType<?> parseType(String type) throws IOException
+ {
+ try
+ {
+ // always treat counters like longs, specifically CCT.compose is not what we need
+ if (type != null && type.equals("org.apache.cassandra.db.marshal.CounterColumnType"))
+ return LongType.instance;
+ return TypeParser.parse(type);
+ }
+ catch (ConfigurationException e)
+ {
+ throw new IOException(e);
+ }
+ catch (SyntaxException e)
+ {
+ throw new IOException(e);
+ }
+ }
+
+ private String getAnyHost() throws IOException, InvalidRequestException, TException
+ {
+ Cassandra.Client client = ConfigHelper.getClientFromOutputAddressList(conf);
+ List<TokenRange> ring = client.describe_ring(ConfigHelper.getOutputKeyspace(conf));
+ try
+ {
+ for (TokenRange range : ring)
+ return range.endpoints.get(0);
+ }
+ finally
+ {
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ client = null;
+ }
+ }
+ throw new IOException("There are no endpoints");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90052d5a/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java b/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java
new file mode 100644
index 0000000..0a73043
--- /dev/null
+++ b/src/java/org/apache/cassandra/thrift/TClientTransportFactory.java
@@ -0,0 +1,70 @@
+package org.apache.cassandra.thrift;
+
+/*
+ *
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.thrift.transport.TTransport;
+
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ * Transport factory for establishing thrift connections from clients to a remote server.
+ */
+public interface TClientTransportFactory
+{
+ static final String PROPERTY_KEY = "cassandra.client.transport.factory";
+ static final String LONG_OPTION = "transport-factory";
+ static final String SHORT_OPTION = "tr";
+
+ /**
+ * Opens a client transport to a thrift server.
+ * Example:
+ *
+ * <pre>
+ * TTransport transport = clientTransportFactory.openTransport(address, port);
+ * Cassandra.Iface client = new Cassandra.Client(new BinaryProtocol(transport));
+ * </pre>
+ *
+ * @param host fully qualified hostname of the server
+ * @param port RPC port of the server
+ * @param conf Hadoop configuration
+ * @return open and ready to use transport
+ * @throws Exception implementation defined; usually throws TTransportException or IOException
+ * if the connection cannot be established
+ */
+ TTransport openTransport(String host, int port, Configuration conf) throws Exception;
+
+ /**
+ * Sets an implementation defined set of options.
+ * Keys in this map must conform to the set set returned by TClientTransportFactory#supportedOptions.
+ * @param options option map
+ */
+ void setOptions(Map<String, String> options);
+
+ /**
+ * @return set of options supported by this transport factory implementation
+ */
+ Set<String> supportedOptions();
+}
+
[06/10] git commit: Add CQL3 input/output formats patch by Alex Liu;
reviewed by jbellis and Mike Schrag for CASSANDRA-4421
Posted by jb...@apache.org.
Add CQL3 input/output formats
patch by Alex Liu; reviewed by jbellis and Mike Schrag for CASSANDRA-4421
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/56e0ad1b
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/56e0ad1b
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/56e0ad1b
Branch: refs/heads/trunk
Commit: 56e0ad1be2c3d7a2eb56b361e021668b8f22c095
Parents: 6850cdc
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Jun 6 14:09:51 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Jun 6 14:18:23 2013 -0500
----------------------------------------------------------------------
CHANGES.txt | 1 +
examples/hadoop_cql3_word_count/README.txt | 47 +
examples/hadoop_cql3_word_count/bin/word_count | 61 ++
.../hadoop_cql3_word_count/bin/word_count_counters | 58 ++
.../hadoop_cql3_word_count/bin/word_count_setup | 61 ++
examples/hadoop_cql3_word_count/build.xml | 113 +++
.../hadoop_cql3_word_count/conf/log4j.properties | 32 +
examples/hadoop_cql3_word_count/ivy.xml | 24 +
examples/hadoop_cql3_word_count/src/WordCount.java | 236 +++++
.../src/WordCountCounters.java | 122 +++
.../hadoop_cql3_word_count/src/WordCountSetup.java | 214 ++++
.../org/apache/cassandra/client/RingCache.java | 2 +-
.../hadoop/AbstractColumnFamilyInputFormat.java | 346 +++++++
.../hadoop/AbstractColumnFamilyOutputFormat.java | 159 +++
.../hadoop/AbstractColumnFamilyRecordWriter.java | 193 ++++
.../cassandra/hadoop/ColumnFamilyInputFormat.java | 307 +------
.../cassandra/hadoop/ColumnFamilyOutputFormat.java | 125 +---
.../cassandra/hadoop/ColumnFamilyRecordReader.java | 43 +-
.../cassandra/hadoop/ColumnFamilyRecordWriter.java | 195 +---
.../apache/cassandra/hadoop/ColumnFamilySplit.java | 4 +-
.../org/apache/cassandra/hadoop/ConfigHelper.java | 68 +-
.../org/apache/cassandra/hadoop/Progressable.java | 4 +-
.../cassandra/hadoop/cql3/CQLConfigHelper.java | 109 ++
.../hadoop/cql3/ColumnFamilyInputFormat.java | 83 ++
.../hadoop/cql3/ColumnFamilyOutputFormat.java | 78 ++
.../hadoop/cql3/ColumnFamilyRecordReader.java | 763 +++++++++++++++
.../hadoop/cql3/ColumnFamilyRecordWriter.java | 386 ++++++++
.../cassandra/thrift/TClientTransportFactory.java | 70 ++
.../cassandra/thrift/TFramedTransportFactory.java | 24 +-
29 files changed, 3280 insertions(+), 648 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 75b3173..36493d6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,6 +1,7 @@
1.2.6
* Reduce SSTableLoader memory usage (CASSANDRA-5555)
* Scale hinted_handoff_throttle_in_kb to cluster size (CASSANDRA-5272)
+ * (Hadoop) Add CQL3 input/output formats (CASSANDRA-4421)
* (Hadoop) Fix InputKeyRange in CFIF (CASSANDRA-5536)
* Fix dealing with ridiculously large max sstable sizes in LCS (CASSANDRA-5589)
* Ignore pre-truncate hints (CASSANDRA-4655)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/README.txt
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/README.txt b/examples/hadoop_cql3_word_count/README.txt
new file mode 100644
index 0000000..f984b02
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/README.txt
@@ -0,0 +1,47 @@
+Introduction
+============
+
+WordCount hadoop example: Inserts a bunch of words across multiple rows,
+and counts them, with RandomPartitioner. The word_count_counters example sums
+the value of counter columns for a key.
+
+The scripts in bin/ assume you are running with cwd of contrib/word_count.
+
+
+Running
+=======
+
+First build and start a Cassandra server with the default configuration*,
+then run
+
+contrib/word_count$ ant
+contrib/word_count$ bin/word_count_setup
+contrib/word_count$ bin/word_count
+contrib/word_count$ bin/word_count_counters
+
+In order to view the results in Cassandra, one can use bin/cassandra-cli and
+perform the following operations:
+$ bin/cqlsh localhost
+> use cql3_worldcount;
+> select * from output_words;
+
+The output of the word count can now be configured. In the bin/word_count
+file, you can specify the OUTPUT_REDUCER. The two options are 'filesystem'
+and 'cassandra'. The filesystem option outputs to the /tmp/word_count*
+directories. The cassandra option outputs to the 'output_words' column family
+in the 'cql3_worldcount' keyspace. 'cassandra' is the default.
+
+Read the code in src/ for more details.
+
+The word_count_counters example sums the counter columns for a row. The output
+is written to a text file in /tmp/word_count_counters.
+
+*If you want to point wordcount at a real cluster, modify the seed
+and listenaddress settings accordingly.
+
+
+Troubleshooting
+===============
+
+word_count uses conf/log4j.properties to log to wc.out.
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/bin/word_count
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/bin/word_count b/examples/hadoop_cql3_word_count/bin/word_count
new file mode 100644
index 0000000..a0c5aa0
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/bin/word_count
@@ -0,0 +1,61 @@
+#!/bin/sh
+
+# 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.
+
+cwd=`dirname $0`
+
+# Cassandra class files.
+if [ ! -d $cwd/../../../build/classes/main ]; then
+ echo "Unable to locate cassandra class files" >&2
+ exit 1
+fi
+
+# word_count Jar.
+if [ ! -e $cwd/../build/word_count.jar ]; then
+ echo "Unable to locate word_count jar" >&2
+ exit 1
+fi
+
+CLASSPATH=$CLASSPATH:$cwd/../conf
+CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/main
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/thrift
+for jar in $cwd/../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../lib/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+
+if [ -x $JAVA_HOME/bin/java ]; then
+ JAVA=$JAVA_HOME/bin/java
+else
+ JAVA=`which java`
+fi
+
+if [ "x$JAVA" = "x" ]; then
+ echo "Java executable not found (hint: set JAVA_HOME)" >&2
+ exit 1
+fi
+
+OUTPUT_REDUCER=cassandra
+
+#echo $CLASSPATH
+$JAVA -Xmx1G -ea -cp $CLASSPATH WordCount output_reducer=$OUTPUT_REDUCER
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/bin/word_count_counters
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/bin/word_count_counters b/examples/hadoop_cql3_word_count/bin/word_count_counters
new file mode 100644
index 0000000..7793477
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/bin/word_count_counters
@@ -0,0 +1,58 @@
+#!/bin/sh
+
+# 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.
+
+cwd=`dirname $0`
+
+# Cassandra class files.
+if [ ! -d $cwd/../../../build/classes/main ]; then
+ echo "Unable to locate cassandra class files" >&2
+ exit 1
+fi
+
+# word_count Jar.
+if [ ! -e $cwd/../build/word_count.jar ]; then
+ echo "Unable to locate word_count jar" >&2
+ exit 1
+fi
+
+CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/main
+CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/thrift
+for jar in $cwd/../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../lib/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+
+if [ -x $JAVA_HOME/bin/java ]; then
+ JAVA=$JAVA_HOME/bin/java
+else
+ JAVA=`which java`
+fi
+
+if [ "x$JAVA" = "x" ]; then
+ echo "Java executable not found (hint: set JAVA_HOME)" >&2
+ exit 1
+fi
+
+#echo $CLASSPATH
+$JAVA -Xmx1G -ea -cp $CLASSPATH WordCountCounters
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/bin/word_count_setup
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/bin/word_count_setup b/examples/hadoop_cql3_word_count/bin/word_count_setup
new file mode 100644
index 0000000..d194a45
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/bin/word_count_setup
@@ -0,0 +1,61 @@
+#!/bin/sh
+
+# 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.
+
+cwd=`dirname $0`
+
+# Cassandra class files.
+if [ ! -d $cwd/../../../build/classes/main ]; then
+ echo "Unable to locate cassandra class files" >&2
+ exit 1
+fi
+
+# word_count Jar.
+if [ ! -e $cwd/../build/word_count.jar ]; then
+ echo "Unable to locate word_count jar" >&2
+ exit 1
+fi
+
+CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
+CLASSPATH=$CLASSPATH:.:$cwd/../../../build/classes/main
+CLASSPATH=$CLASSPATH:.:$cwd/../../../build/classes/thrift
+for jar in $cwd/../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../lib/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+for jar in $cwd/../../../build/lib/jars/*.jar; do
+ CLASSPATH=$CLASSPATH:$jar
+done
+
+if [ -x $JAVA_HOME/bin/java ]; then
+ JAVA=$JAVA_HOME/bin/java
+else
+ JAVA=`which java`
+fi
+
+if [ "x$JAVA" = "x" ]; then
+ echo "Java executable not found (hint: set JAVA_HOME)" >&2
+ exit 1
+fi
+
+HOST=localhost
+PORT=9160
+FRAMED=true
+
+$JAVA -Xmx1G -ea -Dcassandra.host=$HOST -Dcassandra.port=$PORT -Dcassandra.framed=$FRAMED -cp $CLASSPATH WordCountSetup
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/build.xml
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/build.xml b/examples/hadoop_cql3_word_count/build.xml
new file mode 100644
index 0000000..939e1b3
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/build.xml
@@ -0,0 +1,113 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+<project default="jar" name="word_count" xmlns:ivy="antlib:org.apache.ivy.ant">
+ <property name="cassandra.dir" value="../.." />
+ <property name="cassandra.dir.lib" value="${cassandra.dir}/lib" />
+ <property name="cassandra.classes" value="${cassandra.dir}/build/classes" />
+ <property name="build.src" value="${basedir}/src" />
+ <property name="build.dir" value="${basedir}/build" />
+ <property name="ivy.lib.dir" value="${build.dir}/lib" />
+ <property name="build.classes" value="${build.dir}/classes" />
+ <property name="final.name" value="word_count" />
+ <property name="ivy.version" value="2.1.0" />
+ <property name="ivy.url"
+ value="http://repo2.maven.org/maven2/org/apache/ivy/ivy" />
+
+ <condition property="ivy.jar.exists">
+ <available file="${build.dir}/ivy-${ivy.version}.jar" />
+ </condition>
+
+ <path id="autoivy.classpath">
+ <fileset dir="${ivy.lib.dir}">
+ <include name="**/*.jar" />
+ </fileset>
+ <pathelement location="${build.dir}/ivy-${ivy.version}.jar"/>
+ </path>
+
+ <path id="wordcount.build.classpath">
+ <fileset dir="${ivy.lib.dir}">
+ <include name="**/*.jar" />
+ </fileset>
+ <!-- cassandra dependencies -->
+ <fileset dir="${cassandra.dir.lib}">
+ <include name="**/*.jar" />
+ </fileset>
+ <fileset dir="${cassandra.dir}/build/lib/jars">
+ <include name="**/*.jar" />
+ </fileset>
+ <pathelement location="${cassandra.classes}/main" />
+ <pathelement location="${cassandra.classes}/thrift" />
+ </path>
+
+ <target name="init">
+ <mkdir dir="${build.classes}" />
+ </target>
+
+ <target depends="init,ivy-retrieve-build" name="build">
+ <javac destdir="${build.classes}">
+ <src path="${build.src}" />
+ <classpath refid="wordcount.build.classpath" />
+ </javac>
+ </target>
+
+ <target name="jar" depends="build">
+ <mkdir dir="${build.classes}/META-INF" />
+ <jar jarfile="${build.dir}/${final.name}.jar">
+ <fileset dir="${build.classes}" />
+ <fileset dir="${cassandra.classes}/main" />
+ <fileset dir="${cassandra.classes}/thrift" />
+ <fileset dir="${cassandra.dir}">
+ <include name="lib/**/*.jar" />
+ </fileset>
+ <zipfileset dir="${cassandra.dir}/build/lib/jars/" prefix="lib">
+ <include name="**/*.jar" />
+ </zipfileset>
+ <fileset file="${basedir}/cassandra.yaml" />
+ </jar>
+ </target>
+
+ <target name="clean">
+ <delete dir="${build.dir}" />
+ </target>
+
+ <!--
+ Ivy Specific targets
+ to fetch Ivy and this project's dependencies
+ -->
+ <target name="ivy-download" unless="ivy.jar.exists">
+ <echo>Downloading Ivy...</echo>
+ <mkdir dir="${build.dir}" />
+ <get src="${ivy.url}/${ivy.version}/ivy-${ivy.version}.jar"
+ dest="${build.dir}/ivy-${ivy.version}.jar" usetimestamp="true" />
+ </target>
+
+ <target name="ivy-init" depends="ivy-download" unless="ivy.initialized">
+ <mkdir dir="${ivy.lib.dir}"/>
+ <taskdef resource="org/apache/ivy/ant/antlib.xml"
+ uri="antlib:org.apache.ivy.ant"
+ classpathref="autoivy.classpath"/>
+ <property name="ivy.initialized" value="true"/>
+ </target>
+
+ <target name="ivy-retrieve-build" depends="ivy-init">
+ <ivy:retrieve type="jar,source" sync="true"
+ pattern="${ivy.lib.dir}/[type]s/[artifact]-[revision].[ext]" />
+ </target>
+</project>
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/conf/log4j.properties b/examples/hadoop_cql3_word_count/conf/log4j.properties
new file mode 100644
index 0000000..70f7657
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/conf/log4j.properties
@@ -0,0 +1,32 @@
+# 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.
+
+log4j.rootLogger=DEBUG,stdout,F
+
+#stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%5p %d{HH:mm:ss,SSS} %m%n
+
+# log file
+log4j.appender.F=org.apache.log4j.FileAppender
+log4j.appender.F.Append=false
+log4j.appender.F.layout=org.apache.log4j.PatternLayout
+log4j.appender.F.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n
+# Edit the next line to point to your logs directory
+log4j.appender.F.File=wc.out
+
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/ivy.xml
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/ivy.xml b/examples/hadoop_cql3_word_count/ivy.xml
new file mode 100644
index 0000000..9d44895
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/ivy.xml
@@ -0,0 +1,24 @@
+<!--
+ ~ 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.
+ -->
+<ivy-module version="2.0">
+ <info organisation="apache-cassandra" module="word-count"/>
+ <dependencies>
+ <dependency org="org.apache.hadoop" name="hadoop-core" rev="0.20.2"/>
+ </dependencies>
+</ivy-module>
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/src/WordCount.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/src/WordCount.java b/examples/hadoop_cql3_word_count/src/WordCount.java
new file mode 100644
index 0000000..09dd9e4
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/src/WordCount.java
@@ -0,0 +1,236 @@
+/**
+ * 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 java.util.Map.Entry;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.hadoop.cql3.ColumnFamilyOutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.hadoop.cql3.CQLConfigHelper;
+import org.apache.cassandra.hadoop.cql3.ColumnFamilyInputFormat;
+import org.apache.cassandra.hadoop.ConfigHelper;
+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.Reducer.Context;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.nio.charset.CharacterCodingException;
+
+/**
+ * This counts the occurrences of words in ColumnFamily
+ * cql3_worldcount ( user_id text,
+ * category_id text,
+ * sub_category_id text,
+ * title text,
+ * body text,
+ * PRIMARY KEY (user_id, category_id, sub_category_id))
+ *
+ * For each word, we output the total number of occurrences across all body texts.
+ *
+ * When outputting to Cassandra, we write the word counts to column family
+ * output_words ( row_id1 text,
+ * row_id2 text,
+ * word text,
+ * count_num text,
+ * PRIMARY KEY ((row_id1, row_id2), word))
+ * as a {word, count} to columns: word, count_num with a row key of "word sum"
+ */
+public class WordCount extends Configured implements Tool
+{
+ private static final Logger logger = LoggerFactory.getLogger(WordCount.class);
+
+ static final String KEYSPACE = "cql3_worldcount";
+ static final String COLUMN_FAMILY = "inputs";
+
+ static final String OUTPUT_REDUCER_VAR = "output_reducer";
+ static final String OUTPUT_COLUMN_FAMILY = "output_words";
+
+ private static final String OUTPUT_PATH_PREFIX = "/tmp/word_count";
+
+ private static final String PRIMARY_KEY = "row_key";
+
+ 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<Map<String, ByteBuffer>, Map<String, ByteBuffer>, Text, IntWritable>
+ {
+ private final static IntWritable one = new IntWritable(1);
+ private Text word = new Text();
+ private ByteBuffer sourceColumn;
+
+ protected void setup(org.apache.hadoop.mapreduce.Mapper.Context context)
+ throws IOException, InterruptedException
+ {
+ }
+
+ public void map(Map<String, ByteBuffer> keys, Map<String, ByteBuffer> columns, Context context) throws IOException, InterruptedException
+ {
+ for (Entry<String, ByteBuffer> column : columns.entrySet())
+ {
+ if (!"body".equalsIgnoreCase(column.getKey()))
+ continue;
+
+ String value = ByteBufferUtil.string(column.getValue());
+
+ logger.debug("read {}:{}={} from {}",
+ new Object[] {toString(keys), column.getKey(), value, context.getInputSplit()});
+
+ StringTokenizer itr = new StringTokenizer(value);
+ while (itr.hasMoreTokens())
+ {
+ word.set(itr.nextToken());
+ context.write(word, one);
+ }
+ }
+ }
+
+ private String toString(Map<String, ByteBuffer> keys)
+ {
+ String result = "";
+ try
+ {
+ for (ByteBuffer key : keys.values())
+ result = result + ByteBufferUtil.string(key) + ":";
+ }
+ catch (CharacterCodingException e)
+ {
+ logger.error("Failed to print keys", e);
+ }
+ return result;
+ }
+ }
+
+ public static class ReducerToFilesystem extends Reducer<Text, IntWritable, Text, IntWritable>
+ {
+ public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException
+ {
+ int sum = 0;
+ for (IntWritable val : values)
+ sum += val.get();
+ context.write(key, new IntWritable(sum));
+ }
+ }
+
+ public static class ReducerToCassandra extends Reducer<Text, IntWritable, Map<String, ByteBuffer>, List<ByteBuffer>>
+ {
+ private Map<String, ByteBuffer> keys;
+ private ByteBuffer key;
+ protected void setup(org.apache.hadoop.mapreduce.Reducer.Context context)
+ throws IOException, InterruptedException
+ {
+ keys = new LinkedHashMap<String, ByteBuffer>();
+ String[] partitionKeys = context.getConfiguration().get(PRIMARY_KEY).split(",");
+ keys.put("row_id1", ByteBufferUtil.bytes(partitionKeys[0]));
+ keys.put("row_id2", ByteBufferUtil.bytes(partitionKeys[1]));
+ }
+
+ public void reduce(Text word, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException
+ {
+ int sum = 0;
+ for (IntWritable val : values)
+ sum += val.get();
+ context.write(keys, getBindVariables(word, sum));
+ }
+
+ private List<ByteBuffer> getBindVariables(Text word, int sum)
+ {
+ List<ByteBuffer> variables = new ArrayList<ByteBuffer>();
+ variables.add(keys.get("row_id1"));
+ variables.add(keys.get("row_id2"));
+ variables.add(ByteBufferUtil.bytes(word.toString()));
+ variables.add(ByteBufferUtil.bytes(String.valueOf(sum)));
+ return variables;
+ }
+ }
+
+ 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);
+
+ 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));
+ }
+ else
+ {
+ job.setReducerClass(ReducerToCassandra.class);
+
+ job.setMapOutputKeyClass(Text.class);
+ job.setMapOutputValueClass(IntWritable.class);
+ job.setOutputKeyClass(Map.class);
+ job.setOutputValueClass(List.class);
+
+ job.setOutputFormatClass(ColumnFamilyOutputFormat.class);
+
+ ConfigHelper.setOutputColumnFamily(job.getConfiguration(), KEYSPACE, OUTPUT_COLUMN_FAMILY);
+ job.getConfiguration().set(PRIMARY_KEY, "word,sum");
+ String query = "INSERT INTO " + KEYSPACE + "." + OUTPUT_COLUMN_FAMILY +
+ " (row_id1, row_id2, word, count_num) " +
+ " values (?, ?, ?, ?)";
+ CQLConfigHelper.setOutputCql(job.getConfiguration(), query);
+ ConfigHelper.setOutputInitialAddress(job.getConfiguration(), "localhost");
+ ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner");
+ }
+
+ job.setInputFormatClass(ColumnFamilyInputFormat.class);
+
+ ConfigHelper.setInputRpcPort(job.getConfiguration(), "9160");
+ ConfigHelper.setInputInitialAddress(job.getConfiguration(), "localhost");
+ ConfigHelper.setInputColumnFamily(job.getConfiguration(), KEYSPACE, COLUMN_FAMILY);
+ ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner");
+
+ CQLConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), "3");
+ //this is the user defined filter clauses, you can comment it out if you want count all titles
+ CQLConfigHelper.setInputWhereClauses(job.getConfiguration(), "title='A'");
+ job.waitForCompletion(true);
+ return 0;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/src/WordCountCounters.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/src/WordCountCounters.java b/examples/hadoop_cql3_word_count/src/WordCountCounters.java
new file mode 100644
index 0000000..1cf5539
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/src/WordCountCounters.java
@@ -0,0 +1,122 @@
+/**
+ * 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.nio.charset.CharacterCodingException;
+import java.util.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.io.LongWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.cassandra.hadoop.cql3.ColumnFamilyInputFormat;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.cql3.CQLConfigHelper;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+
+/**
+ * This sums the word count stored in the input_words_count ColumnFamily for the key "sum".
+ *
+ * Output is written to a text file.
+ */
+public class WordCountCounters extends Configured implements Tool
+{
+ private static final Logger logger = LoggerFactory.getLogger(WordCountCounters.class);
+
+ static final String COUNTER_COLUMN_FAMILY = "input_words_count";
+ private static final String OUTPUT_PATH_PREFIX = "/tmp/word_count_counters";
+
+ public static void main(String[] args) throws Exception
+ {
+ // Let ToolRunner handle generic command-line options
+ ToolRunner.run(new Configuration(), new WordCountCounters(), args);
+ System.exit(0);
+ }
+
+ public static class SumMapper extends Mapper<Map<String, ByteBuffer>, Map<String, ByteBuffer>, Text, LongWritable>
+ {
+ long sum = -1;
+ public void map(Map<String, ByteBuffer> key, Map<String, ByteBuffer> columns, Context context) throws IOException, InterruptedException
+ {
+ if (sum < 0)
+ sum = 0;
+
+ logger.debug("read " + toString(key) + ":count_num from " + context.getInputSplit());
+ sum += Long.valueOf(ByteBufferUtil.string(columns.get("count_num")));
+ }
+
+ protected void cleanup(Context context) throws IOException, InterruptedException {
+ if (sum > 0)
+ context.write(new Text("total_count"), new LongWritable(sum));
+ }
+
+ private String toString(Map<String, ByteBuffer> keys)
+ {
+ String result = "";
+ try
+ {
+ for (ByteBuffer key : keys.values())
+ result = result + ByteBufferUtil.string(key) + ":";
+ }
+ catch (CharacterCodingException e)
+ {
+ logger.error("Failed to print keys", e);
+ }
+ return result;
+ }
+ }
+
+
+ public int run(String[] args) throws Exception
+ {
+ Job job = new Job(getConf(), "wordcountcounters");
+ job.setJarByClass(WordCountCounters.class);
+ job.setMapperClass(SumMapper.class);
+
+ job.setOutputKeyClass(Text.class);
+ job.setOutputValueClass(LongWritable.class);
+ FileOutputFormat.setOutputPath(job, new Path(OUTPUT_PATH_PREFIX));
+
+ job.setInputFormatClass(ColumnFamilyInputFormat.class);
+
+ ConfigHelper.setInputRpcPort(job.getConfiguration(), "9160");
+ ConfigHelper.setInputInitialAddress(job.getConfiguration(), "localhost");
+ ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner");
+ ConfigHelper.setInputColumnFamily(job.getConfiguration(), WordCount.KEYSPACE, WordCount.OUTPUT_COLUMN_FAMILY);
+
+ CQLConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), "3");
+
+ job.waitForCompletion(true);
+ return 0;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/examples/hadoop_cql3_word_count/src/WordCountSetup.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/src/WordCountSetup.java b/examples/hadoop_cql3_word_count/src/WordCountSetup.java
new file mode 100644
index 0000000..4c22e49
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/src/WordCountSetup.java
@@ -0,0 +1,214 @@
+/**
+ * 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.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class WordCountSetup
+{
+ private static final Logger logger = LoggerFactory.getLogger(WordCountSetup.class);
+
+ public static final int TEST_COUNT = 6;
+
+ public static void main(String[] args) throws Exception
+ {
+ Cassandra.Iface client = createConnection();
+
+ setupKeyspace(client);
+ client.set_keyspace(WordCount.KEYSPACE);
+ setupTable(client);
+ insertData(client);
+
+ System.exit(0);
+ }
+
+ private static void setupKeyspace(Cassandra.Iface client)
+ throws InvalidRequestException,
+ UnavailableException,
+ TimedOutException,
+ SchemaDisagreementException,
+ TException
+ {
+ KsDef ks;
+ try
+ {
+ ks = client.describe_keyspace(WordCount.KEYSPACE);
+ }
+ catch(NotFoundException e)
+ {
+ logger.info("set up keyspace " + WordCount.KEYSPACE);
+ String query = "CREATE KEYSPACE " + WordCount.KEYSPACE +
+ " WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1}";
+
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+
+ int magnitude = client.describe_ring(WordCount.KEYSPACE).size();
+ try
+ {
+ Thread.sleep(1000 * magnitude);
+ }
+ catch (InterruptedException ie)
+ {
+ throw new RuntimeException(ie);
+ }
+ }
+ }
+
+ private static void setupTable(Cassandra.Iface client)
+ throws InvalidRequestException,
+ UnavailableException,
+ TimedOutException,
+ SchemaDisagreementException,
+ TException
+ {
+ String query = "CREATE TABLE " + WordCount.KEYSPACE + "." + WordCount.COLUMN_FAMILY +
+ " ( user_id text," +
+ " category_id text, " +
+ " sub_category_id text," +
+ " title text," +
+ " body text," +
+ " PRIMARY KEY (user_id, category_id, sub_category_id) ) ";
+
+ try
+ {
+ logger.info("set up table " + WordCount.COLUMN_FAMILY);
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ logger.error("failed to create table " + WordCount.KEYSPACE + "." + WordCount.COLUMN_FAMILY, e);
+ }
+
+ query = "CREATE INDEX title on " + WordCount.COLUMN_FAMILY + "(title)";
+ try
+ {
+ logger.info("set up index on title column ");
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ logger.error("Failed to create index on title", e);
+ }
+
+ query = "CREATE TABLE " + WordCount.KEYSPACE + "." + WordCount.OUTPUT_COLUMN_FAMILY +
+ " ( row_id1 text, " +
+ " row_id2 text, " +
+ " word text, " +
+ " count_num text," +
+ " PRIMARY KEY ((row_id1, row_id2), word) ) ";
+
+ try
+ {
+ logger.info("set up table " + WordCount.OUTPUT_COLUMN_FAMILY);
+ client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+ }
+ catch (InvalidRequestException e)
+ {
+ logger.error("failed to create table " + WordCount.KEYSPACE + "." + WordCount.OUTPUT_COLUMN_FAMILY, e);
+ }
+ }
+
+ private static Cassandra.Iface createConnection() throws TTransportException
+ {
+ if (System.getProperty("cassandra.host") == null || System.getProperty("cassandra.port") == null)
+ {
+ logger.warn("cassandra.host or cassandra.port is not defined, using default");
+ }
+ return createConnection(System.getProperty("cassandra.host", "localhost"),
+ Integer.valueOf(System.getProperty("cassandra.port", "9160")));
+ }
+
+ private static Cassandra.Client createConnection(String host, Integer port) throws TTransportException
+ {
+ TSocket socket = new TSocket(host, port);
+ TTransport trans = new TFramedTransport(socket);
+ trans.open();
+ TProtocol protocol = new TBinaryProtocol(trans);
+
+ return new Cassandra.Client(protocol);
+ }
+
+ private static void insertData(Cassandra.Iface client)
+ throws InvalidRequestException,
+ UnavailableException,
+ TimedOutException,
+ SchemaDisagreementException,
+ TException
+ {
+ String query = "INSERT INTO " + WordCount.COLUMN_FAMILY +
+ "(user_id, category_id, sub_category_id, title, body ) " +
+ " values (?, ?, ?, ?, ?) ";
+ CqlPreparedResult result = client.prepare_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE);
+
+ String [] title = titleData();
+ String [] body = bodyData();
+ for (int i=1; i<5; i++)
+ {
+ for (int j=1; j<444; j++)
+ {
+ for (int k=1; k<4; k++)
+ {
+ List<ByteBuffer> values = new ArrayList<ByteBuffer>();
+ values.add(ByteBufferUtil.bytes(String.valueOf(j)));
+ values.add(ByteBufferUtil.bytes(String.valueOf(i)));
+ values.add(ByteBufferUtil.bytes(String.valueOf(k)));
+ values.add(ByteBufferUtil.bytes(title[i]));
+ values.add(ByteBufferUtil.bytes(body[i]));
+ client.execute_prepared_cql3_query(result.itemId, values, ConsistencyLevel.ONE);
+ }
+ }
+ }
+ }
+
+ private static String[] bodyData()
+ { // Public domain context, source http://en.wikisource.org/wiki/If%E2%80%94
+ return new String[]{
+ "",
+ "If you can keep your head when all about you",
+ "Are losing theirs and blaming it on you",
+ "If you can trust yourself when all men doubt you,",
+ "But make allowance for their doubting too:",
+ "If you can wait and not be tired by waiting,"
+ };
+ }
+
+ private static String[] titleData()
+ { // Public domain context, source http://en.wikisource.org/wiki/If%E2%80%94
+ return new String[]{
+ "",
+ "A",
+ "B",
+ "C",
+ "D",
+ "E"
+ };
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/client/RingCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/client/RingCache.java b/src/java/org/apache/cassandra/client/RingCache.java
index 4db4a0d..3308471 100644
--- a/src/java/org/apache/cassandra/client/RingCache.java
+++ b/src/java/org/apache/cassandra/client/RingCache.java
@@ -52,7 +52,7 @@ public class RingCache
private Multimap<Range<Token>, InetAddress> rangeMap;
- public RingCache(Configuration conf) throws IOException
+ public RingCache(Configuration conf)
{
this.conf = conf;
this.partitioner = ConfigHelper.getOutputPartitioner(conf);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
new file mode 100644
index 0000000..1c8fd0b
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
@@ -0,0 +1,346 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.cassandra.auth.IAuthenticator;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.thrift.AuthenticationRequest;
+import org.apache.cassandra.thrift.Cassandra;
+import org.apache.cassandra.thrift.CfSplit;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.thrift.KeyRange;
+import org.apache.cassandra.thrift.TokenRange;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<K, Y> implements org.apache.hadoop.mapred.InputFormat<K, Y>
+{
+ private static final Logger logger = LoggerFactory.getLogger(AbstractColumnFamilyInputFormat.class);
+
+ public static final String MAPRED_TASK_ID = "mapred.task.id";
+ // The simple fact that we need this is because the old Hadoop API wants us to "write"
+ // to the key and value whereas the new asks for it.
+ // I choose 8kb as the default max key size (instanciated only once), but you can
+ // override it in your jobConf with this setting.
+ public static final String CASSANDRA_HADOOP_MAX_KEY_SIZE = "cassandra.hadoop.max_key_size";
+ public static final int CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT = 8192;
+
+ private String keyspace;
+ private String cfName;
+ private IPartitioner partitioner;
+
+ protected void validateConfiguration(Configuration conf)
+ {
+ if (ConfigHelper.getInputKeyspace(conf) == null || ConfigHelper.getInputColumnFamily(conf) == null)
+ {
+ throw new UnsupportedOperationException("you must set the keyspace and columnfamily with setInputColumnFamily()");
+ }
+ if (ConfigHelper.getInputInitialAddress(conf) == null)
+ throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node with setInputInitialAddress");
+ if (ConfigHelper.getInputPartitioner(conf) == null)
+ throw new UnsupportedOperationException("You must set the Cassandra partitioner class with setInputPartitioner");
+ }
+
+ public static Cassandra.Client createAuthenticatedClient(String location, int port, Configuration conf) throws Exception
+ {
+ logger.debug("Creating authenticated client for CF input format");
+ TTransport transport = ConfigHelper.getClientTransportFactory(conf).openTransport(location, port, conf);
+ TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
+ Cassandra.Client client = new Cassandra.Client(binaryProtocol);
+
+ // log in
+ client.set_keyspace(ConfigHelper.getInputKeyspace(conf));
+ if (ConfigHelper.getInputKeyspaceUserName(conf) != null)
+ {
+ Map<String, String> creds = new HashMap<String, String>();
+ creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf));
+ creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf));
+ AuthenticationRequest authRequest = new AuthenticationRequest(creds);
+ client.login(authRequest);
+ }
+ logger.debug("Authenticated client for CF input format created successfully");
+ return client;
+ }
+
+ public List<InputSplit> getSplits(JobContext context) throws IOException
+ {
+ Configuration conf = context.getConfiguration();
+
+ validateConfiguration(conf);
+
+ // cannonical ranges and nodes holding replicas
+ List<TokenRange> masterRangeNodes = getRangeMap(conf);
+
+ keyspace = ConfigHelper.getInputKeyspace(context.getConfiguration());
+ cfName = ConfigHelper.getInputColumnFamily(context.getConfiguration());
+ partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
+ logger.debug("partitioner is " + partitioner);
+
+ // cannonical ranges, split into pieces, fetching the splits in parallel
+ ExecutorService executor = Executors.newCachedThreadPool();
+ List<InputSplit> splits = new ArrayList<InputSplit>();
+
+ try
+ {
+ List<Future<List<InputSplit>>> splitfutures = new ArrayList<Future<List<InputSplit>>>();
+ KeyRange jobKeyRange = ConfigHelper.getInputKeyRange(conf);
+ Range<Token> jobRange = null;
+ if (jobKeyRange != null)
+ {
+ if (jobKeyRange.start_key == null)
+ {
+ logger.warn("ignoring jobKeyRange specified without start_key");
+ }
+ else
+ {
+ if (!partitioner.preservesOrder())
+ throw new UnsupportedOperationException("KeyRange based on keys can only be used with a order preserving paritioner");
+ if (jobKeyRange.start_token != null)
+ throw new IllegalArgumentException("only start_key supported");
+ if (jobKeyRange.end_token != null)
+ throw new IllegalArgumentException("only start_key supported");
+ jobRange = new Range<Token>(partitioner.getToken(jobKeyRange.start_key),
+ partitioner.getToken(jobKeyRange.end_key),
+ partitioner);
+ }
+ }
+
+ for (TokenRange range : masterRangeNodes)
+ {
+ if (jobRange == null)
+ {
+ // for each range, pick a live owner and ask it to compute bite-sized splits
+ splitfutures.add(executor.submit(new SplitCallable(range, conf)));
+ }
+ else
+ {
+ Range<Token> dhtRange = new Range<Token>(partitioner.getTokenFactory().fromString(range.start_token),
+ partitioner.getTokenFactory().fromString(range.end_token),
+ partitioner);
+
+ if (dhtRange.intersects(jobRange))
+ {
+ for (Range<Token> intersection: dhtRange.intersectionWith(jobRange))
+ {
+ range.start_token = partitioner.getTokenFactory().toString(intersection.left);
+ range.end_token = partitioner.getTokenFactory().toString(intersection.right);
+ // for each range, pick a live owner and ask it to compute bite-sized splits
+ splitfutures.add(executor.submit(new SplitCallable(range, conf)));
+ }
+ }
+ }
+ }
+
+ // wait until we have all the results back
+ for (Future<List<InputSplit>> futureInputSplits : splitfutures)
+ {
+ try
+ {
+ splits.addAll(futureInputSplits.get());
+ }
+ catch (Exception e)
+ {
+ throw new IOException("Could not get input splits", e);
+ }
+ }
+ }
+ finally
+ {
+ executor.shutdownNow();
+ }
+
+ assert splits.size() > 0;
+ Collections.shuffle(splits, new Random(System.nanoTime()));
+ return splits;
+ }
+
+ /**
+ * Gets a token range and splits it up according to the suggested
+ * size into input splits that Hadoop can use.
+ */
+ class SplitCallable implements Callable<List<InputSplit>>
+ {
+
+ private final TokenRange range;
+ private final Configuration conf;
+
+ public SplitCallable(TokenRange tr, Configuration conf)
+ {
+ this.range = tr;
+ this.conf = conf;
+ }
+
+ public List<InputSplit> call() throws Exception
+ {
+ ArrayList<InputSplit> splits = new ArrayList<InputSplit>();
+ List<CfSplit> subSplits = getSubSplits(keyspace, cfName, range, conf);
+ assert range.rpc_endpoints.size() == range.endpoints.size() : "rpc_endpoints size must match endpoints size";
+ // turn the sub-ranges into InputSplits
+ String[] endpoints = range.endpoints.toArray(new String[range.endpoints.size()]);
+ // hadoop needs hostname, not ip
+ int endpointIndex = 0;
+ for (String endpoint: range.rpc_endpoints)
+ {
+ String endpoint_address = endpoint;
+ if (endpoint_address == null || endpoint_address.equals("0.0.0.0"))
+ endpoint_address = range.endpoints.get(endpointIndex);
+ endpoints[endpointIndex++] = InetAddress.getByName(endpoint_address).getHostName();
+ }
+
+ Token.TokenFactory factory = partitioner.getTokenFactory();
+ for (CfSplit subSplit : subSplits)
+ {
+ Token left = factory.fromString(subSplit.getStart_token());
+ Token right = factory.fromString(subSplit.getEnd_token());
+ Range<Token> range = new Range<Token>(left, right, partitioner);
+ List<Range<Token>> ranges = range.isWrapAround() ? range.unwrap() : ImmutableList.of(range);
+ for (Range<Token> subrange : ranges)
+ {
+ ColumnFamilySplit split =
+ new ColumnFamilySplit(
+ factory.toString(subrange.left),
+ factory.toString(subrange.right),
+ subSplit.getRow_count(),
+ endpoints);
+
+ logger.debug("adding " + split);
+ splits.add(split);
+ }
+ }
+ return splits;
+ }
+ }
+
+ private List<CfSplit> getSubSplits(String keyspace, String cfName, TokenRange range, Configuration conf) throws IOException
+ {
+ int splitsize = ConfigHelper.getInputSplitSize(conf);
+ for (int i = 0; i < range.rpc_endpoints.size(); i++)
+ {
+ String host = range.rpc_endpoints.get(i);
+
+ if (host == null || host.equals("0.0.0.0"))
+ host = range.endpoints.get(i);
+
+ try
+ {
+ Cassandra.Client client = ConfigHelper.createConnection(conf, host, ConfigHelper.getInputRpcPort(conf));
+ client.set_keyspace(keyspace);
+
+ try
+ {
+ return client.describe_splits_ex(cfName, range.start_token, range.end_token, splitsize);
+ }
+ catch (TApplicationException e)
+ {
+ // fallback to guessing split size if talking to a server without describe_splits_ex method
+ if (e.getType() == TApplicationException.UNKNOWN_METHOD)
+ {
+ List<String> splitPoints = client.describe_splits(cfName, range.start_token, range.end_token, splitsize);
+ return tokenListToSplits(splitPoints, splitsize);
+ }
+ throw e;
+ }
+ }
+ catch (IOException e)
+ {
+ logger.debug("failed connect to endpoint " + host, e);
+ }
+ catch (InvalidRequestException e)
+ {
+ throw new RuntimeException(e);
+ }
+ catch (TException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+ throw new IOException("failed connecting to all endpoints " + StringUtils.join(range.endpoints, ","));
+ }
+
+ private List<CfSplit> tokenListToSplits(List<String> splitTokens, int splitsize)
+ {
+ List<CfSplit> splits = Lists.newArrayListWithExpectedSize(splitTokens.size() - 1);
+ for (int j = 0; j < splitTokens.size() - 1; j++)
+ splits.add(new CfSplit(splitTokens.get(j), splitTokens.get(j + 1), splitsize));
+ return splits;
+ }
+
+ private List<TokenRange> getRangeMap(Configuration conf) throws IOException
+ {
+ Cassandra.Client client = ConfigHelper.getClientFromInputAddressList(conf);
+
+ List<TokenRange> map;
+ try
+ {
+ map = client.describe_ring(ConfigHelper.getInputKeyspace(conf));
+ }
+ catch (InvalidRequestException e)
+ {
+ throw new RuntimeException(e);
+ }
+ catch (TException e)
+ {
+ throw new RuntimeException(e);
+ }
+ return map;
+ }
+
+ //
+ // Old Hadoop API
+ //
+ public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException
+ {
+ TaskAttemptContext tac = new TaskAttemptContext(jobConf, new TaskAttemptID());
+ List<org.apache.hadoop.mapreduce.InputSplit> newInputSplits = this.getSplits(tac);
+ org.apache.hadoop.mapred.InputSplit[] oldInputSplits = new org.apache.hadoop.mapred.InputSplit[newInputSplits.size()];
+ for (int i = 0; i < newInputSplits.size(); i++)
+ oldInputSplits[i] = (ColumnFamilySplit)newInputSplits.get(i);
+ return oldInputSplits;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
new file mode 100644
index 0000000..5a03777
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop;
+
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.auth.IAuthenticator;
+import org.apache.cassandra.thrift.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TTransport;
+
+/**
+ * The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
+ * OutputFormat that allows reduce tasks to store keys (and corresponding
+ * values) as Cassandra rows (and respective columns) in a given
+ * ColumnFamily.
+ *
+ * <p>
+ * As is the case with the {@link ColumnFamilyInputFormat}, you need to set the
+ * Keyspace and ColumnFamily in your
+ * Hadoop job Configuration. The {@link ConfigHelper} class, through its
+ * {@link ConfigHelper#setOutputColumnFamily} method, is provided to make this
+ * simple.
+ * </p>
+ *
+ * <p>
+ * For the sake of performance, this class employs a lazy write-back caching
+ * mechanism, where its record writer batches mutations created based on the
+ * reduce's inputs (in a task-specific map), and periodically makes the changes
+ * official by sending a batch mutate request to Cassandra.
+ * </p>
+ * @param <Y>
+ */
+public abstract class AbstractColumnFamilyOutputFormat<K, Y> extends OutputFormat<K, Y> implements org.apache.hadoop.mapred.OutputFormat<K, Y>
+{
+ public static final String BATCH_THRESHOLD = "mapreduce.output.columnfamilyoutputformat.batch.threshold";
+ public static final String QUEUE_SIZE = "mapreduce.output.columnfamilyoutputformat.queue.size";
+ private static final Logger logger = LoggerFactory.getLogger(AbstractColumnFamilyOutputFormat.class);
+
+
+ /**
+ * Check for validity of the output-specification for the job.
+ *
+ * @param context
+ * information about the job
+ * @throws IOException
+ * when output should not be attempted
+ */
+ public void checkOutputSpecs(JobContext context)
+ {
+ checkOutputSpecs(context.getConfiguration());
+ }
+
+ protected void checkOutputSpecs(Configuration conf)
+ {
+ if (ConfigHelper.getOutputKeyspace(conf) == null)
+ throw new UnsupportedOperationException("You must set the keyspace with setOutputKeyspace()");
+ if (ConfigHelper.getOutputPartitioner(conf) == null)
+ throw new UnsupportedOperationException("You must set the output partitioner to the one used by your Cassandra cluster");
+ if (ConfigHelper.getOutputInitialAddress(conf) == null)
+ throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node");
+ }
+
+ /** Fills the deprecated OutputFormat interface for streaming. */
+ @Deprecated
+ public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
+ {
+ checkOutputSpecs(job);
+ }
+
+ /**
+ * The OutputCommitter for this format does not write any data to the DFS.
+ *
+ * @param context
+ * the task context
+ * @return an output committer
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
+ {
+ return new NullOutputCommitter();
+ }
+
+ /**
+ * Connects to the given server:port and returns a client based on the given socket that points to the configured
+ * keyspace, and is logged in with the configured credentials.
+ *
+ * @param host fully qualified host name to connect to
+ * @param port RPC port of the server
+ * @param conf a job configuration
+ * @return a cassandra client
+ * @throws Exception set of thrown exceptions may be implementation defined,
+ * depending on the used transport factory
+ */
+ public static Cassandra.Client createAuthenticatedClient(String host, int port, Configuration conf) throws Exception
+ {
+ logger.debug("Creating authenticated client for CF output format");
+ TTransport transport = ConfigHelper.getClientTransportFactory(conf).openTransport(host, port, conf);
+ TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
+ Cassandra.Client client = new Cassandra.Client(binaryProtocol);
+ client.set_keyspace(ConfigHelper.getOutputKeyspace(conf));
+ if (ConfigHelper.getOutputKeyspaceUserName(conf) != null)
+ {
+ Map<String, String> creds = new HashMap<String, String>();
+ creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getOutputKeyspaceUserName(conf));
+ creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getOutputKeyspacePassword(conf));
+ AuthenticationRequest authRequest = new AuthenticationRequest(creds);
+ client.login(authRequest);
+ }
+ logger.debug("Authenticated client for CF output format created successfully");
+ return client;
+ }
+
+ /**
+ * An {@link OutputCommitter} that does nothing.
+ */
+ private static class NullOutputCommitter extends OutputCommitter
+ {
+ public void abortTask(TaskAttemptContext taskContext) { }
+
+ public void cleanupJob(JobContext jobContext) { }
+
+ public void commitTask(TaskAttemptContext taskContext) { }
+
+ public boolean needsTaskCommit(TaskAttemptContext taskContext)
+ {
+ return false;
+ }
+
+ public void setupJob(JobContext jobContext) { }
+
+ public void setupTask(TaskAttemptContext taskContext) { }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
new file mode 100644
index 0000000..6428db3
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
@@ -0,0 +1,193 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop;
+
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.cassandra.client.RingCache;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.thrift.transport.TTransport;
+
+
+/**
+ * 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>
+ * 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>
+ *
+ * @see ColumnFamilyOutputFormat
+ */
+public abstract class AbstractColumnFamilyRecordWriter<K, Y> extends RecordWriter<K, Y> implements org.apache.hadoop.mapred.RecordWriter<K, Y>
+{
+ // The configuration this writer is associated with.
+ protected final Configuration conf;
+
+ // The ring cache that describes the token ranges each node in the ring is
+ // responsible for. This is what allows us to group the mutations by
+ // the endpoints they should be targeted at. The targeted endpoint
+ // essentially
+ // acts as the primary replica for the rows being affected by the mutations.
+ protected final RingCache ringCache;
+
+ // The number of mutations to buffer per endpoint
+ protected final int queueSize;
+
+ protected final long batchThreshold;
+
+ protected final ConsistencyLevel consistencyLevel;
+ protected Progressable progressable;
+
+ protected AbstractColumnFamilyRecordWriter(Configuration conf)
+ {
+ this.conf = conf;
+ this.ringCache = new RingCache(conf);
+ this.queueSize = conf.getInt(AbstractColumnFamilyOutputFormat.QUEUE_SIZE, 32 * FBUtilities.getAvailableProcessors());
+ batchThreshold = conf.getLong(AbstractColumnFamilyOutputFormat.BATCH_THRESHOLD, 32);
+ consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getWriteConsistencyLevel(conf));
+ }
+
+ /**
+ * Close this <code>RecordWriter</code> to future operations, but not before
+ * flushing out the batched mutations.
+ *
+ * @param context the context of the task
+ * @throws IOException
+ */
+ public void close(TaskAttemptContext context) throws IOException, InterruptedException
+ {
+ close();
+ }
+
+ /** Fills the deprecated RecordWriter interface for streaming. */
+ @Deprecated
+ public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
+ {
+ close();
+ }
+
+ protected abstract void close() throws IOException;
+
+ /**
+ * A client that runs in a threadpool and connects to the list of endpoints for a particular
+ * range. Mutations for keys in that range are sent to this client via a queue.
+ */
+ public abstract class AbstractRangeClient<K> extends Thread
+ {
+ // The list of endpoints for this range
+ protected final List<InetAddress> endpoints;
+ // A bounded queue of incoming mutations for this range
+ protected final BlockingQueue<Pair<ByteBuffer, K>> queue = new ArrayBlockingQueue<Pair<ByteBuffer, K>>(queueSize);
+
+ protected volatile boolean run = true;
+ // we want the caller to know if something went wrong, so we record any unrecoverable exception while writing
+ // so we can throw it on the caller's stack when he calls put() again, or if there are no more put calls,
+ // when the client is closed.
+ protected volatile IOException lastException;
+
+ protected Cassandra.Client client;
+
+ /**
+ * Constructs an {@link AbstractRangeClient} for the given endpoints.
+ * @param endpoints the possible endpoints to execute the mutations on
+ */
+ public AbstractRangeClient(List<InetAddress> endpoints)
+ {
+ super("client-" + endpoints);
+ this.endpoints = endpoints;
+ }
+
+ /**
+ * enqueues the given value to Cassandra
+ */
+ public void put(Pair<ByteBuffer, K> value) throws IOException
+ {
+ while (true)
+ {
+ if (lastException != null)
+ throw lastException;
+ try
+ {
+ if (queue.offer(value, 100, TimeUnit.MILLISECONDS))
+ break;
+ }
+ catch (InterruptedException e)
+ {
+ throw new AssertionError(e);
+ }
+ }
+ }
+
+ public void close() throws IOException
+ {
+ // stop the run loop. this will result in closeInternal being called by the time join() finishes.
+ run = false;
+ interrupt();
+ try
+ {
+ this.join();
+ }
+ catch (InterruptedException e)
+ {
+ throw new AssertionError(e);
+ }
+
+ if (lastException != null)
+ throw lastException;
+ }
+
+ protected void closeInternal()
+ {
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ }
+ }
+
+ /**
+ * Loops collecting mutations from the queue and sending to Cassandra
+ */
+ public abstract void run();
+
+ @Override
+ public String toString()
+ {
+ return "#<Client for " + endpoints.toString() + ">";
+ }
+ }
+}
+
[02/10] Add CQL3 input/output formats patch by Alex Liu;
reviewed by jbellis and Mike Schrag for CASSANDRA-4421
Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
index e95e7ad..f77352a 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
@@ -18,35 +18,14 @@
package org.apache.cassandra.hadoop;
import java.io.IOException;
-import java.net.InetAddress;
import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Random;
-import java.util.SortedMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.commons.lang.StringUtils;
-import org.apache.thrift.TApplicationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.*;
import org.apache.cassandra.db.IColumn;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.thrift.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.*;
-import org.apache.thrift.TException;
/**
* Hadoop InputFormat allowing map/reduce against Cassandra rows within one ColumnFamily.
@@ -65,283 +44,14 @@ import org.apache.thrift.TException;
*
* The default split size is 64k rows.
*/
-public class ColumnFamilyInputFormat extends InputFormat<ByteBuffer, SortedMap<ByteBuffer, IColumn>>
- implements org.apache.hadoop.mapred.InputFormat<ByteBuffer, SortedMap<ByteBuffer, IColumn>>
+public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<ByteBuffer, SortedMap<ByteBuffer, IColumn>>
{
- private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyInputFormat.class);
-
- public static final String MAPRED_TASK_ID = "mapred.task.id";
- // The simple fact that we need this is because the old Hadoop API wants us to "write"
- // to the key and value whereas the new asks for it.
- // I choose 8kb as the default max key size (instanciated only once), but you can
- // override it in your jobConf with this setting.
- public static final String CASSANDRA_HADOOP_MAX_KEY_SIZE = "cassandra.hadoop.max_key_size";
- public static final int CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT = 8192;
-
- private String keyspace;
- private String cfName;
- private IPartitioner partitioner;
-
- private static void validateConfiguration(Configuration conf)
- {
- if (ConfigHelper.getInputKeyspace(conf) == null || ConfigHelper.getInputColumnFamily(conf) == null)
- {
- throw new UnsupportedOperationException("you must set the keyspace and columnfamily with setInputColumnFamily()");
- }
- if (ConfigHelper.getInputSlicePredicate(conf) == null)
- {
- throw new UnsupportedOperationException("you must set the predicate with setInputSlicePredicate");
- }
- if (ConfigHelper.getInputInitialAddress(conf) == null)
- throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node with setInputInitialAddress");
- if (ConfigHelper.getInputPartitioner(conf) == null)
- throw new UnsupportedOperationException("You must set the Cassandra partitioner class with setInputPartitioner");
- }
-
- public List<InputSplit> getSplits(JobContext context) throws IOException
- {
- Configuration conf = context.getConfiguration();
-
- validateConfiguration(conf);
-
- // cannonical ranges and nodes holding replicas
- List<TokenRange> masterRangeNodes = getRangeMap(conf);
-
- keyspace = ConfigHelper.getInputKeyspace(context.getConfiguration());
- cfName = ConfigHelper.getInputColumnFamily(context.getConfiguration());
- partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
- logger.debug("partitioner is " + partitioner);
-
- // cannonical ranges, split into pieces, fetching the splits in parallel
- ExecutorService executor = Executors.newCachedThreadPool();
- List<InputSplit> splits = new ArrayList<InputSplit>();
-
- try
- {
- List<Future<List<InputSplit>>> splitfutures = new ArrayList<Future<List<InputSplit>>>();
- KeyRange jobKeyRange = ConfigHelper.getInputKeyRange(conf);
- Range<Token> jobRange = null;
- if (jobKeyRange != null)
- {
- if (jobKeyRange.start_key == null)
- {
- logger.warn("ignoring jobKeyRange specified without start_key");
- }
- else
- {
- if (!partitioner.preservesOrder())
- throw new UnsupportedOperationException("KeyRange based on keys can only be used with a order preserving paritioner");
- if (jobKeyRange.start_token != null)
- throw new IllegalArgumentException("only start_key supported");
- if (jobKeyRange.end_token != null)
- throw new IllegalArgumentException("only start_key supported");
- jobRange = new Range<Token>(partitioner.getToken(jobKeyRange.start_key),
- partitioner.getToken(jobKeyRange.end_key),
- partitioner);
- }
- }
-
- for (TokenRange range : masterRangeNodes)
- {
- if (jobRange == null)
- {
- // for each range, pick a live owner and ask it to compute bite-sized splits
- splitfutures.add(executor.submit(new SplitCallable(range, conf)));
- }
- else
- {
- Range<Token> dhtRange = new Range<Token>(partitioner.getTokenFactory().fromString(range.start_token),
- partitioner.getTokenFactory().fromString(range.end_token),
- partitioner);
-
- if (dhtRange.intersects(jobRange))
- {
- for (Range<Token> intersection: dhtRange.intersectionWith(jobRange))
- {
- range.start_token = partitioner.getTokenFactory().toString(intersection.left);
- range.end_token = partitioner.getTokenFactory().toString(intersection.right);
- // for each range, pick a live owner and ask it to compute bite-sized splits
- splitfutures.add(executor.submit(new SplitCallable(range, conf)));
- }
- }
- }
- }
-
- // wait until we have all the results back
- for (Future<List<InputSplit>> futureInputSplits : splitfutures)
- {
- try
- {
- splits.addAll(futureInputSplits.get());
- }
- catch (Exception e)
- {
- throw new IOException("Could not get input splits", e);
- }
- }
- }
- finally
- {
- executor.shutdownNow();
- }
-
- assert splits.size() > 0;
- Collections.shuffle(splits, new Random(System.nanoTime()));
- return splits;
- }
-
- /**
- * Gets a token range and splits it up according to the suggested
- * size into input splits that Hadoop can use.
- */
- class SplitCallable implements Callable<List<InputSplit>>
- {
-
- private final TokenRange range;
- private final Configuration conf;
-
- public SplitCallable(TokenRange tr, Configuration conf)
- {
- this.range = tr;
- this.conf = conf;
- }
-
- public List<InputSplit> call() throws Exception
- {
- ArrayList<InputSplit> splits = new ArrayList<InputSplit>();
- List<CfSplit> subSplits = getSubSplits(keyspace, cfName, range, conf);
- assert range.rpc_endpoints.size() == range.endpoints.size() : "rpc_endpoints size must match endpoints size";
- // turn the sub-ranges into InputSplits
- String[] endpoints = range.endpoints.toArray(new String[range.endpoints.size()]);
- // hadoop needs hostname, not ip
- int endpointIndex = 0;
- for (String endpoint: range.rpc_endpoints)
- {
- String endpoint_address = endpoint;
- if (endpoint_address == null || endpoint_address.equals("0.0.0.0"))
- endpoint_address = range.endpoints.get(endpointIndex);
- endpoints[endpointIndex++] = InetAddress.getByName(endpoint_address).getHostName();
- }
-
- Token.TokenFactory factory = partitioner.getTokenFactory();
- for (CfSplit subSplit : subSplits)
- {
- Token left = factory.fromString(subSplit.getStart_token());
- Token right = factory.fromString(subSplit.getEnd_token());
- Range<Token> range = new Range<Token>(left, right, partitioner);
- List<Range<Token>> ranges = range.isWrapAround() ? range.unwrap() : ImmutableList.of(range);
- for (Range<Token> subrange : ranges)
- {
- ColumnFamilySplit split =
- new ColumnFamilySplit(
- factory.toString(subrange.left),
- factory.toString(subrange.right),
- subSplit.getRow_count(),
- endpoints);
-
- logger.debug("adding " + split);
- splits.add(split);
- }
- }
- return splits;
- }
- }
-
- private List<CfSplit> getSubSplits(String keyspace, String cfName, TokenRange range, Configuration conf) throws IOException
- {
- int splitsize = ConfigHelper.getInputSplitSize(conf);
- for (int i = 0; i < range.rpc_endpoints.size(); i++)
- {
- String host = range.rpc_endpoints.get(i);
-
- if (host == null || host.equals("0.0.0.0"))
- host = range.endpoints.get(i);
-
- try
- {
- Cassandra.Client client = ConfigHelper.createConnection(conf, host, ConfigHelper.getInputRpcPort(conf));
- client.set_keyspace(keyspace);
-
- try
- {
- return client.describe_splits_ex(cfName, range.start_token, range.end_token, splitsize);
- }
- catch (TApplicationException e)
- {
- // fallback to guessing split size if talking to a server without describe_splits_ex method
- if (e.getType() == TApplicationException.UNKNOWN_METHOD)
- {
- List<String> splitPoints = client.describe_splits(cfName, range.start_token, range.end_token, splitsize);
- return tokenListToSplits(splitPoints, splitsize);
- }
- throw e;
- }
- }
- catch (IOException e)
- {
- logger.debug("failed connect to endpoint " + host, e);
- }
- catch (TException e)
- {
- throw new RuntimeException(e);
- }
- catch (InvalidRequestException e)
- {
- throw new RuntimeException(e);
- }
- }
- throw new IOException("failed connecting to all endpoints " + StringUtils.join(range.endpoints, ","));
- }
-
-
- private List<CfSplit> tokenListToSplits(List<String> splitTokens, int splitsize)
- {
- List<CfSplit> splits = Lists.newArrayListWithExpectedSize(splitTokens.size() - 1);
- for (int j = 0; j < splitTokens.size() - 1; j++)
- splits.add(new CfSplit(splitTokens.get(j), splitTokens.get(j + 1), splitsize));
- return splits;
- }
-
-
- private List<TokenRange> getRangeMap(Configuration conf) throws IOException
- {
- Cassandra.Client client = ConfigHelper.getClientFromInputAddressList(conf);
-
- List<TokenRange> map;
- try
- {
- map = client.describe_ring(ConfigHelper.getInputKeyspace(conf));
- }
- catch (TException e)
- {
- throw new RuntimeException(e);
- }
- catch (InvalidRequestException e)
- {
- throw new RuntimeException(e);
- }
- return map;
- }
-
+
public RecordReader<ByteBuffer, SortedMap<ByteBuffer, IColumn>> createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException
{
return new ColumnFamilyRecordReader();
}
-
- //
- // Old Hadoop API
- //
- public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException
- {
- TaskAttemptContext tac = new TaskAttemptContext(jobConf, new TaskAttemptID());
- List<org.apache.hadoop.mapreduce.InputSplit> newInputSplits = this.getSplits(tac);
- org.apache.hadoop.mapred.InputSplit[] oldInputSplits = new org.apache.hadoop.mapred.InputSplit[newInputSplits.size()];
- for (int i = 0; i < newInputSplits.size(); i++)
- oldInputSplits[i] = (ColumnFamilySplit)newInputSplits.get(i);
- return oldInputSplits;
- }
-
public org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<ByteBuffer, IColumn>> getRecordReader(org.apache.hadoop.mapred.InputSplit split, JobConf jobConf, final Reporter reporter) throws IOException
{
TaskAttemptContext tac = new TaskAttemptContext(jobConf, TaskAttemptID.forName(jobConf.get(MAPRED_TASK_ID)))
@@ -357,5 +67,16 @@ public class ColumnFamilyInputFormat extends InputFormat<ByteBuffer, SortedMap<B
recordReader.initialize((org.apache.hadoop.mapreduce.InputSplit)split, tac);
return recordReader;
}
+
+ @Override
+ protected void validateConfiguration(Configuration conf)
+ {
+ super.validateConfiguration(conf);
+
+ if (ConfigHelper.getInputSlicePredicate(conf) == null)
+ {
+ throw new UnsupportedOperationException("you must set the predicate with setInputSlicePredicate");
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
index b3cd516..724ba7d 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
@@ -20,22 +20,9 @@ package org.apache.cassandra.hadoop;
import java.io.IOException;
import java.nio.ByteBuffer;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.auth.IAuthenticator;
import org.apache.cassandra.thrift.*;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.*;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TSocket;
-import org.apache.thrift.transport.TTransport;
-
-import javax.security.auth.login.LoginException;
/**
* The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
@@ -58,63 +45,11 @@ import javax.security.auth.login.LoginException;
* official by sending a batch mutate request to Cassandra.
* </p>
*/
-public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutation>>
- implements org.apache.hadoop.mapred.OutputFormat<ByteBuffer,List<Mutation>>
+public class ColumnFamilyOutputFormat extends AbstractColumnFamilyOutputFormat<ByteBuffer,List<Mutation>>
{
- public static final String BATCH_THRESHOLD = "mapreduce.output.columnfamilyoutputformat.batch.threshold";
- public static final String QUEUE_SIZE = "mapreduce.output.columnfamilyoutputformat.queue.size";
- private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyOutputFormat.class);
-
-
- /**
- * Check for validity of the output-specification for the job.
- *
- * @param context
- * information about the job
- * @throws IOException
- * when output should not be attempted
- */
- @Override
- public void checkOutputSpecs(JobContext context)
- {
- checkOutputSpecs(context.getConfiguration());
- }
-
- private void checkOutputSpecs(Configuration conf)
- {
- if (ConfigHelper.getOutputKeyspace(conf) == null)
- throw new UnsupportedOperationException("You must set the keyspace with setOutputKeyspace()");
- if (ConfigHelper.getOutputPartitioner(conf) == null)
- throw new UnsupportedOperationException("You must set the output partitioner to the one used by your Cassandra cluster");
- if (ConfigHelper.getOutputInitialAddress(conf) == null)
- throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node");
- }
-
- /**
- * The OutputCommitter for this format does not write any data to the DFS.
- *
- * @param context
- * the task context
- * @return an output committer
- * @throws IOException
- * @throws InterruptedException
- */
- @Override
- public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
- {
- return new NullOutputCommitter();
- }
-
- /** Fills the deprecated OutputFormat interface for streaming. */
- @Deprecated
- public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
- {
- checkOutputSpecs(job);
- }
-
/** Fills the deprecated OutputFormat interface for streaming. */
@Deprecated
- public ColumnFamilyRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress) throws IOException
+ public ColumnFamilyRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress)
{
return new ColumnFamilyRecordWriter(job, new Progressable(progress));
}
@@ -127,62 +62,8 @@ public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutat
* @return a {@link RecordWriter} to write the output for the job.
* @throws IOException
*/
- @Override
- public ColumnFamilyRecordWriter getRecordWriter(final TaskAttemptContext context) throws IOException, InterruptedException
+ public ColumnFamilyRecordWriter getRecordWriter(final TaskAttemptContext context) throws InterruptedException
{
return new ColumnFamilyRecordWriter(context);
}
-
- /**
- * Return a client based on the given socket that points to the configured
- * keyspace, and is logged in with the configured credentials.
- *
- * @param socket a socket pointing to a particular node, seed or otherwise
- * @param conf a job configuration
- * @return a cassandra client
- * @throws InvalidRequestException
- * @throws TException
- * @throws AuthenticationException
- * @throws AuthorizationException
- */
- public static Cassandra.Client createAuthenticatedClient(TSocket socket, Configuration conf)
- throws InvalidRequestException, TException, AuthenticationException, AuthorizationException, LoginException
- {
- logger.debug("Creating authenticated client for CF output format");
- TTransport transport = ConfigHelper.getOutputTransportFactory(conf).openTransport(socket, conf);
- TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
- Cassandra.Client client = new Cassandra.Client(binaryProtocol);
- client.set_keyspace(ConfigHelper.getOutputKeyspace(conf));
- if (ConfigHelper.getOutputKeyspaceUserName(conf) != null)
- {
- Map<String, String> creds = new HashMap<String, String>();
- creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getOutputKeyspaceUserName(conf));
- creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getOutputKeyspacePassword(conf));
- AuthenticationRequest authRequest = new AuthenticationRequest(creds);
- client.login(authRequest);
- }
- logger.debug("Authenticated client for CF output format created successfully");
- return client;
- }
-
- /**
- * An {@link OutputCommitter} that does nothing.
- */
- public static class NullOutputCommitter extends OutputCommitter
- {
- public void abortTask(TaskAttemptContext taskContext) { }
-
- public void cleanupJob(JobContext jobContext) { }
-
- public void commitTask(TaskAttemptContext taskContext) { }
-
- public boolean needsTaskCommit(TaskAttemptContext taskContext)
- {
- return false;
- }
-
- public void setupJob(JobContext jobContext) { }
-
- public void setupTask(TaskAttemptContext taskContext) { }
- }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index 30abdd5..daef8ec 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -24,11 +24,9 @@ import java.nio.ByteBuffer;
import java.util.*;
import com.google.common.collect.*;
-import org.apache.thrift.transport.TTransport;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.auth.IAuthenticator;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.db.IColumn;
import org.apache.cassandra.db.marshal.AbstractType;
@@ -43,7 +41,7 @@ import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.thrift.TException;
-import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap<ByteBuffer, IColumn>>
implements org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<ByteBuffer, IColumn>>
@@ -59,14 +57,14 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
private boolean isEmptyPredicate;
private int totalRowCount; // total number of rows to fetch
private int batchSize; // fetch this many per batch
- private String cfName;
private String keyspace;
- private TSocket socket;
+ private String cfName;
private Cassandra.Client client;
private ConsistencyLevel consistencyLevel;
private int keyBufferSize = 8192;
private List<IndexExpression> filter;
+
public ColumnFamilyRecordReader()
{
this(ColumnFamilyRecordReader.CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT);
@@ -80,11 +78,11 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
public void close()
{
- if (socket != null && socket.isOpen())
+ if (client != null)
{
- socket.close();
- socket = null;
- client = null;
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
}
}
@@ -139,36 +137,25 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
predicate = ConfigHelper.getInputSlicePredicate(conf);
boolean widerows = ConfigHelper.getInputIsWide(conf);
isEmptyPredicate = isEmptyPredicate(predicate);
- totalRowCount = (int) this.split.getLength();
+ totalRowCount = (this.split.getLength() < Long.MAX_VALUE)
+ ? (int) this.split.getLength()
+ : ConfigHelper.getInputSplitSize(conf);
batchSize = ConfigHelper.getRangeBatchSize(conf);
cfName = ConfigHelper.getInputColumnFamily(conf);
consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getReadConsistencyLevel(conf));
-
keyspace = ConfigHelper.getInputKeyspace(conf);
try
{
- // only need to connect once
- if (socket != null && socket.isOpen())
+ if (client != null)
return;
// create connection using thrift
String location = getLocation();
- socket = new TSocket(location, ConfigHelper.getInputRpcPort(conf));
- TTransport transport = ConfigHelper.getInputTransportFactory(conf).openTransport(socket, conf);
- TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
- client = new Cassandra.Client(binaryProtocol);
-
- // log in
- client.set_keyspace(keyspace);
- if (ConfigHelper.getInputKeyspaceUserName(conf) != null)
- {
- Map<String, String> creds = new HashMap<String, String>();
- creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf));
- creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf));
- AuthenticationRequest authRequest = new AuthenticationRequest(creds);
- client.login(authRequest);
- }
+
+ int port = ConfigHelper.getInputRpcPort(conf);
+ client = ColumnFamilyInputFormat.createAuthenticatedClient(location, port, conf);
+
}
catch (Exception e)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
index 909c291..50ec059 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
@@ -22,21 +22,14 @@ import java.io.IOException;
import java.net.InetAddress;
import java.nio.ByteBuffer;
import java.util.*;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-import org.apache.cassandra.client.RingCache;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.thrift.TException;
-import org.apache.thrift.transport.TSocket;
/**
@@ -52,33 +45,12 @@ import org.apache.thrift.transport.TSocket;
* </p>
*
* @see ColumnFamilyOutputFormat
- * @see OutputFormat
- *
*/
-final class ColumnFamilyRecordWriter extends RecordWriter<ByteBuffer,List<Mutation>>
-implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
+final class ColumnFamilyRecordWriter extends AbstractColumnFamilyRecordWriter<ByteBuffer, List<Mutation>>
{
- // The configuration this writer is associated with.
- private final Configuration conf;
-
- // The ring cache that describes the token ranges each node in the ring is
- // responsible for. This is what allows us to group the mutations by
- // the endpoints they should be targeted at. The targeted endpoint
- // essentially
- // acts as the primary replica for the rows being affected by the mutations.
- private final RingCache ringCache;
-
- // The number of mutations to buffer per endpoint
- private final int queueSize;
-
// handles for clients for each range running in the threadpool
- private final Map<Range,RangeClient> clients;
- private final long batchThreshold;
-
- private final ConsistencyLevel consistencyLevel;
- private Progressable progressable;
-
-
+ private final Map<Range, RangeClient> clients;
+
/**
* Upon construction, obtain the map that this writer will use to collect
* mutations, and the ring cache for the given keyspace.
@@ -86,28 +58,44 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
* @param context the task attempt context
* @throws IOException
*/
- ColumnFamilyRecordWriter(TaskAttemptContext context) throws IOException
+ ColumnFamilyRecordWriter(TaskAttemptContext context)
{
this(context.getConfiguration());
this.progressable = new Progressable(context);
}
- ColumnFamilyRecordWriter(Configuration conf, Progressable progressable) throws IOException
+ ColumnFamilyRecordWriter(Configuration conf, Progressable progressable)
{
this(conf);
this.progressable = progressable;
}
- ColumnFamilyRecordWriter(Configuration conf) throws IOException
+ ColumnFamilyRecordWriter(Configuration conf)
{
- this.conf = conf;
- this.ringCache = new RingCache(conf);
- this.queueSize = conf.getInt(ColumnFamilyOutputFormat.QUEUE_SIZE, 32 * FBUtilities.getAvailableProcessors());
- this.clients = new HashMap<Range,RangeClient>();
- batchThreshold = conf.getLong(ColumnFamilyOutputFormat.BATCH_THRESHOLD, 32);
- consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getWriteConsistencyLevel(conf));
+ super(conf);
+ this.clients = new HashMap<Range, RangeClient>();
}
-
+
+ @Override
+ public void close() throws IOException
+ {
+ // close all the clients before throwing anything
+ IOException clientException = null;
+ for (RangeClient client : clients.values())
+ {
+ try
+ {
+ client.close();
+ }
+ catch (IOException e)
+ {
+ clientException = e;
+ }
+ }
+ if (clientException != null)
+ throw clientException;
+ }
+
/**
* If the key is to be associated with a valid value, a mutation is created
* for it with the given column family and columns. In the event the value
@@ -143,124 +131,22 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
}
/**
- * Close this <code>RecordWriter</code> to future operations, but not before
- * flushing out the batched mutations.
- *
- * @param context the context of the task
- * @throws IOException
- */
- @Override
- public void close(TaskAttemptContext context) throws IOException, InterruptedException
- {
- close();
- }
-
- /** Fills the deprecated RecordWriter interface for streaming. */
- @Deprecated
- public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
- {
- close();
- }
-
- private void close() throws IOException
- {
- // close all the clients before throwing anything
- IOException clientException = null;
- for (RangeClient client : clients.values())
- {
- try
- {
- client.close();
- }
- catch (IOException e)
- {
- clientException = e;
- }
- }
- if (clientException != null)
- throw clientException;
- }
-
- /**
* A client that runs in a threadpool and connects to the list of endpoints for a particular
* range. Mutations for keys in that range are sent to this client via a queue.
*/
- public class RangeClient extends Thread
+ public class RangeClient extends AbstractRangeClient<Mutation>
{
- // The list of endpoints for this range
- private final List<InetAddress> endpoints;
- private final String columnFamily = ConfigHelper.getOutputColumnFamily(conf);
- // A bounded queue of incoming mutations for this range
- private final BlockingQueue<Pair<ByteBuffer, Mutation>> queue = new ArrayBlockingQueue<Pair<ByteBuffer,Mutation>>(queueSize);
-
- private volatile boolean run = true;
- // we want the caller to know if something went wrong, so we record any unrecoverable exception while writing
- // so we can throw it on the caller's stack when he calls put() again, or if there are no more put calls,
- // when the client is closed.
- private volatile IOException lastException;
-
- private Cassandra.Client thriftClient;
- private TSocket thriftSocket;
-
+ public final String columnFamily = ConfigHelper.getOutputColumnFamily(conf);
+
/**
* Constructs an {@link RangeClient} for the given endpoints.
* @param endpoints the possible endpoints to execute the mutations on
*/
public RangeClient(List<InetAddress> endpoints)
{
- super("client-" + endpoints);
- this.endpoints = endpoints;
+ super(endpoints);
}
-
- /**
- * enqueues the given value to Cassandra
- */
- public void put(Pair<ByteBuffer,Mutation> value) throws IOException
- {
- while (true)
- {
- if (lastException != null)
- throw lastException;
- try
- {
- if (queue.offer(value, 100, TimeUnit.MILLISECONDS))
- break;
- }
- catch (InterruptedException e)
- {
- throw new AssertionError(e);
- }
- }
- }
-
- public void close() throws IOException
- {
- // stop the run loop. this will result in closeInternal being called by the time join() finishes.
- run = false;
- interrupt();
- try
- {
- this.join();
- }
- catch (InterruptedException e)
- {
- throw new AssertionError(e);
- }
-
- if (lastException != null)
- throw lastException;
- }
-
- private void closeInternal()
- {
- if (thriftSocket != null)
- {
- thriftSocket.close();
- thriftSocket = null;
- thriftClient = null;
- }
- }
-
+
/**
* Loops collecting mutations from the queue and sending to Cassandra
*/
@@ -303,7 +189,7 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
// send the mutation to the last-used endpoint. first time through, this will NPE harmlessly.
try
{
- thriftClient.batch_mutate(batch, consistencyLevel);
+ client.batch_mutate(batch, consistencyLevel);
break;
}
catch (Exception e)
@@ -320,8 +206,9 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
try
{
InetAddress address = iter.next();
- thriftSocket = new TSocket(address.getHostName(), ConfigHelper.getOutputRpcPort(conf));
- thriftClient = ColumnFamilyOutputFormat.createAuthenticatedClient(thriftSocket, conf);
+ String host = address.getHostName();
+ int port = ConfigHelper.getOutputRpcPort(conf);
+ client = ColumnFamilyOutputFormat.createAuthenticatedClient(host, port, conf);
}
catch (Exception e)
{
@@ -337,11 +224,5 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
}
}
}
-
- @Override
- public String toString()
- {
- return "#<Client for " + endpoints.toString() + ">";
- }
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
index ed1f160..69c7ddb 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.mapreduce.InputSplit;
import java.io.DataInput;
import java.io.DataOutput;
+import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
@@ -79,7 +80,6 @@ public class ColumnFamilySplit extends InputSplit implements Writable, org.apach
{
out.writeUTF(startToken);
out.writeUTF(endToken);
- out.writeLong(length);
out.writeInt(dataNodes.length);
for (String endpoint : dataNodes)
{
@@ -91,8 +91,6 @@ public class ColumnFamilySplit extends InputSplit implements Writable, org.apach
{
startToken = in.readUTF();
endToken = in.readUTF();
- length = in.readLong();
-
int numOfEndpoints = in.readInt();
dataNodes = new String[numOfEndpoints];
for(int i = 0; i < numOfEndpoints; i++)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
index 0d12812..3dcfdd7 100644
--- a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
+++ b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
@@ -21,11 +21,9 @@ package org.apache.cassandra.hadoop;
*/
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
+import com.google.common.collect.Maps;
import org.apache.cassandra.io.compress.CompressionParameters;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -40,11 +38,8 @@ import org.apache.thrift.TBase;
import org.apache.thrift.TDeserializer;
import org.apache.thrift.TException;
import org.apache.thrift.TSerializer;
-import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-
-import javax.security.auth.login.LoginException;
public class ConfigHelper
@@ -74,6 +69,7 @@ public class ConfigHelper
private static final String WRITE_CONSISTENCY_LEVEL = "cassandra.consistencylevel.write";
private static final String OUTPUT_COMPRESSION_CLASS = "cassandra.output.compression.class";
private static final String OUTPUT_COMPRESSION_CHUNK_LENGTH = "cassandra.output.compression.length";
+
private static final String INPUT_TRANSPORT_FACTORY_CLASS = "cassandra.input.transport.factory.class";
private static final String OUTPUT_TRANSPORT_FACTORY_CLASS = "cassandra.output.transport.factory.class";
private static final String THRIFT_FRAMED_TRANSPORT_SIZE_IN_MB = "cassandra.thrift.framed.size_mb";
@@ -495,19 +491,13 @@ public class ConfigHelper
/**
* @param conf The configuration to use.
- * @return Value (converts MBs to Bytes) set by {@link setThriftFramedTransportSizeInMb(Configuration, int)} or default of 15MB
+ * @return Value (converts MBs to Bytes) set by {@link #setThriftFramedTransportSizeInMb(Configuration, int)} or default of 15MB
*/
public static int getThriftFramedTransportSize(Configuration conf)
{
return conf.getInt(THRIFT_FRAMED_TRANSPORT_SIZE_IN_MB, 15) * 1024 * 1024; // 15MB is default in Cassandra
}
- @Deprecated
- public static void setThriftMaxMessageLengthInMb(Configuration conf, int maxMessageSizeInMB)
- {
- // SEE CASSANDRA-5529
- }
-
public static CompressionParameters getOutputCompressionParamaters(Configuration conf)
{
if (getOutputCompressionClass(conf) == null)
@@ -567,48 +557,44 @@ public class ConfigHelper
{
try
{
- TSocket socket = new TSocket(host, port);
- TTransport transport = getInputTransportFactory(conf).openTransport(socket, conf);
+ TTransport transport = getClientTransportFactory(conf).openTransport(host, port, conf);
return new Cassandra.Client(new TBinaryProtocol(transport, true, true));
}
- catch (LoginException e)
- {
- throw new IOException("Unable to login to server " + host + ":" + port, e);
- }
- catch (TTransportException e)
+ catch (Exception e)
{
throw new IOException("Unable to connect to server " + host + ":" + port, e);
}
}
- public static ITransportFactory getInputTransportFactory(Configuration conf)
- {
- return getTransportFactory(conf.get(INPUT_TRANSPORT_FACTORY_CLASS, TFramedTransportFactory.class.getName()));
- }
-
- public static void setInputTransportFactoryClass(Configuration conf, String classname)
- {
- conf.set(INPUT_TRANSPORT_FACTORY_CLASS, classname);
- }
-
- public static ITransportFactory getOutputTransportFactory(Configuration conf)
- {
- return getTransportFactory(conf.get(OUTPUT_TRANSPORT_FACTORY_CLASS, TFramedTransportFactory.class.getName()));
- }
-
- public static void setOutputTransportFactoryClass(Configuration conf, String classname)
+ public static TClientTransportFactory getClientTransportFactory(Configuration conf)
{
- conf.set(OUTPUT_TRANSPORT_FACTORY_CLASS, classname);
+ String factoryClassName = conf.get(
+ TClientTransportFactory.PROPERTY_KEY,
+ TFramedTransportFactory.class.getName());
+ TClientTransportFactory factory = getClientTransportFactory(factoryClassName);
+ Map<String, String> options = getOptions(conf, factory.supportedOptions());
+ factory.setOptions(options);
+ return factory;
}
- private static ITransportFactory getTransportFactory(String factoryClassName) {
+ private static TClientTransportFactory getClientTransportFactory(String factoryClassName) {
try
{
- return (ITransportFactory) Class.forName(factoryClassName).newInstance();
+ return (TClientTransportFactory) Class.forName(factoryClassName).newInstance();
}
catch (Exception e)
{
throw new RuntimeException("Failed to instantiate transport factory:" + factoryClassName, e);
}
}
+ private static Map<String, String> getOptions(Configuration conf, Set<String> supportedOptions) {
+ Map<String, String> options = Maps.newHashMap();
+ for (String optionKey : supportedOptions)
+ {
+ String optionValue = conf.get(optionKey);
+ if (optionValue != null)
+ options.put(optionKey, optionValue);
+ }
+ return options;
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/Progressable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/Progressable.java b/src/java/org/apache/cassandra/hadoop/Progressable.java
index 091a828..ac253ef 100644
--- a/src/java/org/apache/cassandra/hadoop/Progressable.java
+++ b/src/java/org/apache/cassandra/hadoop/Progressable.java
@@ -29,12 +29,12 @@ public class Progressable
private TaskAttemptContext context;
private org.apache.hadoop.util.Progressable progressable;
- Progressable(TaskAttemptContext context)
+ public Progressable(TaskAttemptContext context)
{
this.context = context;
}
- Progressable(org.apache.hadoop.util.Progressable progressable)
+ public Progressable(org.apache.hadoop.util.Progressable progressable)
{
this.progressable = progressable;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java b/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java
new file mode 100644
index 0000000..66bcfdb
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CQLConfigHelper.java
@@ -0,0 +1,109 @@
+package org.apache.cassandra.hadoop.cql3;
+/*
+*
+* 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 org.apache.hadoop.conf.Configuration;
+
+public class CQLConfigHelper
+{
+ private static final String INPUT_CQL_COLUMNS_CONFIG = "cassandra.input.columnfamily.columns"; // separate by colon ,
+ private static final String INPUT_CQL_PAGE_ROW_SIZE_CONFIG = "cassandra.input.page.row.size";
+ private static final String INPUT_CQL_WHERE_CLAUSE_CONFIG = "cassandra.input.where.clause";
+ private static final String OUTPUT_CQL = "cassandra.output.cql";
+
+ /**
+ * Set the CQL columns for the input of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param columns
+ */
+ public static void setInputColumns(Configuration conf, String columns)
+ {
+ if (columns == null || columns.isEmpty())
+ return;
+
+ conf.set(INPUT_CQL_COLUMNS_CONFIG, columns);
+ }
+
+ /**
+ * Set the CQL query Limit for the input of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param cqlPageRowSize
+ */
+ public static void setInputCQLPageRowSize(Configuration conf, String cqlPageRowSize)
+ {
+ if (cqlPageRowSize == null)
+ {
+ throw new UnsupportedOperationException("cql page row size may not be null");
+ }
+
+ conf.set(INPUT_CQL_PAGE_ROW_SIZE_CONFIG, cqlPageRowSize);
+ }
+
+ /**
+ * Set the CQL user defined where clauses for the input of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param clauses
+ */
+ public static void setInputWhereClauses(Configuration conf, String clauses)
+ {
+ if (clauses == null || clauses.isEmpty())
+ return;
+
+ conf.set(INPUT_CQL_WHERE_CLAUSE_CONFIG, clauses);
+ }
+
+ /**
+ * Set the CQL prepared statement for the output of this job.
+ *
+ * @param conf Job configuration you are about to run
+ * @param cql
+ */
+ public static void setOutputCql(Configuration conf, String cql)
+ {
+ if (cql == null || cql.isEmpty())
+ return;
+
+ conf.set(OUTPUT_CQL, cql);
+ }
+
+
+ public static String getInputcolumns(Configuration conf)
+ {
+ return conf.get(INPUT_CQL_COLUMNS_CONFIG);
+ }
+
+ public static String getInputPageRowSize(Configuration conf)
+ {
+ return conf.get(INPUT_CQL_PAGE_ROW_SIZE_CONFIG);
+ }
+
+ public static String getInputWhereClauses(Configuration conf)
+ {
+ return conf.get(INPUT_CQL_WHERE_CLAUSE_CONFIG);
+ }
+
+ public static String getOutputCql(Configuration conf)
+ {
+ return conf.get(OUTPUT_CQL);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java
new file mode 100644
index 0000000..525ed89
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyInputFormat.java
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import org.apache.cassandra.hadoop.AbstractColumnFamilyInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+
+/**
+ * Hadoop InputFormat allowing map/reduce against Cassandra rows within one ColumnFamily.
+ *
+ * At minimum, you need to set the KS and CF in your Hadoop job Configuration.
+ * The ConfigHelper class is provided to make this
+ * simple:
+ * ConfigHelper.setInputColumnFamily
+ *
+ * You can also configure the number of rows per InputSplit with
+ * ConfigHelper.setInputSplitSize. The default split size is 64k rows.
+ * the number of CQL rows per page
+ *
+ * the number of CQL rows per page
+ * CQLConfigHelper.setInputCQLPageRowSize. The default page row size is 1000. You
+ * should set it to "as big as possible, but no bigger." It set the LIMIT for the CQL
+ * query, so you need set it big enough to minimize the network overhead, and also
+ * not too big to avoid out of memory issue.
+ *
+ * the column names of the select CQL query. The default is all columns
+ * CQLConfigHelper.setInputColumns
+ *
+ * the user defined the where clause
+ * CQLConfigHelper.setInputWhereClauses. The default is no user defined where clause
+ */
+public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<Map<String, ByteBuffer>, Map<String, ByteBuffer>>
+{
+ public RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>> getRecordReader(InputSplit split, JobConf jobConf, final Reporter reporter)
+ throws IOException
+ {
+ TaskAttemptContext tac = new TaskAttemptContext(jobConf, TaskAttemptID.forName(jobConf.get(MAPRED_TASK_ID)))
+ {
+ @Override
+ public void progress()
+ {
+ reporter.progress();
+ }
+ };
+
+ ColumnFamilyRecordReader recordReader = new ColumnFamilyRecordReader();
+ recordReader.initialize((org.apache.hadoop.mapreduce.InputSplit)split, tac);
+ return recordReader;
+ }
+
+ @Override
+ public org.apache.hadoop.mapreduce.RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>> createRecordReader(
+ org.apache.hadoop.mapreduce.InputSplit arg0, TaskAttemptContext arg1) throws IOException,
+ InterruptedException
+ {
+ return new ColumnFamilyRecordReader();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java
new file mode 100644
index 0000000..3f6e2af
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyOutputFormat.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.hadoop.AbstractColumnFamilyOutputFormat;
+import org.apache.cassandra.hadoop.ColumnFamilyInputFormat;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.Progressable;
+import org.apache.hadoop.mapreduce.*;
+
+/**
+ * The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
+ * OutputFormat that allows reduce tasks to store keys (and corresponding
+ * binded variable values) as CQL rows (and respective columns) in a given
+ * ColumnFamily.
+ *
+ * <p>
+ * As is the case with the {@link ColumnFamilyInputFormat}, you need to set the
+ * prepared statement in your
+ * Hadoop job Configuration. The {@link CQLConfigHelper} class, through its
+ * {@link ConfigHelper#setOutputPreparedStatement} method, is provided to make this
+ * simple.
+ * you need to set the Keyspace. The {@link ConfigHelper} class, through its
+ * {@link ConfigHelper#setOutputColumnFamily} method, is provided to make this
+ * simple.
+ * </p>
+ *
+ * <p>
+ * For the sake of performance, this class employs a lazy write-back caching
+ * mechanism, where its record writer prepared statement binded variable values
+ * created based on the reduce's inputs (in a task-specific map), and periodically
+ * makes the changes official by sending a execution of prepared statement request
+ * to Cassandra.
+ * </p>
+ */
+public class ColumnFamilyOutputFormat extends AbstractColumnFamilyOutputFormat<Map<String, ByteBuffer>, List<ByteBuffer>>
+{
+ /** Fills the deprecated OutputFormat interface for streaming. */
+ @Deprecated
+ public ColumnFamilyRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress) throws IOException
+ {
+ return new ColumnFamilyRecordWriter(job, new Progressable(progress));
+ }
+
+ /**
+ * Get the {@link RecordWriter} for the given task.
+ *
+ * @param context
+ * the information about the current task.
+ * @return a {@link RecordWriter} to write the output for the job.
+ * @throws IOException
+ */
+ public ColumnFamilyRecordWriter getRecordWriter(final TaskAttemptContext context) throws IOException, InterruptedException
+ {
+ return new ColumnFamilyRecordWriter(context);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/56e0ad1b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java
new file mode 100644
index 0000000..03d7af5
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/cql3/ColumnFamilyRecordReader.java
@@ -0,0 +1,763 @@
+/*
+ * 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.
+ */
+package org.apache.cassandra.hadoop.cql3;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.util.*;
+
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.hadoop.ColumnFamilySplit;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+
+/**
+ * Hadoop RecordReader read the values return from the CQL query
+ * It use CQL key range query to page through the wide rows.
+ * <p/>
+ * Return List<IColumn> as keys columns
+ * <p/>
+ * Map<ByteBuffer, IColumn> as column name to columns mappings
+ */
+public class ColumnFamilyRecordReader extends RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>>
+ implements org.apache.hadoop.mapred.RecordReader<Map<String, ByteBuffer>, Map<String, ByteBuffer>>
+{
+ private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyRecordReader.class);
+
+ public static final int DEFAULT_CQL_PAGE_LIMIT = 1000; // TODO: find the number large enough but not OOM
+
+ private ColumnFamilySplit split;
+ private RowIterator rowIterator;
+
+ private Pair<Map<String, ByteBuffer>, Map<String, ByteBuffer>> currentRow;
+ private int totalRowCount; // total number of rows to fetch
+ private String keyspace;
+ private String cfName;
+ private Cassandra.Client client;
+ private ConsistencyLevel consistencyLevel;
+
+ // partition keys -- key aliases
+ private List<BoundColumn> partitionBoundColumns = new ArrayList<BoundColumn>();
+
+ // cluster keys -- column aliases
+ private List<BoundColumn> clusterColumns = new ArrayList<BoundColumn>();
+
+ // map prepared query type to item id
+ private Map<Integer, Integer> preparedQueryIds = new HashMap<Integer, Integer>();
+
+ // cql query select columns
+ private String columns;
+
+ // the number of cql rows per page
+ private int pageRowSize;
+
+ // user defined where clauses
+ private String userDefinedWhereClauses;
+
+ private IPartitioner partitioner;
+
+ private AbstractType<?> keyValidator;
+
+ public ColumnFamilyRecordReader()
+ {
+ super();
+ }
+
+ public void initialize(InputSplit split, TaskAttemptContext context) throws IOException
+ {
+ this.split = (ColumnFamilySplit) split;
+ Configuration conf = context.getConfiguration();
+ totalRowCount = (this.split.getLength() < Long.MAX_VALUE)
+ ? (int) this.split.getLength()
+ : ConfigHelper.getInputSplitSize(conf);
+ cfName = ConfigHelper.getInputColumnFamily(conf);
+ consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getReadConsistencyLevel(conf));
+ keyspace = ConfigHelper.getInputKeyspace(conf);
+ columns = CQLConfigHelper.getInputcolumns(conf);
+ userDefinedWhereClauses = CQLConfigHelper.getInputWhereClauses(conf);
+
+ try
+ {
+ pageRowSize = Integer.parseInt(CQLConfigHelper.getInputPageRowSize(conf));
+ }
+ catch (NumberFormatException e)
+ {
+ pageRowSize = DEFAULT_CQL_PAGE_LIMIT;
+ }
+
+ partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
+
+ try
+ {
+ if (client != null)
+ return;
+
+ // create connection using thrift
+ String location = getLocation();
+
+ int port = ConfigHelper.getInputRpcPort(conf);
+ client = ColumnFamilyInputFormat.createAuthenticatedClient(location, port, conf);
+
+ // retrieve partition keys and cluster keys from system.schema_columnfamilies table
+ retrieveKeys();
+
+ client.set_keyspace(keyspace);
+ }
+ catch (Exception e)
+ {
+ throw new RuntimeException(e);
+ }
+
+ rowIterator = new RowIterator();
+
+ logger.debug("created {}", rowIterator);
+ }
+
+ public void close()
+ {
+ if (client != null)
+ {
+ TTransport transport = client.getOutputProtocol().getTransport();
+ if (transport.isOpen())
+ transport.close();
+ client = null;
+ }
+ }
+
+ public Map<String, ByteBuffer> getCurrentKey()
+ {
+ return currentRow.left;
+ }
+
+ public Map<String, ByteBuffer> getCurrentValue()
+ {
+ return currentRow.right;
+ }
+
+ public float getProgress()
+ {
+ if (!rowIterator.hasNext())
+ return 1.0F;
+
+ // the progress is likely to be reported slightly off the actual but close enough
+ float progress = ((float) rowIterator.totalRead / totalRowCount);
+ return progress > 1.0F ? 1.0F : progress;
+ }
+
+ public boolean nextKeyValue() throws IOException
+ {
+ if (!rowIterator.hasNext())
+ {
+ logger.debug("Finished scanning " + rowIterator.totalRead + " rows (estimate was: " + totalRowCount + ")");
+ return false;
+ }
+
+ try
+ {
+ currentRow = rowIterator.next();
+ }
+ catch (Exception e)
+ {
+ // throw it as IOException, so client can catch it and handle it at client side
+ IOException ioe = new IOException(e.getMessage());
+ ioe.initCause(ioe.getCause());
+ throw ioe;
+ }
+ return true;
+ }
+
+ // we don't use endpointsnitch since we are trying to support hadoop nodes that are
+ // not necessarily on Cassandra machines, too. This should be adequate for single-DC clusters, at least.
+ private String getLocation()
+ {
+ Collection<InetAddress> localAddresses = FBUtilities.getAllLocalAddresses();
+
+ for (InetAddress address : localAddresses)
+ {
+ for (String location : split.getLocations())
+ {
+ InetAddress locationAddress;
+ try
+ {
+ locationAddress = InetAddress.getByName(location);
+ }
+ catch (UnknownHostException e)
+ {
+ throw new AssertionError(e);
+ }
+ if (address.equals(locationAddress))
+ {
+ return location;
+ }
+ }
+ }
+ return split.getLocations()[0];
+ }
+
+ // Because the old Hadoop API wants us to write to the key and value
+ // and the new asks for them, we need to copy the output of the new API
+ // to the old. Thus, expect a small performance hit.
+ // And obviously this wouldn't work for wide rows. But since ColumnFamilyInputFormat
+ // and ColumnFamilyRecordReader don't support them, it should be fine for now.
+ public boolean next(Map<String, ByteBuffer> keys, Map<String, ByteBuffer> value) throws IOException
+ {
+ if (nextKeyValue())
+ {
+ value.clear();
+ value.putAll(getCurrentValue());
+
+ keys.clear();
+ keys.putAll(getCurrentKey());
+
+ return true;
+ }
+ return false;
+ }
+
+ public long getPos() throws IOException
+ {
+ return (long) rowIterator.totalRead;
+ }
+
+ public Map<String, ByteBuffer> createKey()
+ {
+ return new LinkedHashMap<String, ByteBuffer>();
+ }
+
+ public Map<String, ByteBuffer> createValue()
+ {
+ return new LinkedHashMap<String, ByteBuffer>();
+ }
+
+ /** CQL row iterator */
+ private class RowIterator extends AbstractIterator<Pair<Map<String, ByteBuffer>, Map<String, ByteBuffer>>>
+ {
+ protected int totalRead = 0; // total number of cf rows read
+ protected Iterator<CqlRow> rows;
+ private int pageRows = 0; // the number of cql rows read of this page
+ private String previousRowKey = null; // previous CF row key
+ private String partitionKeyString; // keys in <key1>, <key2>, <key3> string format
+ private String partitionKeyMarkers; // question marks in ? , ? , ? format which matches the number of keys
+
+ public RowIterator()
+ {
+ // initial page
+ executeQuery();
+ }
+
+ protected Pair<Map<String, ByteBuffer>, Map<String, ByteBuffer>> computeNext()
+ {
+ if (rows == null)
+ return endOfData();
+
+ int index = -2;
+ //check there are more page to read
+ while (!rows.hasNext())
+ {
+ // no more data
+ if (index == -1 || emptyPartitionKeyValues())
+ {
+ logger.debug("no more data.");
+ return endOfData();
+ }
+
+ index = setTailNull(clusterColumns);
+ logger.debug("set tail to null, index: " + index);
+ executeQuery();
+ pageRows = 0;
+
+ if (rows == null || !rows.hasNext() && index < 0)
+ {
+ logger.debug("no more data.");
+ return endOfData();
+ }
+ }
+
+ Map<String, ByteBuffer> valueColumns = createValue();
+ Map<String, ByteBuffer> keyColumns = createKey();
+ int i = 0;
+ CqlRow row = rows.next();
+ for (Column column : row.columns)
+ {
+ String columnName = stringValue(ByteBuffer.wrap(column.getName()));
+ logger.debug("column: " + columnName);
+
+ if (i < partitionBoundColumns.size() + clusterColumns.size())
+ keyColumns.put(stringValue(column.name), column.value);
+ else
+ valueColumns.put(stringValue(column.name), column.value);
+
+ i++;
+ }
+
+ // increase total CQL row read for this page
+ pageRows++;
+
+ // increase total CF row read
+ if (newRow(keyColumns, previousRowKey))
+ totalRead++;
+
+ // read full page
+ if (pageRows >= pageRowSize || !rows.hasNext())
+ {
+ Iterator<String> newKeys = keyColumns.keySet().iterator();
+ for (BoundColumn column : partitionBoundColumns)
+ column.value = keyColumns.get(newKeys.next());
+
+ for (BoundColumn column : clusterColumns)
+ column.value = keyColumns.get(newKeys.next());
+
+ executeQuery();
+ pageRows = 0;
+ }
+
+ return Pair.create(keyColumns, valueColumns);
+ }
+
+ /** check whether start to read a new CF row by comparing the partition keys */
+ private boolean newRow(Map<String, ByteBuffer> keyColumns, String previousRowKey)
+ {
+ if (keyColumns.isEmpty())
+ return false;
+
+ String rowKey = "";
+ if (keyColumns.size() == 1)
+ {
+ rowKey = partitionBoundColumns.get(0).validator.getString(keyColumns.get(partitionBoundColumns.get(0).name));
+ }
+ else
+ {
+ Iterator<ByteBuffer> iter = keyColumns.values().iterator();
+ for (BoundColumn column : partitionBoundColumns)
+ rowKey = rowKey + column.validator.getString(ByteBufferUtil.clone(iter.next())) + ":";
+ }
+
+ logger.debug("previous RowKey: " + previousRowKey + ", new row key: " + rowKey);
+ if (previousRowKey == null)
+ {
+ this.previousRowKey = rowKey;
+ return true;
+ }
+
+ if (rowKey.equals(previousRowKey))
+ return false;
+
+ this.previousRowKey = rowKey;
+ return true;
+ }
+
+ /** set the last non-null key value to null, and return the previous index */
+ private int setTailNull(List<BoundColumn> values)
+ {
+ if (values.isEmpty())
+ return -1;
+
+ Iterator<BoundColumn> iterator = values.iterator();
+ int previousIndex = -1;
+ BoundColumn current;
+ while (iterator.hasNext())
+ {
+ current = iterator.next();
+ if (current.value == null)
+ {
+ int index = previousIndex > 0 ? previousIndex : 0;
+ BoundColumn column = values.get(index);
+ logger.debug("set key " + column.name + " value to null");
+ column.value = null;
+ return previousIndex - 1;
+ }
+
+ previousIndex++;
+ }
+
+ BoundColumn column = values.get(previousIndex);
+ logger.debug("set key " + column.name + " value to null");
+ column.value = null;
+ return previousIndex - 1;
+ }
+
+ /** compose the prepared query, pair.left is query id, pair.right is query */
+ private Pair<Integer, String> composeQuery(String columns)
+ {
+ Pair<Integer, String> clause = whereClause();
+ if (columns == null)
+ {
+ columns = "*";
+ }
+ else
+ {
+ // add keys in the front in order
+ String partitionKey = keyString(partitionBoundColumns);
+ String clusterKey = keyString(clusterColumns);
+
+ columns = withoutKeyColumns(columns);
+ columns = (clusterKey == null || "".equals(clusterKey))
+ ? partitionKey + "," + columns
+ : partitionKey + "," + clusterKey + "," + columns;
+ }
+
+ return Pair.create(clause.left,
+ "SELECT " + columns
+ + " FROM " + cfName
+ + clause.right
+ + (userDefinedWhereClauses == null ? "" : " AND " + userDefinedWhereClauses)
+ + " LIMIT " + pageRowSize
+ + " ALLOW FILTERING");
+ }
+
+
+ /** remove key columns from the column string */
+ private String withoutKeyColumns(String columnString)
+ {
+ Set<String> keyNames = new HashSet<String>();
+ for (BoundColumn column : Iterables.concat(partitionBoundColumns, clusterColumns))
+ keyNames.add(column.name);
+
+ String[] columns = columnString.split(",");
+ String result = null;
+ for (String column : columns)
+ {
+ String trimmed = column.trim();
+ if (keyNames.contains(trimmed))
+ continue;
+
+ result = result == null ? trimmed : result + "," + trimmed;
+ }
+ return result;
+ }
+
+ /** compose the where clause */
+ private Pair<Integer, String> whereClause()
+ {
+ if (partitionKeyString == null)
+ partitionKeyString = keyString(partitionBoundColumns);
+
+ if (partitionKeyMarkers == null)
+ partitionKeyMarkers = partitionKeyMarkers();
+ // initial query token(k) >= start_token and token(k) <= end_token
+ if (emptyPartitionKeyValues())
+ return Pair.create(0, " WHERE token(" + partitionKeyString + ") > ? AND token(" + partitionKeyString + ") <= ?");
+
+ // query token(k) > token(pre_partition_key) and token(k) <= end_token
+ if (clusterColumns.size() == 0 || clusterColumns.get(0).value == null)
+ return Pair.create(1,
+ " WHERE token(" + partitionKeyString + ") > token(" + partitionKeyMarkers + ") "
+ + " AND token(" + partitionKeyString + ") <= ?");
+
+ // query token(k) = token(pre_partition_key) and m = pre_cluster_key_m and n > pre_cluster_key_n
+ Pair<Integer, String> clause = whereClause(clusterColumns, 0);
+ return Pair.create(clause.left,
+ " WHERE token(" + partitionKeyString + ") = token(" + partitionKeyMarkers + ") " + clause.right);
+ }
+
+ /** recursively compose the where clause */
+ private Pair<Integer, String> whereClause(List<BoundColumn> column, int position)
+ {
+ if (position == column.size() - 1 || column.get(position + 1).value == null)
+ return Pair.create(position + 2, " AND " + column.get(position).name + " > ? ");
+
+ Pair<Integer, String> clause = whereClause(column, position + 1);
+ return Pair.create(clause.left, " AND " + column.get(position).name + " = ? " + clause.right);
+ }
+
+ /** check whether all key values are null */
+ private boolean emptyPartitionKeyValues()
+ {
+ for (BoundColumn column : partitionBoundColumns)
+ {
+ if (column.value != null)
+ return false;
+ }
+ return true;
+ }
+
+ /** compose the partition key string in format of <key1>, <key2>, <key3> */
+ private String keyString(List<BoundColumn> columns)
+ {
+ String result = null;
+ for (BoundColumn column : columns)
+ result = result == null ? column.name : result + "," + column.name;
+
+ return result == null ? "" : result;
+ }
+
+ /** compose the question marks for partition key string in format of ?, ? , ? */
+ private String partitionKeyMarkers()
+ {
+ String result = null;
+ for (BoundColumn column : partitionBoundColumns)
+ result = result == null ? "?" : result + ",?";
+
+ return result;
+ }
+
+ /** compose the query binding variables, pair.left is query id, pair.right is the binding variables */
+ private Pair<Integer, List<ByteBuffer>> preparedQueryBindValues()
+ {
+ List<ByteBuffer> values = new LinkedList<ByteBuffer>();
+
+ // initial query token(k) >= start_token and token(k) <= end_token
+ if (emptyPartitionKeyValues())
+ {
+ values.add(partitioner.getTokenValidator().fromString(split.getStartToken()));
+ values.add(partitioner.getTokenValidator().fromString(split.getEndToken()));
+ return Pair.create(0, values);
+ }
+ else
+ {
+ for (BoundColumn partitionBoundColumn1 : partitionBoundColumns)
+ values.add(partitionBoundColumn1.value);
+
+ if (clusterColumns.size() == 0 || clusterColumns.get(0).value == null)
+ {
+ // query token(k) > token(pre_partition_key) and token(k) <= end_token
+ values.add(partitioner.getTokenValidator().fromString(split.getEndToken()));
+ return Pair.create(1, values);
+ }
+ else
+ {
+ // query token(k) = token(pre_partition_key) and m = pre_cluster_key_m and n > pre_cluster_key_n
+ int type = preparedQueryBindValues(clusterColumns, 0, values);
+ return Pair.create(type, values);
+ }
+ }
+ }
+
+ /** recursively compose the query binding variables */
+ private int preparedQueryBindValues(List<BoundColumn> column, int position, List<ByteBuffer> bindValues)
+ {
+ if (position == column.size() - 1 || column.get(position + 1).value == null)
+ {
+ bindValues.add(column.get(position).value);
+ return position + 2;
+ }
+ else
+ {
+ bindValues.add(column.get(position).value);
+ return preparedQueryBindValues(column, position + 1, bindValues);
+ }
+ }
+
+ /** get the prepared query item Id */
+ private int prepareQuery(int type) throws InvalidRequestException, TException
+ {
+ Integer itemId = preparedQueryIds.get(type);
+ if (itemId != null)
+ return itemId;
+
+ Pair<Integer, String> query = null;
+ query = composeQuery(columns);
+ logger.debug("type:" + query.left + ", query: " + query.right);
+ CqlPreparedResult cqlPreparedResult = client.prepare_cql3_query(ByteBufferUtil.bytes(query.right), Compression.NONE);
+ preparedQueryIds.put(query.left, cqlPreparedResult.itemId);
+ return cqlPreparedResult.itemId;
+ }
+
+ /** execute the prepared query */
+ private void executeQuery()
+ {
+ Pair<Integer, List<ByteBuffer>> bindValues = preparedQueryBindValues();
+ logger.debug("query type: " + bindValues.left);
+
+ // check whether it reach end of range for type 1 query CASSANDRA-5573
+ if (bindValues.left == 1 && reachEndRange())
+ {
+ rows = null;
+ return;
+ }
+
+ int retries = 0;
+ // only try three times for TimedOutException and UnavailableException
+ while (retries < 3)
+ {
+ try
+ {
+ CqlResult cqlResult = client.execute_prepared_cql3_query(prepareQuery(bindValues.left), bindValues.right, consistencyLevel);
+ if (cqlResult != null && cqlResult.rows != null)
+ rows = cqlResult.rows.iterator();
+ return;
+ }
+ catch (TimedOutException e)
+ {
+ retries++;
+ if (retries >= 3)
+ {
+ rows = null;
+ RuntimeException rte = new RuntimeException(e.getMessage());
+ rte.initCause(e);
+ throw rte;
+ }
+ }
+ catch (UnavailableException e)
+ {
+ retries++;
+ if (retries >= 3)
+ {
+ rows = null;
+ RuntimeException rte = new RuntimeException(e.getMessage());
+ rte.initCause(e);
+ throw rte;
+ }
+ }
+ catch (Exception e)
+ {
+ rows = null;
+ RuntimeException rte = new RuntimeException(e.getMessage());
+ rte.initCause(e);
+ throw rte;
+ }
+ }
+ }
+ }
+
+ /** retrieve the partition keys and cluster keys from system.schema_columnfamilies table */
+ private void retrieveKeys() throws Exception
+ {
+ String query = "select key_aliases," +
+ "column_aliases, " +
+ "key_validator, " +
+ "comparator " +
+ "from system.schema_columnfamilies " +
+ "where keyspace_name='%s' and columnfamily_name='%s'";
+ String formatted = String.format(query, keyspace, cfName);
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(formatted), Compression.NONE, ConsistencyLevel.ONE);
+
+ CqlRow cqlRow = result.rows.get(0);
+ String keyString = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(0).getValue()));
+ logger.debug("partition keys: " + keyString);
+ List<String> keys = FBUtilities.fromJsonList(keyString);
+
+ for (String key : keys)
+ partitionBoundColumns.add(new BoundColumn(key));
+
+ keyString = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(1).getValue()));
+ logger.debug("cluster columns: " + keyString);
+ keys = FBUtilities.fromJsonList(keyString);
+
+ for (String key : keys)
+ clusterColumns.add(new BoundColumn(key));
+
+ Column rawKeyValidator = cqlRow.columns.get(2);
+ String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));
+ logger.debug("row key validator: " + validator);
+ keyValidator = parseType(validator);
+
+ if (keyValidator instanceof CompositeType)
+ {
+ List<AbstractType<?>> types = ((CompositeType) keyValidator).types;
+ for (int i = 0; i < partitionBoundColumns.size(); i++)
+ partitionBoundColumns.get(i).validator = types.get(i);
+ }
+ else
+ {
+ partitionBoundColumns.get(0).validator = keyValidator;
+ }
+ }
+
+ /** check whether current row is at the end of range */
+ private boolean reachEndRange()
+ {
+ // current row key
+ ByteBuffer rowKey;
+ if (keyValidator instanceof CompositeType)
+ {
+ ByteBuffer[] keys = new ByteBuffer[partitionBoundColumns.size()];
+ for (int i = 0; i < partitionBoundColumns.size(); i++)
+ keys[i] = partitionBoundColumns.get(i).value.duplicate();
+
+ rowKey = ((CompositeType) keyValidator).build(keys);
+ }
+ else
+ {
+ rowKey = partitionBoundColumns.get(0).value;
+ }
+
+ String endToken = split.getEndToken();
+ String currentToken = partitioner.getToken(rowKey).toString();
+ logger.debug("End token: " + endToken + ", current token: " + currentToken);
+
+ return endToken.equals(currentToken);
+ }
+
+ private static AbstractType<?> parseType(String type) throws IOException
+ {
+ try
+ {
+ // always treat counters like longs, specifically CCT.compose is not what we need
+ if (type != null && type.equals("org.apache.cassandra.db.marshal.CounterColumnType"))
+ return LongType.instance;
+ return TypeParser.parse(type);
+ }
+ catch (ConfigurationException e)
+ {
+ throw new IOException(e);
+ }
+ catch (SyntaxException e)
+ {
+ throw new IOException(e);
+ }
+ }
+
+ private class BoundColumn
+ {
+ final String name;
+ ByteBuffer value;
+ AbstractType<?> validator;
+
+ public BoundColumn(String name)
+ {
+ this.name = name;
+ }
+ }
+
+ /** get string from a ByteBuffer, catch the exception and throw it as runtime exception*/
+ private static String stringValue(ByteBuffer value)
+ {
+ try
+ {
+ return ByteBufferUtil.string(value);
+ }
+ catch (CharacterCodingException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+}