You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2014/01/17 07:36:44 UTC
svn commit: r1559022 [1/2] - in /pig/trunk: ./ ivy/
src/org/apache/pig/backend/hadoop/accumulo/ test/
test/org/apache/pig/backend/hadoop/accumulo/
Author: daijy
Date: Fri Jan 17 06:36:44 2014
New Revision: 1559022
URL: http://svn.apache.org/r1559022
Log:
PIG-3573: Provide StoreFunc and LoadFunc for Accumulo
Added:
pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/
pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AbstractAccumuloStorage.java
pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloBinaryConverter.java
pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloStorage.java
pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloStorageOptions.java
pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Column.java
pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/FixedByteArrayOutputStream.java
pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Utils.java
pig/trunk/test/org/apache/pig/backend/hadoop/accumulo/
pig/trunk/test/org/apache/pig/backend/hadoop/accumulo/TestAbstractAccumuloStorage.java
pig/trunk/test/org/apache/pig/backend/hadoop/accumulo/TestAccumuloBinaryConverter.java
pig/trunk/test/org/apache/pig/backend/hadoop/accumulo/TestAccumuloColumns.java
pig/trunk/test/org/apache/pig/backend/hadoop/accumulo/TestAccumuloPigCluster.java
pig/trunk/test/org/apache/pig/backend/hadoop/accumulo/TestAccumuloStorage.java
pig/trunk/test/org/apache/pig/backend/hadoop/accumulo/TestAccumuloStorageConfiguration.java
pig/trunk/test/org/apache/pig/backend/hadoop/accumulo/TestAccumuloStorageOptions.java
Modified:
pig/trunk/CHANGES.txt
pig/trunk/ivy.xml
pig/trunk/ivy/libraries.properties
pig/trunk/test/excluded-tests-23
Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1559022&r1=1559021&r2=1559022&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Fri Jan 17 06:36:44 2014
@@ -28,6 +28,8 @@ PIG-3419: Pluggable Execution Engine (ac
IMPROVEMENTS
+PIG-3573: Provide StoreFunc and LoadFunc for Accumulo (elserj via daijy)
+
PIG-3653: Add support for pre-deployed jars (tmwoodruff via daijy)
PIG-3645: Move FileLocalizer.setR() calls to unit tests (cheolsoo)
Modified: pig/trunk/ivy.xml
URL: http://svn.apache.org/viewvc/pig/trunk/ivy.xml?rev=1559022&r1=1559021&r2=1559022&view=diff
==============================================================================
--- pig/trunk/ivy.xml (original)
+++ pig/trunk/ivy.xml Fri Jan 17 06:36:44 2014
@@ -358,6 +358,38 @@
<dependency org="com.yammer.metrics" name="metrics-core" rev="${metrics-core.version}"
conf="test->default"/>
+ <!-- Dependency for Accumulo{Input,Output}Format -->
+ <dependency org="org.apache.accumulo" name="accumulo-core" rev="${accumulo15.version}" conf="compile->default">
+ <exclude org="com.google.guava" module="guava"/>
+ <exclude org="commons-codec" module="commons-codec"/>
+ <exclude org="commons-collections" module="commons-collections"/>
+ <exclude org="commons-configuration" module="commons-configuration"/>
+ <exclude org="commons-io" module="commons-io"/>
+ <exclude org="commons-lang" module="commons-lang"/>
+ <exclude org="commons-logging" module="commons-logging"/>
+ <exclude org="log4j" module="log4j"/>
+ <exclude org="org.apache.hadoop" module="hadoop-client"/>
+ <exclude org="org.apache.zookeeper" module="zookeeper"/>
+ <exclude org="org.slf4j" module="slf4j-api"/>
+ <exclude org="org.slf4j" module="slf4j-log4j12"/>
+ </dependency>
+
+ <!-- Used for 'functional' Accumulo tests -->
+ <dependency org="org.apache.accumulo" name="accumulo-minicluster" rev="${accumulo15.version}" conf="compile->default">
+ <exclude org="com.google.guava" module="guava"/>
+ <exclude org="commons-codec" module="commons-codec"/>
+ <exclude org="commons-collections" module="commons-collections"/>
+ <exclude org="commons-configuration" module="commons-configuration"/>
+ <exclude org="commons-io" module="commons-io"/>
+ <exclude org="commons-lang" module="commons-lang"/>
+ <exclude org="commons-logging" module="commons-logging"/>
+ <exclude org="log4j" module="log4j"/>
+ <exclude org="org.apache.hadoop" module="hadoop-client"/>
+ <exclude org="org.apache.zookeeper" module="zookeeper"/>
+ <exclude org="org.slf4j" module="slf4j-api"/>
+ <exclude org="org.slf4j" module="slf4j-log4j12"/>
+ </dependency>
+
<!-- for piggybank -->
<dependency org="hsqldb" name="hsqldb" rev="${hsqldb.version}"
conf="test->default" />
Modified: pig/trunk/ivy/libraries.properties
URL: http://svn.apache.org/viewvc/pig/trunk/ivy/libraries.properties?rev=1559022&r1=1559021&r2=1559022&view=diff
==============================================================================
--- pig/trunk/ivy/libraries.properties (original)
+++ pig/trunk/ivy/libraries.properties Fri Jan 17 06:36:44 2014
@@ -14,6 +14,7 @@
#It drives ivy and the generation of a maven POM
#These are the versions of our dependencies (in alphabetical order)
+accumulo15.version=1.5.0
apacheant.version=1.7.1
apacherat.version=0.8
automaton.version=1.11-8
@@ -74,7 +75,7 @@ slf4j-log4j12.version=1.6.1
xerces.version=2.10.0
xalan.version=2.7.1
wagon-http.version=1.0-beta-2
-zookeeper.version=3.4.4
+zookeeper.version=3.4.5
servlet.version=4.0.6
servlet-api.version=2.5
protobuf-java.version=2.4.0a
Added: pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AbstractAccumuloStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AbstractAccumuloStorage.java?rev=1559022&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AbstractAccumuloStorage.java (added)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AbstractAccumuloStorage.java Fri Jan 17 06:36:44 2014
@@ -0,0 +1,764 @@
+/**
+ * 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.pig.backend.hadoop.accumulo;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.pig.LoadCaster;
+import org.apache.pig.LoadFunc;
+import org.apache.pig.LoadStoreCaster;
+import org.apache.pig.ResourceSchema;
+import org.apache.pig.ResourceSchema.ResourceFieldSchema;
+import org.apache.pig.StoreFuncInterface;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
+import org.apache.pig.backend.hadoop.hbase.HBaseBinaryConverter;
+import org.apache.pig.builtin.Utf8StorageConverter;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.util.ObjectSerializer;
+import org.apache.pig.impl.util.UDFContext;
+import org.joda.time.DateTime;
+
+/**
+ * A LoadStoreFunc for retrieving data from and storing data to Accumulo
+ *
+ * A Key/Val pair will be returned as tuples: (key, colfam, colqual, colvis,
+ * timestamp, value). All fields except timestamp are DataByteArray, timestamp
+ * is a long.
+ *
+ * Tuples can be written in 2 forms: (key, colfam, colqual, colvis, value) OR
+ * (key, colfam, colqual, value)
+ *
+ */
+public abstract class AbstractAccumuloStorage extends LoadFunc implements
+ StoreFuncInterface {
+ private static final Log log = LogFactory
+ .getLog(AbstractAccumuloStorage.class);
+
+ protected static final char COLON = ':', COMMA = ',';
+ protected static final String ASTERISK = "*";
+
+ private static final String INPUT_PREFIX = AccumuloInputFormat.class
+ .getSimpleName();
+ private static final String OUTPUT_PREFIX = AccumuloOutputFormat.class
+ .getSimpleName();
+
+ private final static String STRING_CASTER = "UTF8StorageConverter";
+ private final static String BYTE_CASTER = "AccumuloBinaryConverter";
+ private final static String CASTER_PROPERTY = "pig.accumulo.caster";
+
+ protected final AccumuloStorageOptions storageOptions;
+ protected final CommandLine commandLine;
+
+ private RecordReader<Key, Value> reader;
+ private RecordWriter<Text, Mutation> writer;
+
+ protected String inst;
+ protected String zookeepers;
+ protected String user;
+ protected String password;
+ protected String table;
+ protected Text tableName;
+ protected Authorizations authorizations;
+
+ protected List<Column> columns;
+
+ protected String start = null;
+ protected String end = null;
+
+ // Defaults from BatchWriterConfig
+ protected int maxWriteThreads = 3;
+ protected long maxMutationBufferSize = 50 * 1024 * 1024l;
+ protected long maxLatency = Long.MAX_VALUE;
+
+ protected String columnSeparator = ",";
+ protected boolean ignoreWhitespace = true;
+
+ protected LoadStoreCaster caster;
+ protected ResourceSchema schema;
+ protected String contextSignature = null;
+
+ public AbstractAccumuloStorage(String columns, String args)
+ throws ParseException, IOException {
+ storageOptions = new AccumuloStorageOptions();
+ commandLine = storageOptions.getCommandLine(args);
+
+ // Extract any command line args
+ extractArgs(commandLine, storageOptions);
+
+ // Split out the user provided columns
+ parseColumns(columns);
+ }
+
+ /**
+ * Initializes {@link #columnDefs} and splits columns on {@link #COMMA}
+ *
+ * @param columns
+ * CSV of columns
+ */
+ private void parseColumns(String columnStr) {
+ columns = new LinkedList<Column>();
+ if (ignoreWhitespace) {
+ columnStr = StringUtils.strip(columnStr);
+ }
+
+ if (!columnStr.isEmpty()) {
+ for (String column : StringUtils.split(columnStr, columnSeparator)) {
+ columns.add(new Column(ignoreWhitespace ? StringUtils
+ .strip(column) : column));
+ }
+ } else {
+ // Preserve original functionality for empty columns to fetch all
+ // data in a map
+ columns.add(new Column("*"));
+ }
+ }
+
+ /**
+ * Extract arguments passed into the constructor to avoid the URI
+ *
+ * @param cli
+ * @param opts
+ */
+ protected void extractArgs(CommandLine cli, AccumuloStorageOptions opts)
+ throws IOException {
+ if (opts.hasAuthorizations(cli)) {
+ authorizations = opts.getAuthorizations(cli);
+ }
+
+ this.start = cli.getOptionValue(
+ AccumuloStorageOptions.START_ROW_OPTION.getOpt(), null);
+ this.end = cli.getOptionValue(
+ AccumuloStorageOptions.END_ROW_OPTION.getOpt(), null);
+
+ if (cli.hasOption(AccumuloStorageOptions.MAX_LATENCY_OPTION.getOpt())) {
+ this.maxLatency = opts.getInt(cli,
+ AccumuloStorageOptions.MAX_LATENCY_OPTION);
+ }
+
+ if (cli.hasOption(AccumuloStorageOptions.WRITE_THREADS_OPTION.getOpt())) {
+ this.maxWriteThreads = opts.getInt(cli,
+ AccumuloStorageOptions.WRITE_THREADS_OPTION);
+ }
+
+ if (cli.hasOption(AccumuloStorageOptions.MUTATION_BUFFER_SIZE_OPTION
+ .getOpt())) {
+ this.maxMutationBufferSize = opts.getLong(cli,
+ AccumuloStorageOptions.MUTATION_BUFFER_SIZE_OPTION);
+ }
+
+ Properties clientSystemProps = UDFContext.getUDFContext()
+ .getClientSystemProps();
+ String defaultCaster = STRING_CASTER;
+ if (null != clientSystemProps) {
+ defaultCaster = clientSystemProps.getProperty(CASTER_PROPERTY,
+ defaultCaster);
+ }
+
+ String casterOption = cli.getOptionValue("caster", defaultCaster);
+ if (STRING_CASTER.equalsIgnoreCase(casterOption)) {
+ caster = new Utf8StorageConverter();
+ } else if (BYTE_CASTER.equalsIgnoreCase(casterOption)) {
+ caster = new HBaseBinaryConverter();
+ } else {
+ try {
+ caster = (LoadStoreCaster) PigContext
+ .instantiateFuncFromSpec(casterOption);
+ } catch (ClassCastException e) {
+ log.error("Configured caster does not implement LoadCaster interface.");
+ throw new IOException(e);
+ } catch (RuntimeException e) {
+ log.error("Configured caster class not found.", e);
+ throw new IOException(e);
+ }
+ }
+ log.debug("Using caster " + caster.getClass());
+
+ if (cli.hasOption(AccumuloStorageOptions.COLUMN_SEPARATOR_OPTION
+ .getOpt())) {
+ columnSeparator = cli
+ .getOptionValue(AccumuloStorageOptions.COLUMN_SEPARATOR_OPTION
+ .getOpt());
+ }
+
+ if (cli.hasOption(AccumuloStorageOptions.COLUMN_IGNORE_WHITESPACE_OPTION
+ .getOpt())) {
+ String value = cli
+ .getOptionValue(AccumuloStorageOptions.COLUMN_IGNORE_WHITESPACE_OPTION
+ .getOpt());
+ if ("false".equalsIgnoreCase(value)) {
+ ignoreWhitespace = false;
+ } else if ("true".equalsIgnoreCase(value)) {
+ ignoreWhitespace = true;
+ } else {
+ log.warn("Ignoring unknown value for "
+ + AccumuloStorageOptions.COLUMN_IGNORE_WHITESPACE_OPTION
+ .getOpt() + ": " + value);
+ }
+ }
+ }
+
+ protected CommandLine getCommandLine() {
+ return commandLine;
+ }
+
+ protected Map<String, String> getInputFormatEntries(Configuration conf) {
+ return getEntries(conf, INPUT_PREFIX);
+ }
+
+ protected Map<String, String> getOutputFormatEntries(Configuration conf) {
+ return getEntries(conf, OUTPUT_PREFIX);
+ }
+
+ /**
+ * Removes the given values from the configuration, accounting for changes
+ * in the Configuration API given the version of Hadoop being used.
+ *
+ * @param conf
+ * @param entriesToUnset
+ */
+ protected void unsetEntriesFromConfiguration(Configuration conf,
+ Map<String, String> entriesToUnset) {
+ boolean configurationHasUnset = true;
+ try {
+ conf.getClass().getMethod("unset", String.class);
+ } catch (NoSuchMethodException e) {
+ configurationHasUnset = false;
+ } catch (SecurityException e) {
+ configurationHasUnset = false;
+ }
+
+ // Only Hadoop >=1.2.0 and >=0.23 actually contains the method
+ // Configuration#unset
+ if (configurationHasUnset) {
+ simpleUnset(conf, entriesToUnset);
+ } else {
+ // If we're running on something else, we have to remove everything
+ // and re-add it
+ clearUnset(conf, entriesToUnset);
+ }
+ }
+
+ /**
+ * Unsets elements in the Configuration using the unset method
+ *
+ * @param conf
+ * @param entriesToUnset
+ */
+ protected void simpleUnset(Configuration conf,
+ Map<String, String> entriesToUnset) {
+ try {
+ Method unset = conf.getClass().getMethod("unset", String.class);
+
+ for (String key : entriesToUnset.keySet()) {
+ unset.invoke(conf, key);
+ }
+ } catch (NoSuchMethodException e) {
+ log.error("Could not invoke Configuration.unset method", e);
+ throw new RuntimeException(e);
+ } catch (IllegalAccessException e) {
+ log.error("Could not invoke Configuration.unset method", e);
+ throw new RuntimeException(e);
+ } catch (IllegalArgumentException e) {
+ log.error("Could not invoke Configuration.unset method", e);
+ throw new RuntimeException(e);
+ } catch (InvocationTargetException e) {
+ log.error("Could not invoke Configuration.unset method", e);
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Replaces the given entries in the configuration by clearing the
+ * Configuration and re-adding the elements that aren't in the Map of
+ * entries to unset
+ *
+ * @param conf
+ * @param entriesToUnset
+ */
+ protected void clearUnset(Configuration conf,
+ Map<String, String> entriesToUnset) {
+ // Gets a copy of the entries
+ Iterator<Entry<String, String>> originalEntries = conf.iterator();
+ conf.clear();
+
+ while (originalEntries.hasNext()) {
+ Entry<String, String> originalEntry = originalEntries.next();
+
+ // Only re-set() the pairs that aren't in our collection of keys to
+ // unset
+ if (!entriesToUnset.containsKey(originalEntry.getKey())) {
+ conf.set(originalEntry.getKey(), originalEntry.getValue());
+ }
+ }
+ }
+
+ @Override
+ public Tuple getNext() throws IOException {
+ try {
+ // load the next pair
+ if (!reader.nextKeyValue())
+ return null;
+
+ Key key = (Key) reader.getCurrentKey();
+ Value value = (Value) reader.getCurrentValue();
+ assert key != null && value != null;
+ return getTuple(key, value);
+ } catch (InterruptedException e) {
+ throw new IOException(e.getMessage());
+ }
+ }
+
+ protected abstract Tuple getTuple(Key key, Value value) throws IOException;
+
+ @Override
+ @SuppressWarnings("rawtypes")
+ public InputFormat getInputFormat() {
+ return new AccumuloInputFormat();
+ }
+
+ @Override
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ public void prepareToRead(RecordReader reader, PigSplit split) {
+ this.reader = reader;
+ }
+
+ private void setLocationFromUri(String location) throws IOException {
+ // ex:
+ // accumulo://table1?instance=myinstance&user=root&password=secret&zookeepers=127.0.0.1:2181&auths=PRIVATE,PUBLIC&fetch_columns=col1:cq1,col2:cq2&start=abc&end=z
+ String columns = "", auths = "";
+ try {
+ if (!location.startsWith("accumulo://"))
+ throw new Exception("Bad scheme.");
+ String[] urlParts = location.split("\\?");
+ if (urlParts.length > 1) {
+ for (String param : urlParts[1].split("&")) {
+ String[] pair = param.split("=");
+ if (pair[0].equals("instance"))
+ inst = pair[1];
+ else if (pair[0].equals("user"))
+ user = pair[1];
+ else if (pair[0].equals("password"))
+ password = pair[1];
+ else if (pair[0].equals("zookeepers"))
+ zookeepers = pair[1];
+ else if (pair[0].equals("auths"))
+ auths = pair[1];
+ else if (pair[0].equals("fetch_columns"))
+ columns = pair[1];
+ else if (pair[0].equals("start"))
+ start = pair[1];
+ else if (pair[0].equals("end"))
+ end = pair[1];
+ else if (pair[0].equals("write_buffer_size_bytes"))
+ maxMutationBufferSize = Long.parseLong(pair[1]);
+ else if (pair[0].equals("write_threads"))
+ maxWriteThreads = Integer.parseInt(pair[1]);
+ else if (pair[0].equals("write_latency_ms"))
+ maxLatency = Long.parseLong(pair[1]);
+ }
+ }
+ String[] parts = urlParts[0].split("/+");
+ table = parts[1];
+ tableName = new Text(table);
+
+ if (null == authorizations && auths == null) {
+ authorizations = new Authorizations();
+ } else {
+ authorizations = new Authorizations(StringUtils.split(auths,
+ COMMA));
+ }
+
+ if (!StringUtils.isEmpty(columns)) {
+ parseColumns(columns);
+ }
+
+ } catch (Exception e) {
+ throw new IOException(
+ "Expected 'accumulo://<table>[?instance=<instanceName>&user=<user>&password=<password>&zookeepers=<zookeepers>&auths=<authorizations>&"
+ + "[start=startRow,end=endRow,fetch_columns=[cf1:cq1,cf2:cq2,...],write_buffer_size_bytes=10000000,write_threads=10,write_latency_ms=30000]]': "
+ + e.getMessage());
+ }
+ }
+
+ protected RecordWriter<Text, Mutation> getWriter() {
+ return writer;
+ }
+
+ /**
+ * Extract elements from the Configuration whose keys match the given prefix
+ *
+ * @param conf
+ * @param prefix
+ * @return
+ */
+ protected Map<String, String> getEntries(Configuration conf, String prefix) {
+ Map<String, String> entries = new HashMap<String, String>();
+
+ for (Entry<String, String> entry : conf) {
+ String key = entry.getKey();
+ if (key.startsWith(prefix)) {
+ entries.put(key, entry.getValue());
+ }
+ }
+
+ return entries;
+ }
+
+ @Override
+ public void setLocation(String location, Job job) throws IOException {
+ setLocationFromUri(location);
+
+ loadDependentJars(job.getConfiguration());
+
+ Map<String, String> entries = getInputFormatEntries(job
+ .getConfiguration());
+ unsetEntriesFromConfiguration(job.getConfiguration(), entries);
+
+ try {
+ AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(
+ password));
+ } catch (AccumuloSecurityException e) {
+ throw new IOException(e);
+ }
+
+ AccumuloInputFormat.setInputTableName(job, table);
+ AccumuloInputFormat.setScanAuthorizations(job, authorizations);
+ AccumuloInputFormat.setZooKeeperInstance(job, inst, zookeepers);
+
+ List<Pair<Text, Text>> inputFormatColumns = new LinkedList<Pair<Text, Text>>();
+ int colfamPrefix = 0;
+
+ for (Column c : columns) {
+ switch (c.getType()) {
+ case LITERAL:
+ // Pull the colf[:colq] individually
+ inputFormatColumns.add(makePair(c.getColumnFamily(),
+ c.getColumnQualifier()));
+ break;
+ case COLFAM_PREFIX:
+ // Some colfams
+ colfamPrefix++;
+ break;
+ case COLQUAL_PREFIX:
+ // Some colquals in a given colfam
+ inputFormatColumns.add(makePair(c.getColumnFamily(), null));
+ break;
+ default:
+ log.info("Ignoring unhandled column type");
+ break;
+ }
+ }
+
+ // If we have colfam prefixes, we have to pull all columns and filter on
+ // client-side
+ // TODO Create an iterator that lets us push-down *all* of the filter
+ // logic
+ if (0 == colfamPrefix && !inputFormatColumns.isEmpty()) {
+ AccumuloInputFormat.fetchColumns(job, inputFormatColumns);
+ }
+
+ Collection<Range> ranges = Collections.singleton(new Range(start, end));
+
+ log.info("Scanning Accumulo for " + ranges + " for table " + table);
+
+ AccumuloInputFormat.setRanges(job, ranges);
+
+ configureInputFormat(job);
+ }
+
+ protected Pair<Text, Text> makePair(String first, String second) {
+ return new Pair<Text, Text>((null == first) ? null : new Text(first),
+ (null == second) ? null : new Text(second));
+ }
+
+ /**
+ * Ensure that Accumulo's dependent jars are added to the Configuration to
+ * alleviate the need for clients to REGISTER dependency jars.
+ *
+ * @param job
+ * The Mapreduce Job object
+ * @throws IOException
+ */
+ protected void loadDependentJars(Configuration conf) throws IOException {
+ // Thank you, HBase.
+ Utils.addDependencyJars(conf,
+ org.apache.accumulo.trace.instrument.Tracer.class,
+ org.apache.accumulo.core.client.Instance.class,
+ org.apache.accumulo.fate.Fate.class,
+ org.apache.accumulo.server.tabletserver.TabletServer.class,
+ org.apache.zookeeper.ZooKeeper.class,
+ org.apache.thrift.TServiceClient.class);
+ }
+
+ /**
+ * Method to allow specific implementations to add more elements to the Job
+ * for reading data from Accumulo.
+ *
+ * @param job
+ */
+ protected void configureInputFormat(Job job) {
+ }
+
+ /**
+ * Method to allow specific implementations to add more elements to the Job
+ * for writing data to Accumulo.
+ *
+ * @param job
+ */
+ protected void configureOutputFormat(Job job) {
+ }
+
+ @Override
+ public String relativeToAbsolutePath(String location, Path curDir)
+ throws IOException {
+ return location;
+ }
+
+ @Override
+ public void setUDFContextSignature(String signature) {
+ this.contextSignature = signature;
+ }
+
+ /* StoreFunc methods */
+ public void setStoreFuncUDFContextSignature(String signature) {
+ this.contextSignature = signature;
+
+ }
+
+ /**
+ * Returns UDFProperties based on <code>contextSignature</code>.
+ */
+ protected Properties getUDFProperties() {
+ return UDFContext.getUDFContext().getUDFProperties(this.getClass(),
+ new String[] { contextSignature });
+ }
+
+ public String relToAbsPathForStoreLocation(String location, Path curDir)
+ throws IOException {
+ return relativeToAbsolutePath(location, curDir);
+ }
+
+ public void setStoreLocation(String location, Job job) throws IOException {
+ setLocationFromUri(location);
+
+ loadDependentJars(job.getConfiguration());
+
+ Map<String, String> entries = getOutputFormatEntries(job
+ .getConfiguration());
+ unsetEntriesFromConfiguration(job.getConfiguration(), entries);
+
+ try {
+ AccumuloOutputFormat.setConnectorInfo(job, user, new PasswordToken(
+ password));
+ } catch (AccumuloSecurityException e) {
+ throw new IOException(e);
+ }
+
+ AccumuloOutputFormat.setCreateTables(job, true);
+ AccumuloOutputFormat.setZooKeeperInstance(job, inst, zookeepers);
+
+ BatchWriterConfig bwConfig = new BatchWriterConfig();
+ bwConfig.setMaxLatency(maxLatency, TimeUnit.MILLISECONDS);
+ bwConfig.setMaxMemory(maxMutationBufferSize);
+ bwConfig.setMaxWriteThreads(maxWriteThreads);
+ AccumuloOutputFormat.setBatchWriterOptions(job, bwConfig);
+
+ log.info("Writing data to " + table);
+
+ configureOutputFormat(job);
+ }
+
+ @SuppressWarnings("rawtypes")
+ public OutputFormat getOutputFormat() {
+ return new AccumuloOutputFormat();
+ }
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ public void prepareToWrite(RecordWriter writer) {
+ this.writer = writer;
+ }
+
+ protected abstract Collection<Mutation> getMutations(Tuple tuple)
+ throws ExecException, IOException;
+
+ public void putNext(Tuple tuple) throws ExecException, IOException {
+ Collection<Mutation> muts = getMutations(tuple);
+ for (Mutation mut : muts) {
+ try {
+ getWriter().write(tableName, mut);
+ } catch (InterruptedException e) {
+ throw new IOException(e);
+ }
+ }
+ }
+
+ public void cleanupOnFailure(String failure, Job job) {
+ }
+
+ public void cleanupOnSuccess(String location, Job job) {
+ }
+
+ @Override
+ public void checkSchema(ResourceSchema s) throws IOException {
+ if (!(caster instanceof LoadStoreCaster)) {
+ log.error("Caster must implement LoadStoreCaster for writing to Accumulo.");
+ throw new IOException("Bad Caster " + caster.getClass());
+ }
+ schema = s;
+ getUDFProperties().setProperty(contextSignature + "_schema",
+ ObjectSerializer.serialize(schema));
+ }
+
+ protected Text tupleToText(Tuple tuple, int i,
+ ResourceFieldSchema[] fieldSchemas) throws IOException {
+ Object o = tuple.get(i);
+ byte type = schemaToType(o, i, fieldSchemas);
+
+ return objToText(o, type);
+ }
+
+ protected Text objectToText(Object o, ResourceFieldSchema fieldSchema)
+ throws IOException {
+ byte type = schemaToType(o, fieldSchema);
+
+ return objToText(o, type);
+ }
+
+ protected byte schemaToType(Object o, ResourceFieldSchema fieldSchema) {
+ return (fieldSchema == null) ? DataType.findType(o) : fieldSchema
+ .getType();
+ }
+
+ protected byte schemaToType(Object o, int i,
+ ResourceFieldSchema[] fieldSchemas) {
+ return (fieldSchemas == null) ? DataType.findType(o) : fieldSchemas[i]
+ .getType();
+ }
+
+ protected byte[] tupleToBytes(Tuple tuple, int i,
+ ResourceFieldSchema[] fieldSchemas) throws IOException {
+ Object o = tuple.get(i);
+ byte type = schemaToType(o, i, fieldSchemas);
+
+ return objToBytes(o, type);
+
+ }
+
+ protected Text objToText(Object o, byte type) throws IOException {
+ byte[] bytes = objToBytes(o, type);
+
+ if (null == bytes) {
+ log.warn("Creating empty text from null value");
+ return new Text();
+ }
+
+ return new Text(bytes);
+ }
+
+ @SuppressWarnings("unchecked")
+ protected byte[] objToBytes(Object o, byte type) throws IOException {
+ if (o == null)
+ return null;
+ switch (type) {
+ case DataType.BYTEARRAY:
+ return ((DataByteArray) o).get();
+ case DataType.BAG:
+ return caster.toBytes((DataBag) o);
+ case DataType.CHARARRAY:
+ return caster.toBytes((String) o);
+ case DataType.DOUBLE:
+ return caster.toBytes((Double) o);
+ case DataType.FLOAT:
+ return caster.toBytes((Float) o);
+ case DataType.INTEGER:
+ return caster.toBytes((Integer) o);
+ case DataType.LONG:
+ return caster.toBytes((Long) o);
+ case DataType.BIGINTEGER:
+ return caster.toBytes((BigInteger) o);
+ case DataType.BIGDECIMAL:
+ return caster.toBytes((BigDecimal) o);
+ case DataType.BOOLEAN:
+ return caster.toBytes((Boolean) o);
+ case DataType.DATETIME:
+ return caster.toBytes((DateTime) o);
+
+ // The type conversion here is unchecked.
+ // Relying on DataType.findType to do the right thing.
+ case DataType.MAP:
+ return caster.toBytes((Map<String, Object>) o);
+
+ case DataType.NULL:
+ return null;
+ case DataType.TUPLE:
+ return caster.toBytes((Tuple) o);
+ case DataType.ERROR:
+ throw new IOException("Unable to determine type of " + o.getClass());
+ default:
+ throw new IOException("Unable to find a converter for tuple field "
+ + o);
+ }
+ }
+
+ @Override
+ public LoadCaster getLoadCaster() throws IOException {
+ return caster;
+ }
+}
Added: pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloBinaryConverter.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloBinaryConverter.java?rev=1559022&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloBinaryConverter.java (added)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloBinaryConverter.java Fri Jan 17 06:36:44 2014
@@ -0,0 +1,258 @@
+/* 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.pig.backend.hadoop.accumulo;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Map;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.pig.LoadStoreCaster;
+import org.apache.pig.ResourceSchema.ResourceFieldSchema;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.Tuple;
+import org.joda.time.DateTime;
+import org.python.google.common.base.Preconditions;
+
+/**
+ * A LoadStoreCaster implementation which stores most type implementations as
+ * bytes generated from the toString representation with a UTF8 Charset. Pulled
+ * some implementations from the Accumulo Lexicoder implementations in 1.6.0.
+ */
+public class AccumuloBinaryConverter implements LoadStoreCaster {
+ private static final int SIZE_OF_INT = Integer.SIZE / Byte.SIZE;
+ private static final int SIZE_OF_LONG = Long.SIZE / Byte.SIZE;
+
+ /**
+ * NOT IMPLEMENTED
+ */
+ @Override
+ public DataBag bytesToBag(byte[] b, ResourceFieldSchema fieldSchema)
+ throws IOException {
+ throw new ExecException("Can't generate DataBags from byte[]");
+ }
+
+ @Override
+ public BigDecimal bytesToBigDecimal(byte[] b) throws IOException {
+ throw new ExecException("Can't generate a BigInteger from byte[]");
+ }
+
+ @Override
+ public BigInteger bytesToBigInteger(byte[] b) throws IOException {
+ // Taken from Accumulo's BigIntegerLexicoder in 1.6.0
+ DataInputStream dis = new DataInputStream(new ByteArrayInputStream(b));
+ int len = dis.readInt();
+ len = len ^ 0x80000000;
+ len = Math.abs(len);
+
+ byte[] bytes = new byte[len];
+ dis.readFully(bytes);
+
+ bytes[0] = (byte) (0x80 ^ bytes[0]);
+
+ return new BigInteger(bytes);
+ }
+
+ @Override
+ public Boolean bytesToBoolean(byte[] b) throws IOException {
+ Preconditions.checkArgument(1 == b.length);
+ return b[0] == (byte) 1;
+ }
+
+ @Override
+ public String bytesToCharArray(byte[] b) throws IOException {
+ return new String(b, Constants.UTF8);
+ }
+
+ /**
+ * NOT IMPLEMENTED
+ */
+ @Override
+ public DateTime bytesToDateTime(byte[] b) throws IOException {
+ String s = new String(b, Constants.UTF8);
+ return DateTime.parse(s);
+ }
+
+ @Override
+ public Double bytesToDouble(byte[] b) throws IOException {
+ return Double.longBitsToDouble(bytesToLong(b));
+ }
+
+ @Override
+ public Float bytesToFloat(byte[] b) throws IOException {
+ return Float.intBitsToFloat(bytesToInteger(b));
+ }
+
+ @Override
+ public Integer bytesToInteger(byte[] b) throws IOException {
+ Preconditions.checkArgument(b.length == SIZE_OF_INT);
+ int n = 0;
+ for (int i = 0; i < b.length; i++) {
+ n <<= 8;
+ n ^= b[i] & 0xFF;
+ }
+ return n;
+ }
+
+ @Override
+ public Long bytesToLong(byte[] b) throws IOException {
+ Preconditions.checkArgument(b.length == SIZE_OF_LONG);
+ long l = 0;
+ for (int i = 0; i < b.length; i++) {
+ l <<= 8;
+ l ^= b[i] & 0xFF;
+ }
+ return l;
+ }
+
+ /**
+ * NOT IMPLEMENTED
+ */
+ @Override
+ public Map<String, Object> bytesToMap(byte[] b,
+ ResourceFieldSchema fieldSchema) throws IOException {
+ throw new ExecException("Can't generate Map from byte[]");
+ }
+
+ /**
+ * NOT IMPLEMENTED
+ */
+ @Override
+ public Tuple bytesToTuple(byte[] b, ResourceFieldSchema fieldSchema)
+ throws IOException {
+ throw new ExecException("Can't generate a Tuple from byte[]");
+ }
+
+ /**
+ * Not implemented!
+ */
+ @Override
+ public byte[] toBytes(BigDecimal bd) throws IOException {
+ throw new IOException("Can't generate bytes from BigDecimal");
+ }
+
+ @Override
+ public byte[] toBytes(BigInteger bi) throws IOException {
+ // Taken from Accumulo's BigIntegerLexicoder in 1.6.0
+ byte[] bytes = bi.toByteArray();
+
+ byte[] ret = new byte[4 + bytes.length];
+
+ DataOutputStream dos = new DataOutputStream(
+ new FixedByteArrayOutputStream(ret));
+
+ // flip the sign bit
+ bytes[0] = (byte) (0x80 ^ bytes[0]);
+
+ int len = bytes.length;
+ if (bi.signum() < 0)
+ len = -len;
+
+ len = len ^ 0x80000000;
+
+ dos.writeInt(len);
+ dos.write(bytes);
+ dos.close();
+
+ return ret;
+ }
+
+ @Override
+ public byte[] toBytes(Boolean b) throws IOException {
+ return new byte[] { b ? (byte) 1 : (byte) 0 };
+ }
+
+ /**
+ * NOT IMPLEMENTED
+ */
+ @Override
+ public byte[] toBytes(DataBag bag) throws IOException {
+ throw new ExecException("Cant' generate bytes from DataBag");
+ }
+
+ @Override
+ public byte[] toBytes(DataByteArray a) throws IOException {
+ return a.get();
+ }
+
+ @Override
+ public byte[] toBytes(DateTime dt) throws IOException {
+ return dt.toString().getBytes(Constants.UTF8);
+ }
+
+ @Override
+ public byte[] toBytes(Double d) throws IOException {
+ return toBytes(Double.doubleToRawLongBits(d));
+ }
+
+ @Override
+ public byte[] toBytes(Float f) throws IOException {
+ return toBytes(Float.floatToRawIntBits(f));
+ }
+
+ @Override
+ public byte[] toBytes(Integer val) throws IOException {
+ int intVal = val.intValue();
+ byte[] b = new byte[4];
+ for (int i = 3; i > 0; i--) {
+ b[i] = (byte) intVal;
+ intVal >>>= 8;
+ }
+ b[0] = (byte) intVal;
+ return b;
+ }
+
+ @Override
+ public byte[] toBytes(Long val) throws IOException {
+ long longVal = val.longValue();
+ byte[] b = new byte[8];
+ for (int i = 7; i > 0; i--) {
+ b[i] = (byte) longVal;
+ longVal >>>= 8;
+ }
+ b[0] = (byte) longVal;
+ return b;
+ }
+
+ /**
+ * NOT IMPLEMENTED
+ */
+ @Override
+ public byte[] toBytes(Map<String, Object> m) throws IOException {
+ throw new IOException("Can't generate bytes from Map");
+ }
+
+ @Override
+ public byte[] toBytes(String s) throws IOException {
+ return s.getBytes(Constants.UTF8);
+ }
+
+ /**
+ * NOT IMPLEMENTED
+ */
+ @Override
+ public byte[] toBytes(Tuple t) throws IOException {
+ throw new IOException("Can't generate bytes from Tuple");
+ }
+}
Added: pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloStorage.java?rev=1559022&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloStorage.java (added)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloStorage.java Fri Jan 17 06:36:44 2014
@@ -0,0 +1,366 @@
+/**
+ * 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.pig.backend.hadoop.accumulo;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedMap;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.user.WholeRowIterator;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.log4j.Logger;
+import org.apache.pig.ResourceSchema.ResourceFieldSchema;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+
+/**
+ * Basic PigStorage implementation that uses Accumulo as the backing store.
+ *
+ * <p>
+ * When writing data, the first entry in the {@link Tuple} is treated as the row
+ * in the Accumulo key, while subsequent entries in the tuple are handled as
+ * columns in that row. {@link Map}s are expanded, placing the map key in the
+ * column family and the map value in the Accumulo value. Scalars are placed
+ * directly into the value with an empty column qualifier. If the columns
+ * argument on the constructor is omitted, null or the empty String, no column
+ * family is provided on the Keys created for Accumulo
+ * </p>
+ *
+ * <p>
+ * When reading data, if aggregateColfams is true, elements in the same row and
+ * column family are aggregated into a single {@link Map}. This will result in a
+ * {@link Tuple} of length (unique_column_families + 1) for the given row. If
+ * aggregateColfams is false, column family and column qualifier are
+ * concatenated (separated by a colon), and placed into a {@link Map}. This will
+ * result in a {@link Tuple} with two entries, where the latter element has a
+ * number of elements equal to the number of columns in the given row.
+ * </p>
+ */
+public class AccumuloStorage extends AbstractAccumuloStorage {
+ private static final Logger log = Logger.getLogger(AccumuloStorage.class);
+ private static final String COLON = ":", EMPTY = "";
+ private static final Text EMPTY_TEXT = new Text(new byte[0]);
+ private static final DataByteArray EMPTY_DATA_BYTE_ARRAY = new DataByteArray(
+ new byte[0]);
+
+ // Not sure if AccumuloStorage instances need to be thread-safe or not
+ final Text _cfHolder = new Text(), _cqHolder = new Text();
+
+ /**
+ * Creates an AccumuloStorage which writes all values in a {@link Tuple}
+ * with an empty column family and doesn't group column families together on
+ * read (creates on {@link Map} for all columns)
+ */
+ public AccumuloStorage() throws ParseException, IOException {
+ this(EMPTY, EMPTY);
+ }
+
+ /**
+ * Create an AccumuloStorage with a CSV of columns-families to use on write
+ * and whether columns in a row should be grouped by family on read.
+ *
+ * @param columns
+ * A comma-separated list of column families to use when writing
+ * data, aligned to the n'th entry in the tuple
+ * @param aggregateColfams
+ * Should unique column qualifier and value pairs be grouped
+ * together by column family when reading data
+ */
+ public AccumuloStorage(String columns) throws ParseException, IOException {
+ this(columns, EMPTY);
+ }
+
+ public AccumuloStorage(String columnStr, String args)
+ throws ParseException, IOException {
+ super(columnStr, args);
+ }
+
+ @Override
+ protected Tuple getTuple(Key key, Value value) throws IOException {
+ SortedMap<Key, Value> rowKVs = WholeRowIterator.decodeRow(key, value);
+ Tuple tuple = TupleFactory.getInstance().newTuple(columns.size() + 1);
+
+ final Text cfHolder = new Text();
+ final Text cqHolder = new Text();
+ final Text row = key.getRow();
+ int tupleOffset = 0;
+
+ tuple.set(
+ tupleOffset,
+ new DataByteArray(Text.decode(row.getBytes(), 0,
+ row.getLength())));
+
+ for (Column column : this.columns) {
+ tupleOffset++;
+
+ switch (column.getType()) {
+ case LITERAL:
+ cfHolder.set(column.getColumnFamily());
+ if (null != column.getColumnQualifier()) {
+ cqHolder.set(column.getColumnQualifier());
+ } else {
+ cqHolder.set(EMPTY_TEXT);
+ }
+
+ // Get the key where our literal would exist (accounting for
+ // "colf:colq" or "colf:" empty colq)
+ Key literalStartKey = new Key(row, cfHolder, cqHolder);
+
+ SortedMap<Key, Value> tailMap = rowKVs.tailMap(literalStartKey);
+
+ // Find the element
+ if (tailMap.isEmpty()) {
+ tuple.set(tupleOffset, EMPTY_DATA_BYTE_ARRAY);
+ } else {
+ Key actualKey = tailMap.firstKey();
+
+ // Only place it in the tuple if it matches the user
+ // request, avoid using a value from a
+ // key with the wrong colqual
+ if (0 == literalStartKey.compareTo(actualKey,
+ PartialKey.ROW_COLFAM_COLQUAL)) {
+ tuple.set(tupleOffset,
+ new DataByteArray(tailMap.get(actualKey).get()));
+ } else {
+ // This row doesn't have the column we were looking for
+ tuple.set(tupleOffset, EMPTY_DATA_BYTE_ARRAY);
+ }
+ }
+
+ break;
+ case COLFAM_PREFIX:
+ cfHolder.set(column.getColumnFamily());
+ Range colfamPrefixRange = Range.prefix(row, cfHolder);
+ Key colfamPrefixStartKey = new Key(row, cfHolder);
+
+ SortedMap<Key, Value> cfTailMap = rowKVs
+ .tailMap(colfamPrefixStartKey);
+
+ // Find the element
+ if (cfTailMap.isEmpty()) {
+ tuple.set(tupleOffset, EMPTY_DATA_BYTE_ARRAY);
+ } else {
+ HashMap<String, DataByteArray> tupleMap = new HashMap<String, DataByteArray>();
+
+ // Build up a map for all the entries in this row that match
+ // the colfam prefix
+ for (Entry<Key, Value> entry : cfTailMap.entrySet()) {
+ if (colfamPrefixRange.contains(entry.getKey())) {
+ entry.getKey().getColumnFamily(cfHolder);
+ entry.getKey().getColumnQualifier(cqHolder);
+ DataByteArray val = new DataByteArray(entry
+ .getValue().get());
+
+ // Avoid adding an extra ':' when colqual is empty
+ if (0 == cqHolder.getLength()) {
+ tupleMap.put(cfHolder.toString(), val);
+ } else {
+ tupleMap.put(cfHolder.toString() + COLON
+ + cqHolder.toString(), val);
+ }
+ } else {
+ break;
+ }
+ }
+
+ if (!tupleMap.isEmpty()) {
+ tuple.set(tupleOffset, tupleMap);
+ }
+ }
+
+ break;
+ case COLQUAL_PREFIX:
+ cfHolder.set(column.getColumnFamily());
+ cqHolder.set(column.getColumnQualifier());
+ Range colqualPrefixRange = Range
+ .prefix(row, cfHolder, cqHolder);
+ Key colqualPrefixStartKey = new Key(row, cfHolder, cqHolder);
+
+ SortedMap<Key, Value> cqTailMap = rowKVs
+ .tailMap(colqualPrefixStartKey);
+ if (cqTailMap.isEmpty()) {
+ tuple.set(tupleOffset, EMPTY_DATA_BYTE_ARRAY);
+ } else {
+ HashMap<String, DataByteArray> tupleMap = new HashMap<String, DataByteArray>();
+
+ // Build up a map for all the entries in this row that match
+ // the colqual prefix
+ for (Entry<Key, Value> entry : cqTailMap.entrySet()) {
+ if (colqualPrefixRange.contains(entry.getKey())) {
+ entry.getKey().getColumnFamily(cfHolder);
+ entry.getKey().getColumnQualifier(cqHolder);
+ DataByteArray val = new DataByteArray(entry
+ .getValue().get());
+
+ // Avoid the extra ':' on empty colqual
+ if (0 == cqHolder.getLength()) {
+ tupleMap.put(cfHolder.toString(), val);
+ } else {
+ tupleMap.put(cfHolder.toString() + COLON
+ + cqHolder.toString(), val);
+ }
+ } else {
+ break;
+ }
+ }
+
+ if (!tupleMap.isEmpty()) {
+ tuple.set(tupleOffset, tupleMap);
+ }
+ }
+
+ break;
+ default:
+ break;
+ }
+ }
+
+ return tuple;
+ }
+
+ @Override
+ protected void configureInputFormat(Job job) {
+ AccumuloInputFormat.addIterator(job, new IteratorSetting(100,
+ WholeRowIterator.class));
+ }
+
+ @Override
+ protected Collection<Mutation> getMutations(Tuple tuple)
+ throws ExecException, IOException {
+ final ResourceFieldSchema[] fieldSchemas = (schema == null) ? null
+ : schema.getFields();
+
+ Iterator<Object> tupleIter = tuple.iterator();
+
+ if (1 >= tuple.size()) {
+ log.debug("Ignoring tuple of size " + tuple.size());
+ return Collections.emptyList();
+ }
+
+ Mutation mutation = new Mutation(objectToText(tupleIter.next(),
+ (null == fieldSchemas) ? null : fieldSchemas[0]));
+
+ int tupleOffset = 1;
+ Iterator<Column> columnIter = columns.iterator();
+ while (tupleIter.hasNext() && columnIter.hasNext()) {
+ Object o = tupleIter.next();
+ Column column = columnIter.next();
+
+ // Grab the type for this field
+ final byte type = schemaToType(o, (null == fieldSchemas) ? null
+ : fieldSchemas[tupleOffset]);
+
+ switch (column.getType()) {
+ case LITERAL:
+ byte[] bytes = objToBytes(o, type);
+
+ if (null != bytes) {
+ Value value = new Value(bytes);
+
+ // We don't have any column name from non-Maps
+ addColumn(mutation, column.getColumnFamily(),
+ column.getColumnQualifier(), value);
+ }
+ break;
+ case COLFAM_PREFIX:
+ case COLQUAL_PREFIX:
+ Map<String, Object> map;
+ try {
+ map = (Map<String, Object>) o;
+ } catch (ClassCastException e) {
+ log.error("Expected Map at tuple offset " + tupleOffset
+ + " but was " + o.getClass().getSimpleName());
+ throw e;
+ }
+
+ for (Entry<String, Object> entry : map.entrySet()) {
+ String key = entry.getKey();
+ Object objValue = entry.getValue();
+
+ byte valueType = DataType.findType(objValue);
+ byte[] mapValue = objToBytes(objValue, valueType);
+
+ if (Column.Type.COLFAM_PREFIX == column.getType()) {
+ addColumn(mutation, column.getColumnFamily() + key,
+ null, new Value(mapValue));
+ } else if (Column.Type.COLQUAL_PREFIX == column.getType()) {
+ addColumn(mutation, column.getColumnFamily(),
+ column.getColumnQualifier() + key, new Value(
+ mapValue));
+ } else {
+ throw new IOException("Unknown column type");
+ }
+ }
+ break;
+ default:
+ log.info("Ignoring unhandled column type");
+ continue;
+ }
+
+ tupleOffset++;
+ }
+
+ if (0 == mutation.size()) {
+ return Collections.emptyList();
+ }
+
+ return Collections.singletonList(mutation);
+ }
+
+ /**
+ * Adds the given column family, column qualifier and value to the given
+ * mutation
+ *
+ * @param mutation
+ * @param colfam
+ * @param colqual
+ * @param columnValue
+ */
+ protected void addColumn(Mutation mutation, String colfam, String colqual,
+ Value columnValue) {
+ if (null != colfam) {
+ _cfHolder.set(colfam);
+ } else {
+ _cfHolder.clear();
+ }
+
+ if (null != colqual) {
+ _cqHolder.set(colqual);
+ } else {
+ _cqHolder.clear();
+ }
+
+ mutation.put(_cfHolder, _cqHolder, columnValue);
+ }
+}
Added: pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloStorageOptions.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloStorageOptions.java?rev=1559022&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloStorageOptions.java (added)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloStorageOptions.java Fri Jan 17 06:36:44 2014
@@ -0,0 +1,108 @@
+/*
+ * 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.pig.backend.hadoop.accumulo;
+
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang.StringUtils;
+
+public class AccumuloStorageOptions {
+ public static final Option CASTER_OPTION = new Option(
+ "c",
+ "caster",
+ true,
+ "Implementation of LoadStoreCaster to use typically UTF8StringConverter or AccumuloBinaryConverter"),
+ AUTHORIZATIONS_OPTION = new Option("auths", "authorizations", true,
+ "Comma-separated list of authorizations to use"),
+ START_ROW_OPTION = new Option("s", "start", true,
+ "The row to begin reading from, inclusive"),
+ END_ROW_OPTION = new Option("e", "end", true,
+ "The row to read until, inclusive"),
+ MUTATION_BUFFER_SIZE_OPTION = new Option("buff",
+ "mutation-buffer-size", true,
+ "Number of bytes to buffer when writing data"),
+ WRITE_THREADS_OPTION = new Option("wt", "write-threads", true,
+ "Number of threads to use when writing data"),
+ MAX_LATENCY_OPTION = new Option("ml", "max-latency", true,
+ "Maximum latency in milliseconds before Mutations are flushed to Accumulo"),
+ COLUMN_SEPARATOR_OPTION = new Option("sep", "separator", true,
+ "Separator string to use when parsing columns"),
+ COLUMN_IGNORE_WHITESPACE_OPTION = new Option("iw",
+ "ignore-whitespace", true,
+ "Whether or not whitespace should be stripped from column list");
+
+ private Options options;
+ private GnuParser parser;
+
+ public AccumuloStorageOptions() {
+ parser = new GnuParser();
+ options = new Options();
+
+ options.addOption(CASTER_OPTION);
+ options.addOption(AUTHORIZATIONS_OPTION);
+ options.addOption(START_ROW_OPTION);
+ options.addOption(END_ROW_OPTION);
+ options.addOption(MUTATION_BUFFER_SIZE_OPTION);
+ options.addOption(WRITE_THREADS_OPTION);
+ options.addOption(MAX_LATENCY_OPTION);
+ options.addOption(COLUMN_SEPARATOR_OPTION);
+ options.addOption(COLUMN_IGNORE_WHITESPACE_OPTION);
+ }
+
+ public String getHelpMessage() {
+ return "[(-c|--caster) LoadStoreCasterImpl] [(-auths|--authorizations auth1,auth2,auth3] [(-s|--start) startrow]"
+ + " [(-e|--end) endrow] [(-buff|--mutation-buffer-size) bytes] [(-wt|--write-threads) threads] [(-ml|--max-latency) seconds]"
+ + " [(-sep|--separator) ,] [(-iw|--ignore-whitespace) true|false]";
+ }
+
+ public CommandLine getCommandLine(String args) throws ParseException {
+ String[] splitArgs = StringUtils.split(args);
+ try {
+ return parser.parse(options, splitArgs);
+ } catch (ParseException e) {
+ HelpFormatter formatter = new HelpFormatter();
+ formatter.printHelp(getHelpMessage(), options);
+ throw e;
+ }
+ }
+
+ public boolean hasAuthorizations(CommandLine cli) {
+ return cli.hasOption(AUTHORIZATIONS_OPTION.getOpt());
+ }
+
+ public Authorizations getAuthorizations(CommandLine cli) {
+ return new Authorizations(cli.getOptionValue(
+ AUTHORIZATIONS_OPTION.getOpt(), ""));
+ }
+
+ public long getLong(CommandLine cli, Option o) {
+ String value = cli.getOptionValue(o.getOpt());
+
+ return (null == value) ? null : Long.parseLong(value);
+ }
+
+ public int getInt(CommandLine cli, Option o) {
+ String value = cli.getOptionValue(o.getOpt());
+
+ return (null == value) ? null : Integer.parseInt(value);
+ }
+}
Added: pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Column.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Column.java?rev=1559022&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Column.java (added)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Column.java Fri Jan 17 06:36:44 2014
@@ -0,0 +1,126 @@
+/*
+ * 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.pig.backend.hadoop.accumulo;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Extracts necessary information from a user provide column "specification":
+ * colf[[*]:[colq[*]]]
+ *
+ * Removes any trailing asterisk on colfam or colqual, and appropriately sets
+ * the {#link Column.Type}
+ */
+public class Column {
+
+ public static enum Type {
+ LITERAL, COLFAM_PREFIX, COLQUAL_PREFIX,
+ }
+
+ private final Type columnType;
+ private String columnFamily;
+ private String columnQualifier;
+
+ public Column(String col) {
+ Preconditions.checkNotNull(col);
+
+ int index = col.indexOf(AbstractAccumuloStorage.COLON);
+ if (-1 == index) {
+ columnFamily = col;
+ columnQualifier = null;
+
+ if (columnFamily.endsWith(AbstractAccumuloStorage.ASTERISK)) {
+ columnFamily = columnFamily.substring(0,
+ columnFamily.length() - 1);
+ columnType = Type.COLFAM_PREFIX;
+ } else {
+ columnType = Type.LITERAL;
+ }
+ } else {
+ if (1 == col.length()) {
+ throw new IllegalArgumentException("Cannot parse '" + col + "'");
+ }
+
+ columnFamily = col.substring(0, index);
+ columnQualifier = col.substring(index + 1);
+
+ // TODO Handle colf*:colq* ?
+ if (columnFamily.endsWith(AbstractAccumuloStorage.ASTERISK)) {
+ columnType = Type.COLFAM_PREFIX;
+ columnFamily = columnFamily.substring(0,
+ columnFamily.length() - 1);
+ } else if (columnQualifier.isEmpty()) {
+ columnType = Type.COLQUAL_PREFIX;
+ } else if (columnQualifier
+ .endsWith(AbstractAccumuloStorage.ASTERISK)) {
+ columnType = Type.COLQUAL_PREFIX;
+ columnQualifier = columnQualifier.substring(0,
+ columnQualifier.length() - 1);
+ } else {
+ columnType = Type.LITERAL;
+ }
+ }
+ }
+
+ public Type getType() {
+ return columnType;
+ }
+
+ public String getColumnFamily() {
+ return columnFamily;
+ }
+
+ public String getColumnQualifier() {
+ return columnQualifier;
+ }
+
+ public boolean matchAll() {
+ return Type.COLFAM_PREFIX.equals(columnType) && "".equals(columnFamily);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o instanceof Column) {
+ Column other = (Column) o;
+
+ if (null != columnFamily) {
+ if (null == other.columnFamily) {
+ return false;
+ } else if (!columnFamily.equals(other.columnFamily)) {
+ return false;
+ }
+ }
+
+ if (null != columnQualifier) {
+ if (null == other.columnQualifier) {
+ return false;
+ } else if (!columnQualifier.equals(other.columnQualifier)) {
+ return false;
+ }
+ }
+
+ return columnType == other.columnType;
+ }
+
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return columnType + " " + columnFamily + ":" + columnQualifier;
+ }
+}
Added: pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/FixedByteArrayOutputStream.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/FixedByteArrayOutputStream.java?rev=1559022&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/FixedByteArrayOutputStream.java (added)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/FixedByteArrayOutputStream.java Fri Jan 17 06:36:44 2014
@@ -0,0 +1,46 @@
+/*
+ * 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.pig.backend.hadoop.accumulo;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * Uses a fixed length array and will not grow in size dynamically like the
+ * {@link java.io.ByteArrayOutputStream}.
+ */
+public class FixedByteArrayOutputStream extends OutputStream {
+
+ private int i;
+ byte out[];
+
+ public FixedByteArrayOutputStream(byte out[]) {
+ this.out = out;
+ }
+
+ @Override
+ public void write(int b) throws IOException {
+ out[i++] = (byte) b;
+ }
+
+ @Override
+ public void write(byte b[], int off, int len) throws IOException {
+ System.arraycopy(b, off, out, i, len);
+ i += len;
+ }
+
+}
Added: pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Utils.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Utils.java?rev=1559022&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Utils.java (added)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Utils.java Fri Jan 17 06:36:44 2014
@@ -0,0 +1,368 @@
+/**
+ * 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.pig.backend.hadoop.accumulo;
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.text.MessageFormat;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.jar.JarFile;
+import java.util.jar.JarOutputStream;
+import java.util.jar.Manifest;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+import java.util.zip.ZipOutputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Logger;
+
+import com.google.common.base.Preconditions;
+
+public class Utils {
+ private static final Logger log = Logger.getLogger(Utils.class);
+
+ // Thanks, HBase
+ public static void addDependencyJars(Configuration conf,
+ Class<?>... classes) throws IOException {
+
+ FileSystem localFs = FileSystem.getLocal(conf);
+ Set<String> jars = new HashSet<String>();
+ // Add jars that are already in the tmpjars variable
+ jars.addAll(conf.getStringCollection("tmpjars"));
+
+ // add jars as we find them to a map of contents jar name so that we can
+ // avoid
+ // creating new jars for classes that have already been packaged.
+ Map<String, String> packagedClasses = new HashMap<String, String>();
+
+ // Add jars containing the specified classes
+ for (Class<?> clazz : classes) {
+ if (clazz == null)
+ continue;
+
+ Path path = findOrCreateJar(clazz, localFs, packagedClasses);
+ if (path == null) {
+ log.warn("Could not find jar for class " + clazz
+ + " in order to ship it to the cluster.");
+ continue;
+ }
+ if (!localFs.exists(path)) {
+ log.warn("Could not validate jar file " + path + " for class "
+ + clazz);
+ continue;
+ }
+ jars.add(path.toString());
+ }
+ if (jars.isEmpty())
+ return;
+
+ conf.set("tmpjars", StringUtils.arrayToString(jars
+ .toArray(new String[jars.size()])));
+ }
+
+ /**
+ * If org.apache.hadoop.util.JarFinder is available (0.23+ hadoop), finds
+ * the Jar for a class or creates it if it doesn't exist. If the class is in
+ * a directory in the classpath, it creates a Jar on the fly with the
+ * contents of the directory and returns the path to that Jar. If a Jar is
+ * created, it is created in the system temporary directory. Otherwise,
+ * returns an existing jar that contains a class of the same name. Maintains
+ * a mapping from jar contents to the tmp jar created.
+ *
+ * @param my_class
+ * the class to find.
+ * @param fs
+ * the FileSystem with which to qualify the returned path.
+ * @param packagedClasses
+ * a map of class name to path.
+ * @return a jar file that contains the class.
+ * @throws IOException
+ */
+ private static Path findOrCreateJar(Class<?> my_class, FileSystem fs,
+ Map<String, String> packagedClasses) throws IOException {
+ // attempt to locate an existing jar for the class.
+ String jar = findContainingJar(my_class, packagedClasses);
+ if (null == jar || jar.isEmpty()) {
+ jar = getJar(my_class);
+ updateMap(jar, packagedClasses);
+ }
+
+ if (null == jar || jar.isEmpty()) {
+ return null;
+ }
+
+ log.debug(String.format("For class %s, using jar %s",
+ my_class.getName(), jar));
+ return new Path(jar).makeQualified(fs);
+ }
+
+ /**
+ * Add entries to <code>packagedClasses</code> corresponding to class files
+ * contained in <code>jar</code>.
+ *
+ * @param jar
+ * The jar who's content to list.
+ * @param packagedClasses
+ * map[class -> jar]
+ */
+ private static void updateMap(String jar,
+ Map<String, String> packagedClasses) throws IOException {
+ if (null == jar || jar.isEmpty()) {
+ return;
+ }
+ ZipFile zip = null;
+ try {
+ zip = new ZipFile(jar);
+ for (Enumeration<? extends ZipEntry> iter = zip.entries(); iter
+ .hasMoreElements();) {
+ ZipEntry entry = iter.nextElement();
+ if (entry.getName().endsWith("class")) {
+ packagedClasses.put(entry.getName(), jar);
+ }
+ }
+ } finally {
+ if (null != zip)
+ zip.close();
+ }
+ }
+
+ /**
+ * Find a jar that contains a class of the same name, if any. It will return
+ * a jar file, even if that is not the first thing on the class path that
+ * has a class with the same name. Looks first on the classpath and then in
+ * the <code>packagedClasses</code> map.
+ *
+ * @param my_class
+ * the class to find.
+ * @return a jar file that contains the class, or null.
+ * @throws IOException
+ */
+ private static String findContainingJar(Class<?> my_class,
+ Map<String, String> packagedClasses) throws IOException {
+ ClassLoader loader = my_class.getClassLoader();
+ String class_file = my_class.getName().replaceAll("\\.", "/")
+ + ".class";
+
+ // first search the classpath
+ for (Enumeration<URL> itr = loader.getResources(class_file); itr
+ .hasMoreElements();) {
+ URL url = itr.nextElement();
+ if ("jar".equals(url.getProtocol())) {
+ String toReturn = url.getPath();
+ if (toReturn.startsWith("file:")) {
+ toReturn = toReturn.substring("file:".length());
+ }
+ // URLDecoder is a misnamed class, since it actually decodes
+ // x-www-form-urlencoded MIME type rather than actual
+ // URL encoding (which the file path has). Therefore it would
+ // decode +s to ' 's which is incorrect (spaces are actually
+ // either unencoded or encoded as "%20"). Replace +s first, so
+ // that they are kept sacred during the decoding process.
+ toReturn = toReturn.replaceAll("\\+", "%2B");
+ toReturn = URLDecoder.decode(toReturn, "UTF-8");
+ return toReturn.replaceAll("!.*$", "");
+ }
+ }
+
+ // now look in any jars we've packaged using JarFinder. Returns null
+ // when
+ // no jar is found.
+ return packagedClasses.get(class_file);
+ }
+
+ /**
+ * Invoke 'getJar' on a JarFinder implementation. Useful for some job
+ * configuration contexts (HBASE-8140) and also for testing on MRv2. First
+ * check if we have HADOOP-9426. Lacking that, fall back to the backport.
+ *
+ * @param my_class
+ * the class to find.
+ * @return a jar file that contains the class, or null.
+ */
+ private static String getJar(Class<?> my_class) {
+ String ret = null;
+ String hadoopJarFinder = "org.apache.hadoop.util.JarFinder";
+ Class<?> jarFinder = null;
+ try {
+ log.debug("Looking for " + hadoopJarFinder + ".");
+ jarFinder = Class.forName(hadoopJarFinder);
+ log.debug(hadoopJarFinder + " found.");
+ Method getJar = jarFinder.getMethod("getJar", Class.class);
+ ret = (String) getJar.invoke(null, my_class);
+ } catch (ClassNotFoundException e) {
+ log.debug("Using backported JarFinder.");
+ ret = jarFinderGetJar(my_class);
+ } catch (InvocationTargetException e) {
+ // function was properly called, but threw it's own exception.
+ // Unwrap it
+ // and pass it on.
+ throw new RuntimeException(e.getCause());
+ } catch (Exception e) {
+ // toss all other exceptions, related to reflection failure
+ throw new RuntimeException("getJar invocation failed.", e);
+ }
+
+ return ret;
+ }
+
+ /**
+ * Returns the full path to the Jar containing the class. It always return a
+ * JAR.
+ *
+ * @param klass
+ * class.
+ *
+ * @return path to the Jar containing the class.
+ */
+ @SuppressWarnings("rawtypes")
+ public static String jarFinderGetJar(Class klass) {
+ Preconditions.checkNotNull(klass, "klass");
+ ClassLoader loader = klass.getClassLoader();
+ if (loader != null) {
+ String class_file = klass.getName().replaceAll("\\.", "/")
+ + ".class";
+ try {
+ for (Enumeration itr = loader.getResources(class_file); itr
+ .hasMoreElements();) {
+ URL url = (URL) itr.nextElement();
+ String path = url.getPath();
+ if (path.startsWith("file:")) {
+ path = path.substring("file:".length());
+ }
+ path = URLDecoder.decode(path, "UTF-8");
+ if ("jar".equals(url.getProtocol())) {
+ path = URLDecoder.decode(path, "UTF-8");
+ return path.replaceAll("!.*$", "");
+ } else if ("file".equals(url.getProtocol())) {
+ String klassName = klass.getName();
+ klassName = klassName.replace(".", "/") + ".class";
+ path = path.substring(0,
+ path.length() - klassName.length());
+ File baseDir = new File(path);
+ File testDir = new File(System.getProperty(
+ "test.build.dir", "target/test-dir"));
+ testDir = testDir.getAbsoluteFile();
+ if (!testDir.exists()) {
+ testDir.mkdirs();
+ }
+ File tempJar = File.createTempFile("hadoop-", "",
+ testDir);
+ tempJar = new File(tempJar.getAbsolutePath() + ".jar");
+ createJar(baseDir, tempJar);
+ return tempJar.getAbsolutePath();
+ }
+ }
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ return null;
+ }
+
+ private static void copyToZipStream(InputStream is, ZipEntry entry,
+ ZipOutputStream zos) throws IOException {
+ zos.putNextEntry(entry);
+ byte[] arr = new byte[4096];
+ int read = is.read(arr);
+ while (read > -1) {
+ zos.write(arr, 0, read);
+ read = is.read(arr);
+ }
+ is.close();
+ zos.closeEntry();
+ }
+
+ public static void jarDir(File dir, String relativePath, ZipOutputStream zos)
+ throws IOException {
+ Preconditions.checkNotNull(relativePath, "relativePath");
+ Preconditions.checkNotNull(zos, "zos");
+
+ // by JAR spec, if there is a manifest, it must be the first entry in
+ // the
+ // ZIP.
+ File manifestFile = new File(dir, JarFile.MANIFEST_NAME);
+ ZipEntry manifestEntry = new ZipEntry(JarFile.MANIFEST_NAME);
+ if (!manifestFile.exists()) {
+ zos.putNextEntry(manifestEntry);
+ new Manifest().write(new BufferedOutputStream(zos));
+ zos.closeEntry();
+ } else {
+ InputStream is = new FileInputStream(manifestFile);
+ copyToZipStream(is, manifestEntry, zos);
+ }
+ zos.closeEntry();
+ zipDir(dir, relativePath, zos, true);
+ zos.close();
+ }
+
+ private static void zipDir(File dir, String relativePath,
+ ZipOutputStream zos, boolean start) throws IOException {
+ String[] dirList = dir.list();
+ for (String aDirList : dirList) {
+ File f = new File(dir, aDirList);
+ if (!f.isHidden()) {
+ if (f.isDirectory()) {
+ if (!start) {
+ ZipEntry dirEntry = new ZipEntry(relativePath
+ + f.getName() + "/");
+ zos.putNextEntry(dirEntry);
+ zos.closeEntry();
+ }
+ String filePath = f.getPath();
+ File file = new File(filePath);
+ zipDir(file, relativePath + f.getName() + "/", zos, false);
+ } else {
+ String path = relativePath + f.getName();
+ if (!path.equals(JarFile.MANIFEST_NAME)) {
+ ZipEntry anEntry = new ZipEntry(path);
+ InputStream is = new FileInputStream(f);
+ copyToZipStream(is, anEntry, zos);
+ }
+ }
+ }
+ }
+ }
+
+ private static void createJar(File dir, File jarFile) throws IOException {
+ Preconditions.checkNotNull(dir, "dir");
+ Preconditions.checkNotNull(jarFile, "jarFile");
+ File jarDir = jarFile.getParentFile();
+ if (!jarDir.exists()) {
+ if (!jarDir.mkdirs()) {
+ throw new IOException(MessageFormat.format(
+ "could not create dir [{0}]", jarDir));
+ }
+ }
+ JarOutputStream zos = new JarOutputStream(new FileOutputStream(jarFile));
+ jarDir(dir, "", zos);
+ }
+}
Modified: pig/trunk/test/excluded-tests-23
URL: http://svn.apache.org/viewvc/pig/trunk/test/excluded-tests-23?rev=1559022&r1=1559021&r2=1559022&view=diff
==============================================================================
--- pig/trunk/test/excluded-tests-23 (original)
+++ pig/trunk/test/excluded-tests-23 Fri Jan 17 06:36:44 2014
@@ -1,2 +1,3 @@
**/TestHBaseStorage.java
**/TestStreamingUDF.java
+**/TestAccumuloPigCluster.java
\ No newline at end of file