You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by cj...@apache.org on 2013/10/16 02:22:09 UTC
[1/5] git commit: ACCUMULO-391 Renaming TableQueryConfig to
BatchScanConfig and moving it into proper client location
Updated Branches:
refs/heads/master 611463972 -> 9a63ff4ec
ACCUMULO-391 Renaming TableQueryConfig to BatchScanConfig and moving it into proper client location
Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/87b104d2
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/87b104d2
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/87b104d2
Branch: refs/heads/master
Commit: 87b104d2322cb17e177322e76c51f8ea5eaaa206
Parents: 5c49655
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Sun Oct 13 09:12:47 2013 -0400
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Tue Oct 15 20:21:45 2013 -0400
----------------------------------------------------------------------
.../core/client/mapred/AccumuloInputFormat.java | 2 +-
.../core/client/mapred/InputFormatBase.java | 26 +-
.../client/mapreduce/AbstractInputFormat.java | 19 +-
.../client/mapreduce/AccumuloInputFormat.java | 2 +-
.../AccumuloMultiTableInputFormat.java | 5 +-
.../core/client/mapreduce/BatchScanConfig.java | 369 +++++++++++++++++++
.../mapreduce/lib/util/InputConfigurator.java | 36 +-
.../accumulo/core/conf/TableQueryConfig.java | 369 -------------------
.../client/mapred/AccumuloInputFormatTest.java | 16 +-
.../AccumuloMultiTableInputFormatTest.java | 11 +-
.../core/conf/TableQueryConfigTest.java | 19 +-
11 files changed, 436 insertions(+), 438 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
index b3b9fc7..f7b6966 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.mapred.Reporter;
* <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, String)}
* <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, Authorizations)}
* <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, String, String)} OR {@link AccumuloInputFormat#setMockInstance(JobConf, String)}
- * <li>{@link AccumuloInputFormat#setTableQueryConfigs(JobConf, org.apache.accumulo.core.conf.TableQueryConfig...)}</li>
+ * <li>{@link AccumuloInputFormat#setTableQueryConfigs(JobConf, org.apache.accumulo.core.client.mapreduce.BatchScanConfig...)}</li>
* </ul>
*
* Other static methods are optional.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
index 78fa1bc..258a13a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
@@ -44,11 +44,11 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
import org.apache.accumulo.core.client.impl.OfflineScanner;
import org.apache.accumulo.core.client.impl.Tables;
import org.apache.accumulo.core.client.impl.TabletLocator;
+import org.apache.accumulo.core.client.mapreduce.BatchScanConfig;
import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
import org.apache.accumulo.core.client.mock.MockInstance;
import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.conf.TableQueryConfig;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.KeyExtent;
import org.apache.accumulo.core.data.PartialKey;
@@ -528,7 +528,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
}
/**
- * Sets the {@link TableQueryConfig} objects on the given Hadoop configuration
+ * Sets the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} objects on the given Hadoop configuration
*
* @param job
* the Hadoop job instance to be configured
@@ -536,16 +536,16 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
* the table query configs to be set on the configuration.
* @since 1.6.0
*/
- public static void setTableQueryConfigs(JobConf job, TableQueryConfig... configs) {
+ public static void setTableQueryConfigs(JobConf job, BatchScanConfig... configs) {
checkNotNull(configs);
InputConfigurator.setTableQueryConfigs(CLASS, job, configs);
}
/**
- * Fetches all {@link TableQueryConfig}s that have been set on the given Hadoop configuration.
+ * Fetches all {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig}s that have been set on the given Hadoop configuration.
*
* <p>
- * Note this also returns the {@link TableQueryConfig} representing the table configurations set through the single table input methods (
+ * Note this also returns the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} representing the table configurations set through the single table input methods (
* {@link #setInputTableName(JobConf, String)}, {@link #setRanges(JobConf, java.util.Collection)}, {@link #fetchColumns(JobConf, java.util.Collection)},
* {@link #addIterator(JobConf, IteratorSetting)}, etc...)
*
@@ -554,12 +554,12 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
* @return
* @since 1.6.0
*/
- public static List<TableQueryConfig> getTableQueryConfigs(JobConf job) {
+ public static List<BatchScanConfig> getTableQueryConfigs(JobConf job) {
return InputConfigurator.getTableQueryConfigs(CLASS, job);
}
/**
- * Fetches a {@link TableQueryConfig} that has been set on the configuration for a specific table.
+ * Fetches a {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} that has been set on the configuration for a specific table.
*
* <p>
* null is returned in the event that the table doesn't exist.
@@ -568,10 +568,10 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
* the Hadoop job instance to be configured
* @param tableName
* the table name for which to grab the config object
- * @return the {@link TableQueryConfig} for the given table
+ * @return the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} for the given table
* @since 1.6.0
*/
- public static TableQueryConfig getTableQueryConfig(JobConf job, String tableName) {
+ public static BatchScanConfig getTableQueryConfig(JobConf job, String tableName) {
return InputConfigurator.getTableQueryConfig(CLASS, job, tableName);
}
@@ -627,7 +627,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
* the scanner to configure
*/
protected void setupIterators(JobConf job, Scanner scanner, String tableName) {
- TableQueryConfig config = getTableQueryConfig(job, tableName);
+ BatchScanConfig config = getTableQueryConfig(job, tableName);
List<IteratorSetting> iterators = config.getIterators();
for (IteratorSetting iterator : iterators)
scanner.addScanIterator(iterator);
@@ -645,7 +645,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
AuthenticationToken token = getAuthenticationToken(job);
Authorizations authorizations = getScanAuthorizations(job);
- TableQueryConfig tableConfig = getTableQueryConfig(job, split.getTableName());
+ BatchScanConfig tableConfig = getTableQueryConfig(job, split.getTableName());
// in case the table name changed, we can still use the previous name for terms of configuration,
// but for the scanner, we'll need to reference the new table name.
@@ -828,8 +828,8 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
validateOptions(job);
LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
- List<TableQueryConfig> tableConfigs = getTableQueryConfigs(job);
- for (TableQueryConfig tableConfig : tableConfigs) {
+ List<BatchScanConfig> tableConfigs = getTableQueryConfigs(job);
+ for (BatchScanConfig tableConfig : tableConfigs) {
boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
String tableId = null;
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 6568f35..503718e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -33,7 +33,6 @@ import org.apache.accumulo.core.client.impl.TabletLocator;
import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
import org.apache.accumulo.core.client.mock.MockInstance;
import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.TableQueryConfig;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.KeyExtent;
@@ -259,19 +258,19 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
}
/**
- * Fetches all {@link TableQueryConfig}s that have been set on the given Hadoop configuration.
+ * Fetches all {@link BatchScanConfig}s that have been set on the given Hadoop configuration.
*
* @param job
* the Hadoop job instance to be configured
* @return
* @since 1.6.0
*/
- protected static List<TableQueryConfig> getBatchScanConfigs(JobContext job) {
+ protected static List<BatchScanConfig> getBatchScanConfigs(JobContext job) {
return InputConfigurator.getTableQueryConfigs(CLASS, getConfiguration(job));
}
/**
- * Fetches a {@link TableQueryConfig} that has been set on the configuration for a specific table.
+ * Fetches a {@link BatchScanConfig} that has been set on the configuration for a specific table.
*
* <p>
* null is returned in the event that the table doesn't exist.
@@ -280,10 +279,10 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
* the Hadoop job instance to be configured
* @param tableName
* the table name for which to grab the config object
- * @return the {@link TableQueryConfig} for the given table
+ * @return the {@link BatchScanConfig} for the given table
* @since 1.6.0
*/
- protected static TableQueryConfig getBatchScanConfig(JobContext job, String tableName) {
+ protected static BatchScanConfig getBatchScanConfig(JobContext job, String tableName) {
return InputConfigurator.getTableQueryConfig(CLASS, getConfiguration(job), tableName);
}
@@ -345,7 +344,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
* the table name for which to set up the iterators
*/
protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName) {
- TableQueryConfig config = getBatchScanConfig(context, tableName);
+ BatchScanConfig config = getBatchScanConfig(context, tableName);
List<IteratorSetting> iterators = config.getIterators();
for (IteratorSetting iterator : iterators)
scanner.addScanIterator(iterator);
@@ -363,7 +362,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
Instance instance = getInstance(attempt);
String principal = getPrincipal(attempt);
- TableQueryConfig tableConfig = getBatchScanConfig(attempt, split.getTableName());
+ BatchScanConfig tableConfig = getBatchScanConfig(attempt, split.getTableName());
// in case the table name changed, we can still use the previous name for terms of configuration,
// but for the scanner, we'll need to reference the new table name.
@@ -558,8 +557,8 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
validateOptions(conf);
LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
- List<TableQueryConfig> tableConfigs = getBatchScanConfigs(conf);
- for (TableQueryConfig tableConfig : tableConfigs) {
+ List<BatchScanConfig> tableConfigs = getBatchScanConfigs(conf);
+ for (BatchScanConfig tableConfig : tableConfigs) {
boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
String tableId = null;
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
index 44dfc33..c7dfda5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
* <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, String)}
* <li>{@link AccumuloInputFormat#setScanAuthorizations(Job, Authorizations)}
* <li>{@link AccumuloInputFormat#setZooKeeperInstance(Job, String, String)} OR {@link AccumuloInputFormat#setMockInstance(Job, String)}
- * <li>{@link AccumuloInputFormat#setTableQueryConfigs(Job, org.apache.accumulo.core.conf.TableQueryConfig...)}
+ * <li>{@link AccumuloInputFormat#setTableQueryConfigs(Job, BatchScanConfig...)}
* </ul>
*
* Other static methods are optional.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
index 2123ab9..f92de4b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
@@ -1,7 +1,6 @@
package org.apache.accumulo.core.client.mapreduce;
import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
-import org.apache.accumulo.core.conf.TableQueryConfig;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.util.format.DefaultFormatter;
@@ -18,7 +17,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value>{
/**
- * Sets the {@link org.apache.accumulo.core.conf.TableQueryConfig} objects on the given Hadoop configuration
+ * Sets the {@link BatchScanConfig} objects on the given Hadoop configuration
*
* @param job
* the Hadoop job instance to be configured
@@ -26,7 +25,7 @@ public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value
* the table query configs to be set on the configuration.
* @since 1.6.0
*/
- public static void setBatchScanConfigs(Job job, TableQueryConfig... configs) {
+ public static void setBatchScanConfigs(Job job, BatchScanConfig... configs) {
checkNotNull(configs);
InputConfigurator.setTableQueryConfigs(CLASS, getConfiguration(job), configs);
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
new file mode 100644
index 0000000..d084b1a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
@@ -0,0 +1,369 @@
+/*
+ * 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.accumulo.core.client.mapreduce;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * This class to holds a query configuration for a table. It contains all the properties needed to specify how rows should be returned from the table.
+ */
+public class BatchScanConfig implements Writable {
+
+ private String tableName;
+ private List<IteratorSetting> iterators;
+ private List<Range> ranges;
+ private Set<Pair<Text,Text>> columns;
+
+ private boolean autoAdjustRanges = true;
+ private boolean useLocalIterators = false;
+ private boolean useIsolatedScanners = false;
+ private boolean offlineScan = false;
+
+ public BatchScanConfig(String tableName) {
+ checkNotNull(tableName);
+ this.tableName = tableName;
+ }
+
+ public BatchScanConfig(DataInput input) throws IOException {
+ readFields(input);
+ }
+
+ /**
+ * Returns the table name associated with this configuration
+ */
+ public String getTableName() {
+ return tableName;
+ }
+
+ /**
+ * Sets the input ranges to scan for all tables associated with this job. This will be added to any per-table ranges that have been set using
+ *
+ * @param ranges
+ * the ranges that will be mapped over
+ * @since 1.6.0
+ */
+ public BatchScanConfig setRanges(List<Range> ranges) {
+ this.ranges = ranges;
+ return this;
+ }
+
+ /**
+ * Returns the ranges to be queried in the configuration
+ */
+ public List<Range> getRanges() {
+ return ranges != null ? ranges : new ArrayList<Range>();
+ }
+
+ /**
+ * Restricts the columns that will be mapped over for this job for the default input table.
+ *
+ * @param columns
+ * a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
+ * selected. An empty set is the default and is equivalent to scanning the all columns.
+ * @since 1.6.0
+ */
+ public BatchScanConfig fetchColumns(Set<Pair<Text,Text>> columns) {
+ this.columns = columns;
+ return this;
+ }
+
+ /**
+ * Returns the columns to be fetched for this configuration
+ */
+ public Set<Pair<Text,Text>> getFetchedColumns() {
+ return columns != null ? columns : new HashSet<Pair<Text,Text>>();
+ }
+
+ /**
+ * Set iterators on to be used in the query.
+ *
+ * @param iterators
+ * the configurations for the iterators
+ * @since 1.6.0
+ */
+ public BatchScanConfig setIterators(List<IteratorSetting> iterators) {
+ this.iterators = iterators;
+ return this;
+ }
+
+ /**
+ * Returns the iterators to be set on this configuration
+ */
+ public List<IteratorSetting> getIterators() {
+ return iterators != null ? iterators : new ArrayList<IteratorSetting>();
+ }
+
+ /**
+ * Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
+ * Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
+ *
+ * <p>
+ * By default, this feature is <b>enabled</b>.
+ *
+ * @param autoAdjustRanges
+ * the feature is enabled if true, disabled otherwise
+ * @see #setRanges(java.util.List)
+ * @since 1.6.0
+ */
+ public BatchScanConfig setAutoAdjustRanges(boolean autoAdjustRanges) {
+ this.autoAdjustRanges = autoAdjustRanges;
+ return this;
+ }
+
+ /**
+ * Determines whether a configuration has auto-adjust ranges enabled.
+ *
+ * @return false if the feature is disabled, true otherwise
+ * @since 1.6.0
+ * @see #setAutoAdjustRanges(boolean)
+ */
+ public boolean shouldAutoAdjustRanges() {
+ return autoAdjustRanges;
+ }
+
+ /**
+ * Controls the use of the {@link org.apache.accumulo.core.client.ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack
+ * to be constructed within the Map task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be
+ * available on the classpath for the task.
+ *
+ * <p>
+ * By default, this feature is <b>disabled</b>.
+ *
+ * @param useLocalIterators
+ * the feature is enabled if true, disabled otherwise
+ * @since 1.6.0
+ */
+ public BatchScanConfig setUseLocalIterators(boolean useLocalIterators) {
+ this.useLocalIterators = useLocalIterators;
+ return this;
+ }
+
+ /**
+ * Determines whether a configuration uses local iterators.
+ *
+ * @return true if the feature is enabled, false otherwise
+ * @since 1.6.0
+ * @see #setUseLocalIterators(boolean)
+ * @deprecated since 1.6.0
+ */
+ public boolean shouldUseLocalIterators() {
+ return useLocalIterators;
+ }
+
+ /**
+ * <p>
+ * Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
+ * table's files. If the table is not offline, then the job will fail. If the table comes online during the map reduce job, it is likely that the job will
+ * fail.
+ *
+ * <p>
+ * To use this option, the map reduce user will need access to read the Accumulo directory in HDFS.
+ *
+ * <p>
+ * Reading the offline table will create the scan time iterator stack in the map process. So any iterators that are configured for the table will need to be
+ * on the mapper's classpath. The accumulo-site.xml may need to be on the mapper's classpath if HDFS or the Accumulo directory in HDFS are non-standard.
+ *
+ * <p>
+ * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
+ * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
+ * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
+ *
+ * <p>
+ * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
+ * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
+ *
+ * <p>
+ * By default, this feature is <b>disabled</b>.
+ *
+ * @param offlineScan
+ * the feature is enabled if true, disabled otherwise
+ * @since 1.6.0
+ */
+ public BatchScanConfig setOfflineScan(boolean offlineScan) {
+ this.offlineScan = offlineScan;
+ return this;
+ }
+
+ /**
+ * Determines whether a configuration has the offline table scan feature enabled.
+ *
+ * @return true if the feature is enabled, false otherwise
+ * @since 1.6.0
+ * @see #setOfflineScan(boolean)
+ */
+ public boolean isOfflineScan() {
+ return offlineScan;
+ }
+
+ /**
+ * Controls the use of the {@link org.apache.accumulo.core.client.IsolatedScanner} in this job.
+ *
+ * <p>
+ * By default, this feature is <b>disabled</b>.
+ *
+ * @param useIsolatedScanners
+ * the feature is enabled if true, disabled otherwise
+ * @since 1.6.0
+ */
+ public BatchScanConfig setUseIsolatedScanners(boolean useIsolatedScanners) {
+ this.useIsolatedScanners = useIsolatedScanners;
+ return this;
+ }
+
+ /**
+ * Determines whether a configuration has isolation enabled.
+ *
+ * @return true if the feature is enabled, false otherwise
+ * @since 1.6.0
+ * @see #setUseIsolatedScanners(boolean)
+ */
+ public boolean shouldUseIsolatedScanners() {
+ return useIsolatedScanners;
+ }
+
+ @Override
+ public void write(DataOutput dataOutput) throws IOException {
+ dataOutput.writeUTF(tableName);
+ if (iterators != null) {
+ dataOutput.writeInt(iterators.size());
+ for (IteratorSetting setting : iterators)
+ setting.write(dataOutput);
+ } else {
+ dataOutput.writeInt(0);
+ }
+ if (ranges != null) {
+ dataOutput.writeInt(ranges.size());
+ for (Range range : ranges)
+ range.write(dataOutput);
+ } else {
+ dataOutput.writeInt(0);
+ }
+ if (columns != null) {
+ dataOutput.writeInt(columns.size());
+ for (Pair<Text,Text> column : columns) {
+ if (column.getSecond() == null) {
+ dataOutput.writeInt(1);
+ column.getFirst().write(dataOutput);
+ } else {
+ dataOutput.writeInt(2);
+ column.getFirst().write(dataOutput);
+ column.getSecond().write(dataOutput);
+ }
+ }
+ } else {
+ dataOutput.writeInt(0);
+ }
+ dataOutput.writeBoolean(autoAdjustRanges);
+ dataOutput.writeBoolean(useLocalIterators);
+ dataOutput.writeBoolean(useIsolatedScanners);
+ }
+
+ @Override
+ public void readFields(DataInput dataInput) throws IOException {
+ this.tableName = dataInput.readUTF();
+ // load iterators
+ long iterSize = dataInput.readInt();
+ if (iterSize > 0)
+ iterators = new ArrayList<IteratorSetting>();
+ for (int i = 0; i < iterSize; i++)
+ iterators.add(new IteratorSetting(dataInput));
+ // load ranges
+ long rangeSize = dataInput.readInt();
+ if (rangeSize > 0)
+ ranges = new ArrayList<Range>();
+ for (int i = 0; i < rangeSize; i++) {
+ Range range = new Range();
+ range.readFields(dataInput);
+ ranges.add(range);
+ }
+ // load columns
+ long columnSize = dataInput.readInt();
+ if (columnSize > 0)
+ columns = new HashSet<Pair<Text,Text>>();
+ for (int i = 0; i < columnSize; i++) {
+ long numPairs = dataInput.readInt();
+ Text colFam = new Text();
+ colFam.readFields(dataInput);
+ if (numPairs == 1) {
+ columns.add(new Pair<Text,Text>(colFam, null));
+ } else if (numPairs == 2) {
+ Text colQual = new Text();
+ colQual.readFields(dataInput);
+ columns.add(new Pair<Text,Text>(colFam, colQual));
+ }
+ }
+ autoAdjustRanges = dataInput.readBoolean();
+ useLocalIterators = dataInput.readBoolean();
+ useIsolatedScanners = dataInput.readBoolean();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (o == null || getClass() != o.getClass())
+ return false;
+
+ BatchScanConfig that = (BatchScanConfig) o;
+
+ if (autoAdjustRanges != that.autoAdjustRanges)
+ return false;
+ if (offlineScan != that.offlineScan)
+ return false;
+ if (useIsolatedScanners != that.useIsolatedScanners)
+ return false;
+ if (useLocalIterators != that.useLocalIterators)
+ return false;
+ if (columns != null ? !columns.equals(that.columns) : that.columns != null)
+ return false;
+ if (iterators != null ? !iterators.equals(that.iterators) : that.iterators != null)
+ return false;
+ if (ranges != null ? !ranges.equals(that.ranges) : that.ranges != null)
+ return false;
+ if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null)
+ return false;
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int result = tableName != null ? tableName.hashCode() : 0;
+ result = 31 * result + (iterators != null ? iterators.hashCode() : 0);
+ result = 31 * result + (ranges != null ? ranges.hashCode() : 0);
+ result = 31 * result + (columns != null ? columns.hashCode() : 0);
+ result = 31 * result + (autoAdjustRanges ? 1 : 0);
+ result = 31 * result + (useLocalIterators ? 1 : 0);
+ result = 31 * result + (useIsolatedScanners ? 1 : 0);
+ result = 31 * result + (offlineScan ? 1 : 0);
+ return result;
+ }
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
index f75bcc9..1b76051 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
@@ -42,9 +42,9 @@ import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.TableNotFoundException;
import org.apache.accumulo.core.client.impl.Tables;
import org.apache.accumulo.core.client.impl.TabletLocator;
+import org.apache.accumulo.core.client.mapreduce.BatchScanConfig;
import org.apache.accumulo.core.client.mock.MockTabletLocator;
import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.TableQueryConfig;
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
import org.apache.accumulo.core.security.Authorizations;
@@ -480,12 +480,12 @@ public class InputConfigurator extends ConfiguratorBase {
* @param conf
* the Hadoop configuration object to configure
* @param tconf
- * an array of {@link TableQueryConfig} objects to associate with the job
+ * an array of {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} objects to associate with the job
* @since 1.6.0
*/
- public static void setTableQueryConfigs(Class<?> implementingClass, Configuration conf, TableQueryConfig... tconf) {
+ public static void setTableQueryConfigs(Class<?> implementingClass, Configuration conf, BatchScanConfig... tconf) {
List<String> tableQueryConfigStrings = new ArrayList<String>();
- for (TableQueryConfig queryConfig : tconf) {
+ for (BatchScanConfig queryConfig : tconf) {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
queryConfig.write(new DataOutputStream(baos));
@@ -499,7 +499,7 @@ public class InputConfigurator extends ConfiguratorBase {
}
/**
- * Returns all {@link TableQueryConfig} objects associated with this job.
+ * Returns all {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} objects associated with this job.
*
* @param implementingClass
* the class whose name will be used as a prefix for the property configuration key
@@ -508,22 +508,22 @@ public class InputConfigurator extends ConfiguratorBase {
* @return all of the table query configs for the job
* @since 1.6.0
*/
- public static List<TableQueryConfig> getTableQueryConfigs(Class<?> implementingClass, Configuration conf) {
- List<TableQueryConfig> configs = new ArrayList<TableQueryConfig>();
+ public static List<BatchScanConfig> getTableQueryConfigs(Class<?> implementingClass, Configuration conf) {
+ List<BatchScanConfig> configs = new ArrayList<BatchScanConfig>();
Collection<String> configStrings = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS));
if (configStrings != null) {
for (String str : configStrings) {
try {
byte[] bytes = Base64.decodeBase64(str.getBytes());
ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
- configs.add(new TableQueryConfig(new DataInputStream(bais)));
+ configs.add(new BatchScanConfig(new DataInputStream(bais)));
bais.close();
} catch (IOException e) {
throw new IllegalStateException("The table query configurations could not be deserialized from the given configuration");
}
}
}
- TableQueryConfig defaultQueryConfig;
+ BatchScanConfig defaultQueryConfig;
try {
defaultQueryConfig = getDefaultTableConfig(implementingClass, conf);
} catch (IOException e) {
@@ -536,7 +536,7 @@ public class InputConfigurator extends ConfiguratorBase {
}
/**
- * Returns the {@link TableQueryConfig} for the given table
+ * Returns the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} for the given table
*
* @param implementingClass
* the class whose name will be used as a prefix for the property configuration key
@@ -547,9 +547,9 @@ public class InputConfigurator extends ConfiguratorBase {
* @return the table query config for the given table name (if it exists) and null if it does not
* @since 1.6.0
*/
- public static TableQueryConfig getTableQueryConfig(Class<?> implementingClass, Configuration conf, String tableName) {
- List<TableQueryConfig> queryConfigs = getTableQueryConfigs(implementingClass, conf);
- for (TableQueryConfig queryConfig : queryConfigs) {
+ public static BatchScanConfig getTableQueryConfig(Class<?> implementingClass, Configuration conf, String tableName) {
+ List<BatchScanConfig> queryConfigs = getTableQueryConfigs(implementingClass, conf);
+ for (BatchScanConfig queryConfig : queryConfigs) {
if (queryConfig.getTableName().equals(tableName)) {
return queryConfig;
}
@@ -605,12 +605,12 @@ public class InputConfigurator extends ConfiguratorBase {
if (!c.securityOperations().authenticateUser(principal, token))
throw new IOException("Unable to authenticate user");
- for (TableQueryConfig tableConfig : getTableQueryConfigs(implementingClass, conf)) {
+ for (BatchScanConfig tableConfig : getTableQueryConfigs(implementingClass, conf)) {
if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), tableConfig.getTableName(), TablePermission.READ))
throw new IOException("Unable to access table");
}
- for (TableQueryConfig tableConfig : getTableQueryConfigs(implementingClass, conf)) {
+ for (BatchScanConfig tableConfig : getTableQueryConfigs(implementingClass, conf)) {
if (!tableConfig.shouldUseLocalIterators()) {
if (tableConfig.getIterators() != null) {
for (IteratorSetting iter : tableConfig.getIterators()) {
@@ -631,7 +631,7 @@ public class InputConfigurator extends ConfiguratorBase {
}
/**
- * Returns the {@link TableQueryConfig} for the configuration based on the properties set using the single-table input methods.
+ * Returns the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} for the configuration based on the properties set using the single-table input methods.
*
* @param implementingClass
* the class whose name will be used as a prefix for the property configuration key
@@ -641,10 +641,10 @@ public class InputConfigurator extends ConfiguratorBase {
* @throws IOException
* @since 1.6.0
*/
- protected static TableQueryConfig getDefaultTableConfig(Class<?> implementingClass, Configuration conf) throws IOException {
+ protected static BatchScanConfig getDefaultTableConfig(Class<?> implementingClass, Configuration conf) throws IOException {
String tableName = getInputTableName(implementingClass, conf);
if (tableName != null) {
- TableQueryConfig queryConfig = new TableQueryConfig(getInputTableName(implementingClass, conf));
+ BatchScanConfig queryConfig = new BatchScanConfig(getInputTableName(implementingClass, conf));
List<IteratorSetting> itrs = getIterators(implementingClass, conf);
if (itrs != null)
queryConfig.setIterators(itrs);
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/main/java/org/apache/accumulo/core/conf/TableQueryConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/TableQueryConfig.java b/core/src/main/java/org/apache/accumulo/core/conf/TableQueryConfig.java
deleted file mode 100644
index a5278fa..0000000
--- a/core/src/main/java/org/apache/accumulo/core/conf/TableQueryConfig.java
+++ /dev/null
@@ -1,369 +0,0 @@
-/*
- * 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.accumulo.core.conf;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-/**
- * This class to holds a query configuration for a table. It contains all the properties needed to specify how rows should be returned from the table.
- */
-public class TableQueryConfig implements Writable {
-
- private String tableName;
- private List<IteratorSetting> iterators;
- private List<Range> ranges;
- private Set<Pair<Text,Text>> columns;
-
- private boolean autoAdjustRanges = true;
- private boolean useLocalIterators = false;
- private boolean useIsolatedScanners = false;
- private boolean offlineScan = false;
-
- public TableQueryConfig(String tableName) {
- checkNotNull(tableName);
- this.tableName = tableName;
- }
-
- public TableQueryConfig(DataInput input) throws IOException {
- readFields(input);
- }
-
- /**
- * Returns the table name associated with this configuration
- */
- public String getTableName() {
- return tableName;
- }
-
- /**
- * Sets the input ranges to scan for all tables associated with this job. This will be added to any per-table ranges that have been set using
- *
- * @param ranges
- * the ranges that will be mapped over
- * @since 1.6.0
- */
- public TableQueryConfig setRanges(List<Range> ranges) {
- this.ranges = ranges;
- return this;
- }
-
- /**
- * Returns the ranges to be queried in the configuration
- */
- public List<Range> getRanges() {
- return ranges != null ? ranges : new ArrayList<Range>();
- }
-
- /**
- * Restricts the columns that will be mapped over for this job for the default input table.
- *
- * @param columns
- * a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
- * selected. An empty set is the default and is equivalent to scanning the all columns.
- * @since 1.6.0
- */
- public TableQueryConfig fetchColumns(Set<Pair<Text,Text>> columns) {
- this.columns = columns;
- return this;
- }
-
- /**
- * Returns the columns to be fetched for this configuration
- */
- public Set<Pair<Text,Text>> getFetchedColumns() {
- return columns != null ? columns : new HashSet<Pair<Text,Text>>();
- }
-
- /**
- * Set iterators on to be used in the query.
- *
- * @param iterators
- * the configurations for the iterators
- * @since 1.6.0
- */
- public TableQueryConfig setIterators(List<IteratorSetting> iterators) {
- this.iterators = iterators;
- return this;
- }
-
- /**
- * Returns the iterators to be set on this configuration
- */
- public List<IteratorSetting> getIterators() {
- return iterators != null ? iterators : new ArrayList<IteratorSetting>();
- }
-
- /**
- * Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
- * Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
- *
- * <p>
- * By default, this feature is <b>enabled</b>.
- *
- * @param autoAdjustRanges
- * the feature is enabled if true, disabled otherwise
- * @see #setRanges(java.util.List)
- * @since 1.6.0
- */
- public TableQueryConfig setAutoAdjustRanges(boolean autoAdjustRanges) {
- this.autoAdjustRanges = autoAdjustRanges;
- return this;
- }
-
- /**
- * Determines whether a configuration has auto-adjust ranges enabled.
- *
- * @return false if the feature is disabled, true otherwise
- * @since 1.6.0
- * @see #setAutoAdjustRanges(boolean)
- */
- public boolean shouldAutoAdjustRanges() {
- return autoAdjustRanges;
- }
-
- /**
- * Controls the use of the {@link org.apache.accumulo.core.client.ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack
- * to be constructed within the Map task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be
- * available on the classpath for the task.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param useLocalIterators
- * the feature is enabled if true, disabled otherwise
- * @since 1.6.0
- */
- public TableQueryConfig setUseLocalIterators(boolean useLocalIterators) {
- this.useLocalIterators = useLocalIterators;
- return this;
- }
-
- /**
- * Determines whether a configuration uses local iterators.
- *
- * @return true if the feature is enabled, false otherwise
- * @since 1.6.0
- * @see #setUseLocalIterators(boolean)
- * @deprecated since 1.6.0
- */
- public boolean shouldUseLocalIterators() {
- return useLocalIterators;
- }
-
- /**
- * <p>
- * Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
- * table's files. If the table is not offline, then the job will fail. If the table comes online during the map reduce job, it is likely that the job will
- * fail.
- *
- * <p>
- * To use this option, the map reduce user will need access to read the Accumulo directory in HDFS.
- *
- * <p>
- * Reading the offline table will create the scan time iterator stack in the map process. So any iterators that are configured for the table will need to be
- * on the mapper's classpath. The accumulo-site.xml may need to be on the mapper's classpath if HDFS or the Accumulo directory in HDFS are non-standard.
- *
- * <p>
- * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
- * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
- * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
- *
- * <p>
- * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
- * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param offlineScan
- * the feature is enabled if true, disabled otherwise
- * @since 1.6.0
- */
- public TableQueryConfig setOfflineScan(boolean offlineScan) {
- this.offlineScan = offlineScan;
- return this;
- }
-
- /**
- * Determines whether a configuration has the offline table scan feature enabled.
- *
- * @return true if the feature is enabled, false otherwise
- * @since 1.6.0
- * @see #setOfflineScan(boolean)
- */
- public boolean isOfflineScan() {
- return offlineScan;
- }
-
- /**
- * Controls the use of the {@link org.apache.accumulo.core.client.IsolatedScanner} in this job.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param useIsolatedScanners
- * the feature is enabled if true, disabled otherwise
- * @since 1.6.0
- */
- public TableQueryConfig setUseIsolatedScanners(boolean useIsolatedScanners) {
- this.useIsolatedScanners = useIsolatedScanners;
- return this;
- }
-
- /**
- * Determines whether a configuration has isolation enabled.
- *
- * @return true if the feature is enabled, false otherwise
- * @since 1.6.0
- * @see #setUseIsolatedScanners(boolean)
- */
- public boolean shouldUseIsolatedScanners() {
- return useIsolatedScanners;
- }
-
- @Override
- public void write(DataOutput dataOutput) throws IOException {
- dataOutput.writeUTF(tableName);
- if (iterators != null) {
- dataOutput.writeInt(iterators.size());
- for (IteratorSetting setting : iterators)
- setting.write(dataOutput);
- } else {
- dataOutput.writeInt(0);
- }
- if (ranges != null) {
- dataOutput.writeInt(ranges.size());
- for (Range range : ranges)
- range.write(dataOutput);
- } else {
- dataOutput.writeInt(0);
- }
- if (columns != null) {
- dataOutput.writeInt(columns.size());
- for (Pair<Text,Text> column : columns) {
- if (column.getSecond() == null) {
- dataOutput.writeInt(1);
- column.getFirst().write(dataOutput);
- } else {
- dataOutput.writeInt(2);
- column.getFirst().write(dataOutput);
- column.getSecond().write(dataOutput);
- }
- }
- } else {
- dataOutput.writeInt(0);
- }
- dataOutput.writeBoolean(autoAdjustRanges);
- dataOutput.writeBoolean(useLocalIterators);
- dataOutput.writeBoolean(useIsolatedScanners);
- }
-
- @Override
- public void readFields(DataInput dataInput) throws IOException {
- this.tableName = dataInput.readUTF();
- // load iterators
- long iterSize = dataInput.readInt();
- if (iterSize > 0)
- iterators = new ArrayList<IteratorSetting>();
- for (int i = 0; i < iterSize; i++)
- iterators.add(new IteratorSetting(dataInput));
- // load ranges
- long rangeSize = dataInput.readInt();
- if (rangeSize > 0)
- ranges = new ArrayList<Range>();
- for (int i = 0; i < rangeSize; i++) {
- Range range = new Range();
- range.readFields(dataInput);
- ranges.add(range);
- }
- // load columns
- long columnSize = dataInput.readInt();
- if (columnSize > 0)
- columns = new HashSet<Pair<Text,Text>>();
- for (int i = 0; i < columnSize; i++) {
- long numPairs = dataInput.readInt();
- Text colFam = new Text();
- colFam.readFields(dataInput);
- if (numPairs == 1) {
- columns.add(new Pair<Text,Text>(colFam, null));
- } else if (numPairs == 2) {
- Text colQual = new Text();
- colQual.readFields(dataInput);
- columns.add(new Pair<Text,Text>(colFam, colQual));
- }
- }
- autoAdjustRanges = dataInput.readBoolean();
- useLocalIterators = dataInput.readBoolean();
- useIsolatedScanners = dataInput.readBoolean();
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o)
- return true;
- if (o == null || getClass() != o.getClass())
- return false;
-
- TableQueryConfig that = (TableQueryConfig) o;
-
- if (autoAdjustRanges != that.autoAdjustRanges)
- return false;
- if (offlineScan != that.offlineScan)
- return false;
- if (useIsolatedScanners != that.useIsolatedScanners)
- return false;
- if (useLocalIterators != that.useLocalIterators)
- return false;
- if (columns != null ? !columns.equals(that.columns) : that.columns != null)
- return false;
- if (iterators != null ? !iterators.equals(that.iterators) : that.iterators != null)
- return false;
- if (ranges != null ? !ranges.equals(that.ranges) : that.ranges != null)
- return false;
- if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null)
- return false;
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int result = tableName != null ? tableName.hashCode() : 0;
- result = 31 * result + (iterators != null ? iterators.hashCode() : 0);
- result = 31 * result + (ranges != null ? ranges.hashCode() : 0);
- result = 31 * result + (columns != null ? columns.hashCode() : 0);
- result = 31 * result + (autoAdjustRanges ? 1 : 0);
- result = 31 * result + (useLocalIterators ? 1 : 0);
- result = 31 * result + (useIsolatedScanners ? 1 : 0);
- result = 31 * result + (offlineScan ? 1 : 0);
- return result;
- }
-}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
index d440a6c..70cf60d 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
@@ -30,9 +30,9 @@ import org.apache.accumulo.core.client.BatchWriter;
import org.apache.accumulo.core.client.BatchWriterConfig;
import org.apache.accumulo.core.client.Connector;
import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.mapreduce.BatchScanConfig;
import org.apache.accumulo.core.client.mock.MockInstance;
import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.TableQueryConfig;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.Range;
@@ -254,8 +254,8 @@ public class AccumuloInputFormatTest {
AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
- TableQueryConfig tableConfig1 = new TableQueryConfig(table1);
- TableQueryConfig tableConfig2 = new TableQueryConfig(table2);
+ BatchScanConfig tableConfig1 = new BatchScanConfig(table1);
+ BatchScanConfig tableConfig2 = new BatchScanConfig(table2);
AccumuloInputFormat.setTableQueryConfigs(job, tableConfig1, tableConfig2);
@@ -299,18 +299,18 @@ public class AccumuloInputFormatTest {
}
/**
- * Verify {@link org.apache.accumulo.core.conf.TableQueryConfig} objects get correctly serialized in the JobContext.
+ * Verify {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} objects get correctly serialized in the JobContext.
*/
@Test
public void testTableQueryConfigSerialization() throws IOException {
JobConf job = new JobConf();
- TableQueryConfig table1 = new TableQueryConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
+ BatchScanConfig table1 = new BatchScanConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
.fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
.setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
- TableQueryConfig table2 = new TableQueryConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
+ BatchScanConfig table2 = new BatchScanConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
.fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
.setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
@@ -328,11 +328,11 @@ public class AccumuloInputFormatTest {
JobConf job = new JobConf();
- TableQueryConfig table1 = new TableQueryConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
+ BatchScanConfig table1 = new BatchScanConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
.fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
.setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
- TableQueryConfig table2 = new TableQueryConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
+ BatchScanConfig table2 = new BatchScanConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
.fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
.setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
index b3b6d8b..6b2eba1 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
@@ -6,7 +6,6 @@ import org.apache.accumulo.core.client.Connector;
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.mock.MockInstance;
import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.TableQueryConfig;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.Range;
@@ -88,8 +87,8 @@ public class AccumuloMultiTableInputFormatTest {
AccumuloMultiTableInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
- TableQueryConfig tableConfig1 = new TableQueryConfig(table1);
- TableQueryConfig tableConfig2 = new TableQueryConfig(table2);
+ BatchScanConfig tableConfig1 = new BatchScanConfig(table1);
+ BatchScanConfig tableConfig2 = new BatchScanConfig(table2);
AccumuloMultiTableInputFormat.setBatchScanConfigs(job, tableConfig1, tableConfig2);
AccumuloMultiTableInputFormat.setMockInstance(job, INSTANCE_NAME);
@@ -139,18 +138,18 @@ public class AccumuloMultiTableInputFormatTest {
}
/**
- * Verify {@link TableQueryConfig} objects get correctly serialized in the JobContext.
+ * Verify {@link BatchScanConfig} objects get correctly serialized in the JobContext.
*/
@Test
public void testTableQueryConfigSerialization() throws IOException {
Job job = new Job();
- TableQueryConfig table1 = new TableQueryConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
+ BatchScanConfig table1 = new BatchScanConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
.fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
.setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
- TableQueryConfig table2 = new TableQueryConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
+ BatchScanConfig table2 = new BatchScanConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
.fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
.setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
http://git-wip-us.apache.org/repos/asf/accumulo/blob/87b104d2/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java b/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
index ed5469d..9910182 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
@@ -29,6 +29,7 @@ import java.util.List;
import java.util.Set;
import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.mapreduce.BatchScanConfig;
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.util.Pair;
import org.apache.hadoop.io.Text;
@@ -38,17 +39,17 @@ import org.junit.Test;
public class TableQueryConfigTest {
private static final String TEST_TABLE = "TEST_TABLE";
- private TableQueryConfig tableQueryConfig;
+ private BatchScanConfig tableQueryConfig;
@Before
public void setUp() {
- tableQueryConfig = new TableQueryConfig(TEST_TABLE);
+ tableQueryConfig = new BatchScanConfig(TEST_TABLE);
}
@Test
public void testSerialization_OnlyTable() throws IOException {
byte[] serialized = serialize(tableQueryConfig);
- TableQueryConfig actualConfig = deserialize(serialized);
+ BatchScanConfig actualConfig = deserialize(serialized);
assertEquals(tableQueryConfig, actualConfig);
}
@@ -61,7 +62,7 @@ public class TableQueryConfigTest {
tableQueryConfig.setRanges(ranges);
byte[] serialized = serialize(tableQueryConfig);
- TableQueryConfig actualConfig = deserialize(serialized);
+ BatchScanConfig actualConfig = deserialize(serialized);
assertEquals(ranges, actualConfig.getRanges());
}
@@ -74,7 +75,7 @@ public class TableQueryConfigTest {
tableQueryConfig.fetchColumns(columns);
byte[] serialized = serialize(tableQueryConfig);
- TableQueryConfig actualConfig = deserialize(serialized);
+ BatchScanConfig actualConfig = deserialize(serialized);
assertEquals(actualConfig.getFetchedColumns(), columns);
}
@@ -86,21 +87,21 @@ public class TableQueryConfigTest {
settings.add(new IteratorSetting(55, "iter2", "iterclass2"));
tableQueryConfig.setIterators(settings);
byte[] serialized = serialize(tableQueryConfig);
- TableQueryConfig actualConfig = deserialize(serialized);
+ BatchScanConfig actualConfig = deserialize(serialized);
assertEquals(actualConfig.getIterators(), settings);
}
- private byte[] serialize(TableQueryConfig tableQueryConfig) throws IOException {
+ private byte[] serialize(BatchScanConfig tableQueryConfig) throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
tableQueryConfig.write(new DataOutputStream(baos));
baos.close();
return baos.toByteArray();
}
- private TableQueryConfig deserialize(byte[] bytes) throws IOException {
+ private BatchScanConfig deserialize(byte[] bytes) throws IOException {
ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
- TableQueryConfig actualConfig = new TableQueryConfig(new DataInputStream(bais));
+ BatchScanConfig actualConfig = new BatchScanConfig(new DataInputStream(bais));
bais.close();
return actualConfig;
}
[2/5] git commit: ACCUMULO-391 Adding AccumuloMultiTableInputFormat
and tests. Reverting AccumuloInputFormatTest back to pre-multi-table version.
Posted by cj...@apache.org.
ACCUMULO-391 Adding AccumuloMultiTableInputFormat and tests. Reverting AccumuloInputFormatTest back to pre-multi-table version.
Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/5c496552
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/5c496552
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/5c496552
Branch: refs/heads/master
Commit: 5c496552e024817bbd197514c3a37c84b6e28f50
Parents: 6114639
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Sun Oct 13 00:14:38 2013 -0400
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Tue Oct 15 20:21:45 2013 -0400
----------------------------------------------------------------------
.../client/mapreduce/AbstractInputFormat.java | 804 +++++++++++++++++++
.../AccumuloMultiTableInputFormat.java | 53 ++
.../core/client/mapreduce/InputFormatBase.java | 796 +-----------------
.../mapreduce/AccumuloInputFormatTest.java | 89 +-
.../AccumuloMultiTableInputFormatTest.java | 163 ++++
5 files changed, 1035 insertions(+), 870 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/5c496552/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
new file mode 100644
index 0000000..6568f35
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -0,0 +1,804 @@
+package org.apache.accumulo.core.client.mapreduce;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientSideIteratorScanner;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IsolatedScanner;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.RowIterator;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableDeletedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.impl.OfflineScanner;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.TabletLocator;
+import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.conf.TableQueryConfig;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyExtent;
+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.master.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
+
+ protected static final Class<?> CLASS = AccumuloInputFormat.class;
+ protected static final Logger log = Logger.getLogger(CLASS);
+
+ /**
+ * Sets the connector information needed to communicate with Accumulo in this job.
+ *
+ * <p>
+ * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
+ * conversion to a string, and is not intended to be secure.
+ *
+ * @param job
+ * the Hadoop job instance to be configured
+ * @param principal
+ * a valid Accumulo user name (user must have Table.CREATE permission)
+ * @param token
+ * the user's password
+ * @throws org.apache.accumulo.core.client.AccumuloSecurityException
+ * @since 1.5.0
+ */
+ public static void setConnectorInfo(Job job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
+ InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
+ }
+
+ /**
+ * Sets the connector information needed to communicate with Accumulo in this job.
+ *
+ * <p>
+ * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt to be more secure than storing it in the Configuration.
+ *
+ * @param job
+ * the Hadoop job instance to be configured
+ * @param principal
+ * a valid Accumulo user name (user must have Table.CREATE permission)
+ * @param tokenFile
+ * the path to the token file
+ * @throws AccumuloSecurityException
+ * @since 1.6.0
+ */
+ public static void setConnectorInfo(Job job, String principal, String tokenFile) throws AccumuloSecurityException {
+ InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
+ }
+
+ /**
+ * Determines if the connector has been configured.
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @return true if the connector has been configured, false otherwise
+ * @since 1.5.0
+ * @see #setConnectorInfo(Job, String, AuthenticationToken)
+ */
+ protected static Boolean isConnectorInfoSet(JobContext context) {
+ return InputConfigurator.isConnectorInfoSet(CLASS, getConfiguration(context));
+ }
+
+ /**
+ * Gets the user name from the configuration.
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @return the user name
+ * @since 1.5.0
+ * @see #setConnectorInfo(Job, String, AuthenticationToken)
+ */
+ protected static String getPrincipal(JobContext context) {
+ return InputConfigurator.getPrincipal(CLASS, getConfiguration(context));
+ }
+
+ /**
+ * Gets the serialized token class from either the configuration or the token file.
+ *
+ * @since 1.5.0
+ * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
+ */
+ @Deprecated
+ protected static String getTokenClass(JobContext context) {
+ return getAuthenticationToken(context).getClass().getName();
+ }
+
+ /**
+ * Gets the serialized token from either the configuration or the token file.
+ *
+ * @since 1.5.0
+ * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
+ */
+ @Deprecated
+ protected static byte[] getToken(JobContext context) {
+ return AuthenticationToken.AuthenticationTokenSerializer.serialize(getAuthenticationToken(context));
+ }
+
+ /**
+ * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @return the principal's authentication token
+ * @since 1.6.0
+ * @see #setConnectorInfo(Job, String, AuthenticationToken)
+ * @see #setConnectorInfo(Job, String, String)
+ */
+ protected static AuthenticationToken getAuthenticationToken(JobContext context) {
+ return InputConfigurator.getAuthenticationToken(CLASS, getConfiguration(context));
+ }
+
+ /**
+ * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
+ *
+ * @param job
+ * the Hadoop job instance to be configured
+ * @param instanceName
+ * the Accumulo instance name
+ * @param zooKeepers
+ * a comma-separated list of zookeeper servers
+ * @since 1.5.0
+ */
+ public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
+ InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), instanceName, zooKeepers);
+ }
+
+ /**
+ * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
+ *
+ * @param job
+ * the Hadoop job instance to be configured
+ * @param instanceName
+ * the Accumulo instance name
+ * @since 1.5.0
+ */
+ public static void setMockInstance(Job job, String instanceName) {
+ InputConfigurator.setMockInstance(CLASS, job.getConfiguration(), instanceName);
+ }
+
+ /**
+ * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the configuration.
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @return an Accumulo instance
+ * @since 1.5.0
+ * @see #setZooKeeperInstance(Job, String, String)
+ * @see #setMockInstance(Job, String)
+ */
+ protected static Instance getInstance(JobContext context) {
+ return InputConfigurator.getInstance(CLASS, getConfiguration(context));
+ }
+
+ /**
+ * Sets the log level for this job.
+ *
+ * @param job
+ * the Hadoop job instance to be configured
+ * @param level
+ * the logging level
+ * @since 1.5.0
+ */
+ public static void setLogLevel(Job job, Level level) {
+ InputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
+ }
+
+ /**
+ * Gets the log level from this configuration.
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @return the log level
+ * @since 1.5.0
+ * @see #setLogLevel(Job, Level)
+ */
+ protected static Level getLogLevel(JobContext context) {
+ return InputConfigurator.getLogLevel(CLASS, getConfiguration(context));
+ }
+
+ /**
+ * Sets the {@link org.apache.accumulo.core.security.Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
+ *
+ * @param job
+ * the Hadoop job instance to be configured
+ * @param auths
+ * the user's authorizations
+ */
+ public static void setScanAuthorizations(Job job, Authorizations auths) {
+ InputConfigurator.setScanAuthorizations(CLASS, job.getConfiguration(), auths);
+ }
+
+ /**
+ * Gets the authorizations to set for the scans from the configuration.
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @return the Accumulo scan authorizations
+ * @since 1.5.0
+ * @see #setScanAuthorizations(Job, Authorizations)
+ */
+ protected static Authorizations getScanAuthorizations(JobContext context) {
+ return InputConfigurator.getScanAuthorizations(CLASS, getConfiguration(context));
+ }
+
+ /**
+ * Fetches all {@link TableQueryConfig}s that have been set on the given Hadoop configuration.
+ *
+ * @param job
+ * the Hadoop job instance to be configured
+ * @return
+ * @since 1.6.0
+ */
+ protected static List<TableQueryConfig> getBatchScanConfigs(JobContext job) {
+ return InputConfigurator.getTableQueryConfigs(CLASS, getConfiguration(job));
+ }
+
+ /**
+ * Fetches a {@link TableQueryConfig} that has been set on the configuration for a specific table.
+ *
+ * <p>
+ * null is returned in the event that the table doesn't exist.
+ *
+ * @param job
+ * the Hadoop job instance to be configured
+ * @param tableName
+ * the table name for which to grab the config object
+ * @return the {@link TableQueryConfig} for the given table
+ * @since 1.6.0
+ */
+ protected static TableQueryConfig getBatchScanConfig(JobContext job, String tableName) {
+ return InputConfigurator.getTableQueryConfig(CLASS, getConfiguration(job), tableName);
+ }
+
+ /**
+ * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @param table
+ * the table for which to initialize the locator
+ * @return an Accumulo tablet locator
+ * @throws org.apache.accumulo.core.client.TableNotFoundException
+ * if the table name set on the configuration doesn't exist
+ * @since 1.6.0
+ */
+ protected static TabletLocator getTabletLocator(JobContext context, String table) throws TableNotFoundException {
+ return InputConfigurator.getTabletLocator(CLASS, getConfiguration(context), table);
+ }
+
+ // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
+ /**
+ * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @throws java.io.IOException
+ * if the context is improperly configured
+ * @since 1.5.0
+ */
+ protected static void validateOptions(JobContext context) throws IOException {
+ InputConfigurator.validateOptions(CLASS, getConfiguration(context));
+ }
+
+ /**
+ * An abstract base class to be used to create {@link org.apache.hadoop.mapreduce.RecordReader} instances that convert from Accumulo
+ * {@link org.apache.accumulo.core.data.Key}/{@link org.apache.accumulo.core.data.Value} pairs to the user's K/V types.
+ *
+ * Subclasses must implement {@link #nextKeyValue()} and use it to update the following variables:
+ * <ul>
+ * <li>K {@link #currentK}</li>
+ * <li>V {@link #currentV}</li>
+ * <li>Key {@link #currentKey} (used for progress reporting)</li>
+ * <li>int {@link #numKeysRead} (used for progress reporting)</li>
+ * </ul>
+ */
+ protected abstract static class RecordReaderBase<K,V> extends RecordReader<K,V> {
+ protected long numKeysRead;
+ protected Iterator<Map.Entry<Key,Value>> scannerIterator;
+ protected RangeInputSplit split;
+
+ /**
+ * Apply the configured iterators from the configuration to the scanner. This applies both the default iterators and the per-table iterators.
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @param scanner
+ * the scanner to configure
+ * @param tableName
+ * the table name for which to set up the iterators
+ */
+ protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName) {
+ TableQueryConfig config = getBatchScanConfig(context, tableName);
+ List<IteratorSetting> iterators = config.getIterators();
+ for (IteratorSetting iterator : iterators)
+ scanner.addScanIterator(iterator);
+ }
+
+ /**
+ * Initialize a scanner over the given input split using this task attempt configuration.
+ */
+ @Override
+ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
+
+ Scanner scanner;
+ split = (RangeInputSplit) inSplit;
+ log.debug("Initializing input split: " + split.getRange());
+ Instance instance = getInstance(attempt);
+ String principal = getPrincipal(attempt);
+
+ TableQueryConfig tableConfig = getBatchScanConfig(attempt, split.getTableName());
+
+ // in case the table name changed, we can still use the previous name for terms of configuration,
+ // but for the scanner, we'll need to reference the new table name.
+ String actualNameForId = split.getTableName();
+ if (!(instance instanceof MockInstance)) {
+ try {
+ actualNameForId = Tables.getTableName(instance, split.getTableId());
+ if (!actualNameForId.equals(split.getTableName()))
+ log.debug("Table name changed from " + split.getTableName() + " to " + actualNameForId);
+ } catch (TableNotFoundException e) {
+ throw new IOException("The specified table was not found for id=" + split.getTableId());
+ }
+ }
+
+ AuthenticationToken token = getAuthenticationToken(attempt);
+ Authorizations authorizations = getScanAuthorizations(attempt);
+ try {
+ log.debug("Creating connector with user: " + principal);
+
+ Connector conn = instance.getConnector(principal, token);
+ log.debug("Creating scanner for table: " + split.getTableName());
+ log.debug("Authorizations are: " + authorizations);
+ if (tableConfig.isOfflineScan()) {
+ scanner = new OfflineScanner(instance, new Credentials(principal, token), split.getTableId(), authorizations);
+ } else {
+ scanner = conn.createScanner(actualNameForId, authorizations);
+ }
+ if (tableConfig.shouldUseIsolatedScanners()) {
+ log.info("Creating isolated scanner");
+ scanner = new IsolatedScanner(scanner);
+ }
+ if (tableConfig.shouldUseLocalIterators()) {
+ log.info("Using local iterators");
+ scanner = new ClientSideIteratorScanner(scanner);
+ }
+ setupIterators(attempt, scanner, split.getTableName());
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+
+ // setup a scanner within the bounds of this split
+ for (Pair<Text,Text> c : tableConfig.getFetchedColumns()) {
+ if (c.getSecond() != null) {
+ log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
+ scanner.fetchColumn(c.getFirst(), c.getSecond());
+ } else {
+ log.debug("Fetching column family " + c.getFirst());
+ scanner.fetchColumnFamily(c.getFirst());
+ }
+ }
+
+ scanner.setRange(split.getRange());
+ numKeysRead = 0;
+
+ // do this last after setting all scanner options
+ scannerIterator = scanner.iterator();
+ }
+
+ @Override
+ public void close() {}
+
+ @Override
+ public float getProgress() throws IOException {
+ if (numKeysRead > 0 && currentKey == null)
+ return 1.0f;
+ return split.getProgress(currentKey);
+ }
+
+ protected K currentK = null;
+ protected V currentV = null;
+ protected Key currentKey = null;
+ protected Value currentValue = null;
+
+ @Override
+ public K getCurrentKey() throws IOException, InterruptedException {
+ return currentK;
+ }
+
+ @Override
+ public V getCurrentValue() throws IOException, InterruptedException {
+ return currentV;
+ }
+ }
+
+ Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext context, String tableName, List<Range> ranges) throws TableNotFoundException,
+ AccumuloException, AccumuloSecurityException {
+
+ Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
+
+ Instance instance = getInstance(context);
+ Connector conn = instance.getConnector(getPrincipal(context), getAuthenticationToken(context));
+ String tableId = Tables.getTableId(instance, tableName);
+
+ if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
+ Tables.clearCache(instance);
+ if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
+ throw new AccumuloException("Table is online " + tableName + "(" + tableId + ") cannot scan table in offline mode ");
+ }
+ }
+
+ for (Range range : ranges) {
+ Text startRow;
+
+ if (range.getStartKey() != null)
+ startRow = range.getStartKey().getRow();
+ else
+ startRow = new Text();
+
+ Range metadataRange = new Range(new KeyExtent(new Text(tableId), startRow, null).getMetadataEntry(), true, null, false);
+ Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+ MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
+ scanner.fetchColumnFamily(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME);
+ scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
+ scanner.fetchColumnFamily(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME);
+ scanner.setRange(metadataRange);
+
+ RowIterator rowIter = new RowIterator(scanner);
+ KeyExtent lastExtent = null;
+ while (rowIter.hasNext()) {
+ Iterator<Map.Entry<Key,Value>> row = rowIter.next();
+ String last = "";
+ KeyExtent extent = null;
+ String location = null;
+
+ while (row.hasNext()) {
+ Map.Entry<Key,Value> entry = row.next();
+ Key key = entry.getKey();
+
+ if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME)) {
+ last = entry.getValue().toString();
+ }
+
+ if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME)
+ || key.getColumnFamily().equals(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME)) {
+ location = entry.getValue().toString();
+ }
+
+ if (MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
+ extent = new KeyExtent(key.getRow(), entry.getValue());
+ }
+
+ }
+
+ if (location != null)
+ return null;
+
+ if (!extent.getTableId().toString().equals(tableId)) {
+ throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
+ }
+
+ if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
+ throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
+ }
+
+ Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
+ if (tabletRanges == null) {
+ tabletRanges = new HashMap<KeyExtent,List<Range>>();
+ binnedRanges.put(last, tabletRanges);
+ }
+
+ List<Range> rangeList = tabletRanges.get(extent);
+ if (rangeList == null) {
+ rangeList = new ArrayList<Range>();
+ tabletRanges.put(extent, rangeList);
+ }
+
+ rangeList.add(range);
+
+ if (extent.getEndRow() == null || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
+ break;
+ }
+
+ lastExtent = extent;
+ }
+
+ }
+
+ return binnedRanges;
+ }
+
+ /**
+ * Gets the splits of the tables that have been set on the job.
+ *
+ * @param conf
+ * the configuration of the job
+ * @return the splits from the tables based on the ranges.
+ * @throws java.io.IOException
+ * if a table set on the job doesn't exist or an error occurs initializing the tablet locator
+ */
+ public List<InputSplit> getSplits(JobContext conf) throws IOException {
+ log.setLevel(getLogLevel(conf));
+ validateOptions(conf);
+
+ LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
+ List<TableQueryConfig> tableConfigs = getBatchScanConfigs(conf);
+ for (TableQueryConfig tableConfig : tableConfigs) {
+
+ boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
+ String tableId = null;
+ List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
+ if (ranges.isEmpty()) {
+ ranges = new ArrayList<Range>(1);
+ ranges.add(new Range());
+ }
+
+ // get the metadata information for these ranges
+ Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
+ TabletLocator tl;
+ try {
+ if (tableConfig.isOfflineScan()) {
+ binnedRanges = binOfflineTable(conf, tableConfig.getTableName(), ranges);
+ while (binnedRanges == null) {
+ // Some tablets were still online, try again
+ UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
+ binnedRanges = binOfflineTable(conf, tableConfig.getTableName(), ranges);
+
+ }
+ } else {
+ Instance instance = getInstance(conf);
+ tl = getTabletLocator(conf, tableConfig.getTableName());
+ // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
+ tl.invalidateCache();
+ Credentials creds = new Credentials(getPrincipal(conf), getAuthenticationToken(conf));
+
+ while (!tl.binRanges(creds, ranges, binnedRanges).isEmpty()) {
+ if (!(instance instanceof MockInstance)) {
+ if (!Tables.exists(instance, tableId))
+ throw new TableDeletedException(tableId);
+ if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
+ throw new TableOfflineException(instance, tableId);
+ tableId = Tables.getTableId(instance, tableConfig.getTableName());
+ }
+ binnedRanges.clear();
+ log.warn("Unable to locate bins for specified ranges. Retrying.");
+ UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
+ tl.invalidateCache();
+ }
+ }
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+
+ HashMap<Range,ArrayList<String>> splitsToAdd = null;
+
+ if (!autoAdjust)
+ splitsToAdd = new HashMap<Range,ArrayList<String>>();
+
+ HashMap<String,String> hostNameCache = new HashMap<String,String>();
+ for (Map.Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
+ String ip = tserverBin.getKey().split(":", 2)[0];
+ String location = hostNameCache.get(ip);
+ if (location == null) {
+ InetAddress inetAddress = InetAddress.getByName(ip);
+ location = inetAddress.getHostName();
+ hostNameCache.put(ip, location);
+ }
+ for (Map.Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
+ Range ke = extentRanges.getKey().toDataRange();
+ for (Range r : extentRanges.getValue()) {
+ if (autoAdjust) {
+ // divide ranges into smaller ranges, based on the tablets
+ splits.add(new RangeInputSplit(tableConfig.getTableName(), tableId, ke.clip(r), new String[] {location}));
+ } else {
+ // don't divide ranges
+ ArrayList<String> locations = splitsToAdd.get(r);
+ if (locations == null)
+ locations = new ArrayList<String>(1);
+ locations.add(location);
+ splitsToAdd.put(r, locations);
+ }
+ }
+ }
+ }
+
+ if (!autoAdjust)
+ for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
+ splits.add(new RangeInputSplit(tableConfig.getTableName(), tableId, entry.getKey(), entry.getValue().toArray(new String[0])));
+ }
+ return splits;
+ }
+
+ /**
+ * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
+ */
+ public static class RangeInputSplit extends InputSplit implements Writable {
+ private Range range;
+ private String[] locations;
+ private String tableId;
+ private String tableName;
+
+ public RangeInputSplit() {
+ range = new Range();
+ locations = new String[0];
+ tableId = "";
+ tableName = "";
+ }
+
+ public RangeInputSplit(RangeInputSplit split) throws IOException {
+ this.setRange(split.getRange());
+ this.setLocations(split.getLocations());
+ this.setTableName(split.getTableName());
+ }
+
+ protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
+ this.range = range;
+ this.locations = locations;
+ this.tableName = table;
+ this.tableId = tableId;
+ }
+
+ public Range getRange() {
+ return range;
+ }
+
+ public void setRange(Range range) {
+ this.range = range;
+ }
+
+ public String getTableName() {
+ return tableName;
+ }
+
+ public void setTableName(String tableName) {
+ this.tableName = tableName;
+ }
+
+ public void setTableId(String tableId) {
+ this.tableId = tableId;
+ }
+
+ public String getTableId() {
+ return tableId;
+ }
+
+ private static byte[] extractBytes(ByteSequence seq, int numBytes) {
+ byte[] bytes = new byte[numBytes + 1];
+ bytes[0] = 0;
+ for (int i = 0; i < numBytes; i++) {
+ if (i >= seq.length())
+ bytes[i + 1] = 0;
+ else
+ bytes[i + 1] = seq.byteAt(i);
+ }
+ return bytes;
+ }
+
+ public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
+ int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
+ BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
+ BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
+ BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
+ return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
+ }
+
+ public float getProgress(Key currentKey) {
+ if (currentKey == null)
+ return 0f;
+ if (range.getStartKey() != null && range.getEndKey() != null) {
+ if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW)) {
+ // just look at the row progress
+ return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
+ } else if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW_COLFAM)) {
+ // just look at the column family progress
+ return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
+ } else if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL)) {
+ // just look at the column qualifier progress
+ return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
+ }
+ }
+ // if we can't figure it out, then claim no progress
+ return 0f;
+ }
+
+ /**
+ * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
+ */
+ @Override
+ public long getLength() throws IOException {
+ Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
+ Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
+ int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
+ long diff = 0;
+
+ byte[] start = startRow.getBytes();
+ byte[] stop = stopRow.getBytes();
+ for (int i = 0; i < maxCommon; ++i) {
+ diff |= 0xff & (start[i] ^ stop[i]);
+ diff <<= Byte.SIZE;
+ }
+
+ if (startRow.getLength() != stopRow.getLength())
+ diff |= 0xff;
+
+ return diff + 1;
+ }
+
+ @Override
+ public String[] getLocations() throws IOException {
+ return locations;
+ }
+
+ public void setLocations(String[] locations) {
+ this.locations = locations;
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ range.readFields(in);
+ tableName = in.readUTF();
+ int numLocs = in.readInt();
+ locations = new String[numLocs];
+ for (int i = 0; i < numLocs; ++i)
+ locations[i] = in.readUTF();
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ range.write(out);
+ out.writeUTF(tableName);
+ out.writeInt(locations.length);
+ for (int i = 0; i < locations.length; ++i)
+ out.writeUTF(locations[i]);
+ }
+ }
+
+ // use reflection to pull the Configuration out of the JobContext for Hadoop 1 and Hadoop 2 compatibility
+ static Configuration getConfiguration(JobContext context) {
+ try {
+ Class<?> c = AbstractInputFormat.class.getClassLoader().loadClass("org.apache.hadoop.mapreduce.JobContext");
+ Method m = c.getMethod("getConfiguration");
+ Object o = m.invoke(context, new Object[0]);
+ return (Configuration) o;
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/5c496552/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
new file mode 100644
index 0000000..2123ab9
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
@@ -0,0 +1,53 @@
+package org.apache.accumulo.core.client.mapreduce;
+
+import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.conf.TableQueryConfig;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.util.format.DefaultFormatter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value>{
+
+ /**
+ * Sets the {@link org.apache.accumulo.core.conf.TableQueryConfig} objects on the given Hadoop configuration
+ *
+ * @param job
+ * the Hadoop job instance to be configured
+ * @param configs
+ * the table query configs to be set on the configuration.
+ * @since 1.6.0
+ */
+ public static void setBatchScanConfigs(Job job, TableQueryConfig... configs) {
+ checkNotNull(configs);
+ InputConfigurator.setTableQueryConfigs(CLASS, getConfiguration(job), configs);
+ }
+
+ @Override
+ public RecordReader<Key, Value> createRecordReader(InputSplit inputSplit, TaskAttemptContext context) throws IOException, InterruptedException {
+ log.setLevel(getLogLevel(context));
+ return new RecordReaderBase<Key,Value>() {
+ @Override
+ public boolean nextKeyValue() throws IOException, InterruptedException {
+ if (scannerIterator.hasNext()) {
+ ++numKeysRead;
+ Map.Entry<Key,Value> entry = scannerIterator.next();
+ currentK = currentKey = entry.getKey();
+ currentV = currentValue = entry.getValue();
+ if (log.isTraceEnabled())
+ log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
+ return true;
+ }
+ return false;
+ }
+ };
+ }
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/5c496552/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index e2d6b33..ad9c454 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -16,69 +16,28 @@
*/
package org.apache.accumulo.core.client.mapreduce;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-
-import java.io.DataInput;
-import java.io.DataOutput;
import java.io.IOException;
-import java.lang.reflect.Method;
-import java.math.BigInteger;
-import java.net.InetAddress;
-import java.util.ArrayList;
import java.util.Collection;
-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.Set;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
import org.apache.accumulo.core.client.IsolatedScanner;
import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.RowIterator;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableDeletedException;
import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.TableOfflineException;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.impl.OfflineScanner;
-import org.apache.accumulo.core.client.impl.Tables;
import org.apache.accumulo.core.client.impl.TabletLocator;
import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.TableQueryConfig;
-import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
-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.master.state.tables.TableState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
/**
* This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of K,V pairs.
@@ -90,75 +49,7 @@ import org.apache.log4j.Logger;
* <p>
* See {@link AccumuloInputFormat} for an example implementation.
*/
-public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
-
- private static final Class<?> CLASS = AccumuloInputFormat.class;
- protected static final Logger log = Logger.getLogger(CLASS);
-
- /**
- * Sets the connector information needed to communicate with Accumulo in this job.
- *
- * <p>
- * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
- * conversion to a string, and is not intended to be secure.
- *
- * @param job
- * the Hadoop job instance to be configured
- * @param principal
- * a valid Accumulo user name (user must have Table.CREATE permission)
- * @param token
- * the user's password
- * @throws AccumuloSecurityException
- * @since 1.5.0
- */
- public static void setConnectorInfo(Job job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
- InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
- }
-
- /**
- * Sets the connector information needed to communicate with Accumulo in this job.
- *
- * <p>
- * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt to be more secure than storing it in the Configuration.
- *
- * @param job
- * the Hadoop job instance to be configured
- * @param principal
- * a valid Accumulo user name (user must have Table.CREATE permission)
- * @param tokenFile
- * the path to the token file
- * @throws AccumuloSecurityException
- * @since 1.6.0
- */
- public static void setConnectorInfo(Job job, String principal, String tokenFile) throws AccumuloSecurityException {
- InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
- }
-
- /**
- * Determines if the connector has been configured.
- *
- * @param context
- * the Hadoop context for the configured job
- * @return true if the connector has been configured, false otherwise
- * @since 1.5.0
- * @see #setConnectorInfo(Job, String, AuthenticationToken)
- */
- protected static Boolean isConnectorInfoSet(JobContext context) {
- return InputConfigurator.isConnectorInfoSet(CLASS, getConfiguration(context));
- }
-
- /**
- * Gets the user name from the configuration.
- *
- * @param context
- * the Hadoop context for the configured job
- * @return the user name
- * @since 1.5.0
- * @see #setConnectorInfo(Job, String, AuthenticationToken)
- */
- protected static String getPrincipal(JobContext context) {
- return InputConfigurator.getPrincipal(CLASS, getConfiguration(context));
- }
+public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
/**
* Gets the table name from the configuration.
@@ -174,110 +65,6 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
}
/**
- * Gets the serialized token class from either the configuration or the token file.
- *
- * @since 1.5.0
- * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
- */
- @Deprecated
- protected static String getTokenClass(JobContext context) {
- return getAuthenticationToken(context).getClass().getName();
- }
-
- /**
- * Gets the serialized token from either the configuration or the token file.
- *
- * @since 1.5.0
- * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
- */
- @Deprecated
- protected static byte[] getToken(JobContext context) {
- return AuthenticationTokenSerializer.serialize(getAuthenticationToken(context));
- }
-
- /**
- * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
- *
- * @param context
- * the Hadoop context for the configured job
- * @return the principal's authentication token
- * @since 1.6.0
- * @see #setConnectorInfo(Job, String, AuthenticationToken)
- * @see #setConnectorInfo(Job, String, String)
- */
- protected static AuthenticationToken getAuthenticationToken(JobContext context) {
- return InputConfigurator.getAuthenticationToken(CLASS, getConfiguration(context));
- }
-
- /**
- * Configures a {@link ZooKeeperInstance} for this job.
- *
- * @param job
- * the Hadoop job instance to be configured
- * @param instanceName
- * the Accumulo instance name
- * @param zooKeepers
- * a comma-separated list of zookeeper servers
- * @since 1.5.0
- */
- public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
- InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), instanceName, zooKeepers);
- }
-
- /**
- * Configures a {@link MockInstance} for this job.
- *
- * @param job
- * the Hadoop job instance to be configured
- * @param instanceName
- * the Accumulo instance name
- * @since 1.5.0
- */
- public static void setMockInstance(Job job, String instanceName) {
- InputConfigurator.setMockInstance(CLASS, job.getConfiguration(), instanceName);
- }
-
- /**
- * Initializes an Accumulo {@link Instance} based on the configuration.
- *
- * @param context
- * the Hadoop context for the configured job
- * @return an Accumulo instance
- * @since 1.5.0
- * @see #setZooKeeperInstance(Job, String, String)
- * @see #setMockInstance(Job, String)
- */
- protected static Instance getInstance(JobContext context) {
- return InputConfigurator.getInstance(CLASS, getConfiguration(context));
- }
-
- /**
- * Sets the log level for this job.
- *
- * @param job
- * the Hadoop job instance to be configured
- * @param level
- * the logging level
- * @since 1.5.0
- */
- public static void setLogLevel(Job job, Level level) {
- InputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
- }
-
- /**
- * Gets the log level from this configuration.
- *
- * @param context
- * the Hadoop context for the configured job
- * @return the log level
- * @since 1.5.0
- * @see #setLogLevel(Job, Level)
- */
- protected static Level getLogLevel(JobContext context) {
- return InputConfigurator.getLogLevel(CLASS, getConfiguration(context));
- }
-
- /**
* Sets the name of the input table, over which this job will scan.
*
* @param job
@@ -291,31 +78,6 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
}
/**
- * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
- *
- * @param job
- * the Hadoop job instance to be configured
- * @param auths
- * the user's authorizations
- */
- public static void setScanAuthorizations(Job job, Authorizations auths) {
- InputConfigurator.setScanAuthorizations(CLASS, job.getConfiguration(), auths);
- }
-
- /**
- * Gets the authorizations to set for the scans from the configuration.
- *
- * @param context
- * the Hadoop context for the configured job
- * @return the Accumulo scan authorizations
- * @since 1.5.0
- * @see #setScanAuthorizations(Job, Authorizations)
- */
- protected static Authorizations getScanAuthorizations(JobContext context) {
- return InputConfigurator.getScanAuthorizations(CLASS, getConfiguration(context));
- }
-
- /**
* Sets the input ranges to scan for the single input table associated with this job.
*
* @param job
@@ -427,55 +189,6 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
}
/**
- * Sets the {@link TableQueryConfig} objects on the given Hadoop configuration
- *
- * @param job
- * the Hadoop job instance to be configured
- * @param configs
- * the table query configs to be set on the configuration.
- * @since 1.6.0
- */
- public static void setTableQueryConfigs(Job job, TableQueryConfig... configs) {
- checkNotNull(configs);
- InputConfigurator.setTableQueryConfigs(CLASS, getConfiguration(job), configs);
- }
-
- /**
- * Fetches all {@link TableQueryConfig}s that have been set on the given Hadoop configuration.
- *
- * <p>
- * Note this also returns the {@link TableQueryConfig} representing the table configurations set through the single table input methods like
- * {@link #setInputTableName(org.apache.hadoop.mapreduce.Job, String)}, {@link #setRanges(org.apache.hadoop.mapreduce.Job, java.util.Collection)},
- * {@link #fetchColumns(org.apache.hadoop.mapreduce.Job, java.util.Collection)},
- * {@link #addIterator(org.apache.hadoop.mapreduce.Job, org.apache.accumulo.core.client.IteratorSetting)}, etc...)
- *
- * @param job
- * the Hadoop job instance to be configured
- * @return
- * @since 1.6.0
- */
- public static List<TableQueryConfig> getTableQueryConfigs(JobContext job) {
- return InputConfigurator.getTableQueryConfigs(CLASS, getConfiguration(job));
- }
-
- /**
- * Fetches a {@link TableQueryConfig} that has been set on the configuration for a specific table.
- *
- * <p>
- * null is returned in the event that the table doesn't exist.
- *
- * @param job
- * the Hadoop job instance to be configured
- * @param tableName
- * the table name for which to grab the config object
- * @return the {@link TableQueryConfig} for the given table
- * @since 1.6.0
- */
- public static TableQueryConfig getTableQueryConfig(JobContext job, String tableName) {
- return InputConfigurator.getTableQueryConfig(CLASS,getConfiguration(job),tableName);
- }
-
- /**
* Controls the use of the {@link IsolatedScanner} in this job.
*
* <p>
@@ -583,516 +296,17 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
}
/**
- * Initializes an Accumulo {@link TabletLocator} based on the configuration.
+ * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
*
* @param context
* the Hadoop context for the configured job
- * @param table
- * the table for which to initialize the locator
* @return an Accumulo tablet locator
- * @throws TableNotFoundException
+ * @throws org.apache.accumulo.core.client.TableNotFoundException
* if the table name set on the configuration doesn't exist
* @since 1.5.0
*/
- protected static TabletLocator getTabletLocator(JobContext context, String table) throws TableNotFoundException {
- return InputConfigurator.getTabletLocator(CLASS, getConfiguration(context), table);
- }
-
- // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
- /**
- * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
- *
- * @param context
- * the Hadoop context for the configured job
- * @throws IOException
- * if the context is improperly configured
- * @since 1.5.0
- */
- protected static void validateOptions(JobContext context) throws IOException {
- InputConfigurator.validateOptions(CLASS, getConfiguration(context));
- }
-
- /**
- * An abstract base class to be used to create {@link RecordReader} instances that convert from Accumulo {@link Key}/{@link Value} pairs to the user's K/V
- * types.
- *
- * Subclasses must implement {@link #nextKeyValue()} and use it to update the following variables:
- * <ul>
- * <li>K {@link #currentK}</li>
- * <li>V {@link #currentV}</li>
- * <li>Key {@link #currentKey} (used for progress reporting)</li>
- * <li>int {@link #numKeysRead} (used for progress reporting)</li>
- * </ul>
- */
- protected abstract static class RecordReaderBase<K,V> extends RecordReader<K,V> {
- protected long numKeysRead;
- protected Iterator<Entry<Key,Value>> scannerIterator;
- protected RangeInputSplit split;
-
- /**
- * Apply the configured iterators from the configuration to the scanner. This applies both the default iterators and the per-table iterators.
- *
- * @param context
- * the Hadoop context for the configured job
- * @param scanner
- * the scanner to configure
- * @param tableName
- * the table name for which to set up the iterators
- */
- protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName) {
- TableQueryConfig config = getTableQueryConfig(context, tableName);
- List<IteratorSetting> iterators = config.getIterators();
- for (IteratorSetting iterator : iterators)
- scanner.addScanIterator(iterator);
- }
-
- /**
- * Initialize a scanner over the given input split using this task attempt configuration.
- */
- @Override
- public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
-
- Scanner scanner;
- split = (RangeInputSplit) inSplit;
- log.debug("Initializing input split: " + split.getRange());
- Instance instance = getInstance(attempt);
- String principal = getPrincipal(attempt);
-
- TableQueryConfig tableConfig = getTableQueryConfig(attempt, split.getTableName());
-
- // in case the table name changed, we can still use the previous name for terms of configuration,
- // but for the scanner, we'll need to reference the new table name.
- String actualNameForId = split.getTableName();
- if (!(instance instanceof MockInstance)) {
- try {
- actualNameForId = Tables.getTableName(instance, split.getTableId());
- if (!actualNameForId.equals(split.getTableName()))
- log.debug("Table name changed from " + split.getTableName() + " to " + actualNameForId);
- } catch (TableNotFoundException e) {
- throw new IOException("The specified table was not found for id=" + split.getTableId());
- }
- }
-
- AuthenticationToken token = getAuthenticationToken(attempt);
- Authorizations authorizations = getScanAuthorizations(attempt);
- try {
- log.debug("Creating connector with user: " + principal);
-
- Connector conn = instance.getConnector(principal, token);
- log.debug("Creating scanner for table: " + split.getTableName());
- log.debug("Authorizations are: " + authorizations);
- if (tableConfig.isOfflineScan()) {
- scanner = new OfflineScanner(instance, new Credentials(principal, token), split.getTableId(), authorizations);
- } else {
- scanner = conn.createScanner(actualNameForId, authorizations);
- }
- if (tableConfig.shouldUseIsolatedScanners()) {
- log.info("Creating isolated scanner");
- scanner = new IsolatedScanner(scanner);
- }
- if (tableConfig.shouldUseLocalIterators()) {
- log.info("Using local iterators");
- scanner = new ClientSideIteratorScanner(scanner);
- }
- setupIterators(attempt, scanner, split.getTableName());
- } catch (Exception e) {
- throw new IOException(e);
- }
-
- // setup a scanner within the bounds of this split
- for (Pair<Text,Text> c : tableConfig.getFetchedColumns()) {
- if (c.getSecond() != null) {
- log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
- scanner.fetchColumn(c.getFirst(), c.getSecond());
- } else {
- log.debug("Fetching column family " + c.getFirst());
- scanner.fetchColumnFamily(c.getFirst());
- }
- }
-
- scanner.setRange(split.getRange());
- numKeysRead = 0;
-
- // do this last after setting all scanner options
- scannerIterator = scanner.iterator();
- }
-
- @Override
- public void close() {}
-
- @Override
- public float getProgress() throws IOException {
- if (numKeysRead > 0 && currentKey == null)
- return 1.0f;
- return split.getProgress(currentKey);
- }
-
- protected K currentK = null;
- protected V currentV = null;
- protected Key currentKey = null;
- protected Value currentValue = null;
-
- @Override
- public K getCurrentKey() throws IOException, InterruptedException {
- return currentK;
- }
-
- @Override
- public V getCurrentValue() throws IOException, InterruptedException {
- return currentV;
- }
- }
-
- Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext context, String tableName, List<Range> ranges) throws TableNotFoundException,
- AccumuloException, AccumuloSecurityException {
-
- Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
-
- Instance instance = getInstance(context);
- Connector conn = instance.getConnector(getPrincipal(context), getAuthenticationToken(context));
- String tableId = Tables.getTableId(instance, tableName);
-
- if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
- Tables.clearCache(instance);
- if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
- throw new AccumuloException("Table is online " + tableName + "(" + tableId + ") cannot scan table in offline mode ");
- }
- }
-
- for (Range range : ranges) {
- Text startRow;
-
- if (range.getStartKey() != null)
- startRow = range.getStartKey().getRow();
- else
- startRow = new Text();
-
- Range metadataRange = new Range(new KeyExtent(new Text(tableId), startRow, null).getMetadataEntry(), true, null, false);
- Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
- TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
- scanner.fetchColumnFamily(TabletsSection.LastLocationColumnFamily.NAME);
- scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
- scanner.fetchColumnFamily(TabletsSection.FutureLocationColumnFamily.NAME);
- scanner.setRange(metadataRange);
-
- RowIterator rowIter = new RowIterator(scanner);
- KeyExtent lastExtent = null;
- while (rowIter.hasNext()) {
- Iterator<Entry<Key,Value>> row = rowIter.next();
- String last = "";
- KeyExtent extent = null;
- String location = null;
-
- while (row.hasNext()) {
- Entry<Key,Value> entry = row.next();
- Key key = entry.getKey();
-
- if (key.getColumnFamily().equals(TabletsSection.LastLocationColumnFamily.NAME)) {
- last = entry.getValue().toString();
- }
-
- if (key.getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME)
- || key.getColumnFamily().equals(TabletsSection.FutureLocationColumnFamily.NAME)) {
- location = entry.getValue().toString();
- }
-
- if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
- extent = new KeyExtent(key.getRow(), entry.getValue());
- }
-
- }
-
- if (location != null)
- return null;
-
- if (!extent.getTableId().toString().equals(tableId)) {
- throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
- }
-
- if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
- throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
- }
-
- Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
- if (tabletRanges == null) {
- tabletRanges = new HashMap<KeyExtent,List<Range>>();
- binnedRanges.put(last, tabletRanges);
- }
-
- List<Range> rangeList = tabletRanges.get(extent);
- if (rangeList == null) {
- rangeList = new ArrayList<Range>();
- tabletRanges.put(extent, rangeList);
- }
-
- rangeList.add(range);
-
- if (extent.getEndRow() == null || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
- break;
- }
-
- lastExtent = extent;
- }
-
- }
-
- return binnedRanges;
- }
-
- /**
- * Gets the splits of the tables that have been set on the job.
- *
- * @param conf
- * the configuration of the job
- * @return the splits from the tables based on the ranges.
- * @throws IOException
- * if a table set on the job doesn't exist or an error occurs initializing the tablet locator
- */
- public List<InputSplit> getSplits(JobContext conf) throws IOException {
- log.setLevel(getLogLevel(conf));
- validateOptions(conf);
-
- LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
- List<TableQueryConfig> tableConfigs = getTableQueryConfigs(conf);
- for (TableQueryConfig tableConfig : tableConfigs) {
-
- boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
- String tableId = null;
- List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
- if (ranges.isEmpty()) {
- ranges = new ArrayList<Range>(1);
- ranges.add(new Range());
- }
-
- // get the metadata information for these ranges
- Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
- TabletLocator tl;
- try {
- if (tableConfig.isOfflineScan()) {
- binnedRanges = binOfflineTable(conf, tableConfig.getTableName(), ranges);
- while (binnedRanges == null) {
- // Some tablets were still online, try again
- UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
- binnedRanges = binOfflineTable(conf, tableConfig.getTableName(), ranges);
-
- }
- } else {
- Instance instance = getInstance(conf);
- tl = getTabletLocator(conf, tableConfig.getTableName());
- // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
- tl.invalidateCache();
- Credentials creds = new Credentials(getPrincipal(conf), getAuthenticationToken(conf));
-
- while (!tl.binRanges(creds, ranges, binnedRanges).isEmpty()) {
- if (!(instance instanceof MockInstance)) {
- if (!Tables.exists(instance, tableId))
- throw new TableDeletedException(tableId);
- if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
- throw new TableOfflineException(instance, tableId);
- tableId = Tables.getTableId(instance, tableConfig.getTableName());
- }
- binnedRanges.clear();
- log.warn("Unable to locate bins for specified ranges. Retrying.");
- UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
- tl.invalidateCache();
- }
- }
- } catch (Exception e) {
- throw new IOException(e);
- }
-
- HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
- if (!autoAdjust)
- splitsToAdd = new HashMap<Range,ArrayList<String>>();
-
- HashMap<String,String> hostNameCache = new HashMap<String,String>();
- for (Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
- String ip = tserverBin.getKey().split(":", 2)[0];
- String location = hostNameCache.get(ip);
- if (location == null) {
- InetAddress inetAddress = InetAddress.getByName(ip);
- location = inetAddress.getHostName();
- hostNameCache.put(ip, location);
- }
- for (Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
- Range ke = extentRanges.getKey().toDataRange();
- for (Range r : extentRanges.getValue()) {
- if (autoAdjust) {
- // divide ranges into smaller ranges, based on the tablets
- splits.add(new RangeInputSplit(tableConfig.getTableName(), tableId, ke.clip(r), new String[] {location}));
- } else {
- // don't divide ranges
- ArrayList<String> locations = splitsToAdd.get(r);
- if (locations == null)
- locations = new ArrayList<String>(1);
- locations.add(location);
- splitsToAdd.put(r, locations);
- }
- }
- }
- }
-
- if (!autoAdjust)
- for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
- splits.add(new RangeInputSplit(tableConfig.getTableName(), tableId, entry.getKey(), entry.getValue().toArray(new String[0])));
- }
- return splits;
+ protected static TabletLocator getTabletLocator(JobContext context) throws TableNotFoundException {
+ return InputConfigurator.getTabletLocator(CLASS, getConfiguration(context), InputConfigurator.getInputTableName(CLASS, getConfiguration(context)));
}
- /**
- * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
- */
- public static class RangeInputSplit extends InputSplit implements Writable {
- private Range range;
- private String[] locations;
- private String tableId;
- private String tableName;
-
- public RangeInputSplit() {
- range = new Range();
- locations = new String[0];
- tableId = "";
- tableName = "";
- }
-
- public RangeInputSplit(RangeInputSplit split) throws IOException {
- this.setRange(split.getRange());
- this.setLocations(split.getLocations());
- this.setTableName(split.getTableName());
- }
-
- protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
- this.range = range;
- this.locations = locations;
- this.tableName = table;
- this.tableId = tableId;
- }
-
- public Range getRange() {
- return range;
- }
-
- public void setRange(Range range) {
- this.range = range;
- }
-
- public String getTableName() {
- return tableName;
- }
-
- public void setTableName(String tableName) {
- this.tableName = tableName;
- }
-
- public void setTableId(String tableId) {
- this.tableId = tableId;
- }
-
- public String getTableId() {
- return tableId;
- }
-
- private static byte[] extractBytes(ByteSequence seq, int numBytes) {
- byte[] bytes = new byte[numBytes + 1];
- bytes[0] = 0;
- for (int i = 0; i < numBytes; i++) {
- if (i >= seq.length())
- bytes[i + 1] = 0;
- else
- bytes[i + 1] = seq.byteAt(i);
- }
- return bytes;
- }
-
- public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
- int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
- BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
- BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
- BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
- return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
- }
-
- public float getProgress(Key currentKey) {
- if (currentKey == null)
- return 0f;
- if (range.getStartKey() != null && range.getEndKey() != null) {
- if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW)) {
- // just look at the row progress
- return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
- } else if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW_COLFAM)) {
- // just look at the column family progress
- return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
- } else if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL)) {
- // just look at the column qualifier progress
- return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
- }
- }
- // if we can't figure it out, then claim no progress
- return 0f;
- }
-
- /**
- * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
- */
- @Override
- public long getLength() throws IOException {
- Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
- Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
- int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
- long diff = 0;
-
- byte[] start = startRow.getBytes();
- byte[] stop = stopRow.getBytes();
- for (int i = 0; i < maxCommon; ++i) {
- diff |= 0xff & (start[i] ^ stop[i]);
- diff <<= Byte.SIZE;
- }
-
- if (startRow.getLength() != stopRow.getLength())
- diff |= 0xff;
-
- return diff + 1;
- }
-
- @Override
- public String[] getLocations() throws IOException {
- return locations;
- }
-
- public void setLocations(String[] locations) {
- this.locations = locations;
- }
-
- @Override
- public void readFields(DataInput in) throws IOException {
- range.readFields(in);
- tableName = in.readUTF();
- int numLocs = in.readInt();
- locations = new String[numLocs];
- for (int i = 0; i < numLocs; ++i)
- locations[i] = in.readUTF();
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
- range.write(out);
- out.writeUTF(tableName);
- out.writeInt(locations.length);
- for (int i = 0; i < locations.length; ++i)
- out.writeUTF(locations[i]);
- }
- }
-
- // use reflection to pull the Configuration out of the JobContext for Hadoop 1 and Hadoop 2 compatibility
- static Configuration getConfiguration(JobContext context) {
- try {
- Class<?> c = InputFormatBase.class.getClassLoader().loadClass("org.apache.hadoop.mapreduce.JobContext");
- Method m = c.getMethod("getConfiguration");
- Object o = m.invoke(context, new Object[0]);
- return (Configuration) o;
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/5c496552/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index 6f92dec..c9539c4 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
-import java.util.Collections;
import java.util.List;
import org.apache.accumulo.core.client.BatchWriter;
@@ -32,15 +31,12 @@ import org.apache.accumulo.core.client.Connector;
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.mock.MockInstance;
import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.TableQueryConfig;
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.iterators.user.RegExFilter;
import org.apache.accumulo.core.iterators.user.WholeRowIterator;
import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.core.util.Pair;
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
@@ -57,7 +53,6 @@ public class AccumuloInputFormatTest {
private static final String PREFIX = AccumuloInputFormatTest.class.getSimpleName();
private static final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
private static final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
- private static final String TEST_TABLE_2 = PREFIX + "_mapreduce_table_2";
/**
* Check that the iterator configuration is getting stored in the Job conf correctly.
@@ -201,7 +196,6 @@ public class AccumuloInputFormatTest {
private static AssertionError e2 = null;
private static class MRTester extends Configured implements Tool {
-
private static class TestMapper extends Mapper<Key,Value,Key,Value> {
Key key = null;
int count = 0;
@@ -209,11 +203,10 @@ public class AccumuloInputFormatTest {
@Override
protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
try {
- String tableName = ((InputFormatBase.RangeInputSplit) context.getInputSplit()).getTableName();
if (key != null)
assertEquals(key.getRow().toString(), new String(v.get()));
- assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
- assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
+ assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
+ assertEquals(new String(v.get()), String.format("%09x", count));
} catch (AssertionError e) {
e1 = e;
}
@@ -234,14 +227,13 @@ public class AccumuloInputFormatTest {
@Override
public int run(String[] args) throws Exception {
- if (args.length != 4) {
- throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table1> <table2>");
+ if (args.length != 3) {
+ throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table>");
}
String user = args[0];
String pass = args[1];
- String table1 = args[2];
- String table2 = args[3];
+ String table = args[2];
Job job = new Job(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
job.setJarByClass(this.getClass());
@@ -249,11 +241,7 @@ public class AccumuloInputFormatTest {
job.setInputFormatClass(AccumuloInputFormat.class);
AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-
- TableQueryConfig tableConfig1 = new TableQueryConfig(table1);
- TableQueryConfig tableConfig2 = new TableQueryConfig(table2);
-
- AccumuloInputFormat.setTableQueryConfigs(job, tableConfig1, tableConfig2);
+ AccumuloInputFormat.setInputTableName(job, table);
AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
job.setMapperClass(TestMapper.class);
@@ -273,78 +261,21 @@ public class AccumuloInputFormatTest {
}
}
- /**
- * Generate incrementing counts and attach table name to the key/value so that order and multi-table data can be verified.
- */
@Test
public void testMap() throws Exception {
MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
Connector c = mockInstance.getConnector("root", new PasswordToken(""));
c.tableOperations().create(TEST_TABLE_1);
- c.tableOperations().create(TEST_TABLE_2);
BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
- BatchWriter bw2 = c.createBatchWriter(TEST_TABLE_2, new BatchWriterConfig());
for (int i = 0; i < 100; i++) {
- Mutation t1m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_1, i + 1)));
- t1m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_1, i).getBytes()));
- bw.addMutation(t1m);
- Mutation t2m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_2, i + 1)));
- t2m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_2, i).getBytes()));
- bw2.addMutation(t2m);
+ Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+ m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+ bw.addMutation(m);
}
bw.close();
- bw2.close();
- MRTester.main(new String[] {"root", "", TEST_TABLE_1, TEST_TABLE_2});
+ MRTester.main(new String[] {"root", "", TEST_TABLE_1});
assertNull(e1);
assertNull(e2);
}
-
- /**
- * Verify {@link TableQueryConfig} objects get correctly serialized in the JobContext.
- */
- @Test
- public void testTableQueryConfigSerialization() throws IOException {
-
- Job job = new Job();
-
- TableQueryConfig table1 = new TableQueryConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
- .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
- .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
-
- TableQueryConfig table2 = new TableQueryConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
- .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
- .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
-
- AccumuloInputFormat.setTableQueryConfigs(job, table1, table2);
-
- assertEquals(table1, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_1));
- assertEquals(table2, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_2));
- }
-
- /**
- * Verify that union of legacy input and new multi-table input get returned for backwards compatibility.
- */
- @Test
- public void testTableQueryConfigSingleAndMultitableMethods() throws IOException {
-
- Job job = new Job();
-
- TableQueryConfig table1 = new TableQueryConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
- .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
- .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
-
- TableQueryConfig table2 = new TableQueryConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
- .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
- .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
-
- AccumuloInputFormat.setTableQueryConfigs(job, table1);
- AccumuloInputFormat.setInputTableName(job, table2.getTableName());
- AccumuloInputFormat.setRanges(job, table2.getRanges());
- AccumuloInputFormat.fetchColumns(job, table2.getFetchedColumns());
- AccumuloInputFormat.addIterator(job, table2.getIterators().get(0));
-
- assertEquals(table1, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_1));
- assertEquals(table2, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_2));
- }
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/5c496552/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
new file mode 100644
index 0000000..b3b6d8b
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
@@ -0,0 +1,163 @@
+package org.apache.accumulo.core.client.mapreduce;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.TableQueryConfig;
+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.util.CachedConfiguration;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class AccumuloMultiTableInputFormatTest {
+
+ private static final String PREFIX = AccumuloMultiTableInputFormatTest.class.getSimpleName();
+ private static final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
+ private static final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
+ private static final String TEST_TABLE_2 = PREFIX + "_mapreduce_table_2";
+
+ private static AssertionError e1 = null;
+ private static AssertionError e2 = null;
+
+ private static class MRTester extends Configured implements Tool {
+
+ private static class TestMapper extends Mapper<Key,Value,Key,Value> {
+ Key key = null;
+ int count = 0;
+
+ @Override
+ protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+ try {
+ String tableName = ((InputFormatBase.RangeInputSplit) context.getInputSplit()).getTableName();
+ if (key != null)
+ assertEquals(key.getRow().toString(), new String(v.get()));
+ assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
+ assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
+ } catch (AssertionError e) {
+ e1 = e;
+ }
+ key = new Key(k);
+ count++;
+ }
+
+ @Override
+ protected void cleanup(Context context) throws IOException, InterruptedException {
+ try {
+ assertEquals(100, count);
+ } catch (AssertionError e) {
+ e2 = e;
+ }
+ }
+ }
+
+ @Override
+ public int run(String[] args) throws Exception {
+
+ if (args.length != 4) {
+ throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table1> <table2>");
+ }
+
+ String user = args[0];
+ String pass = args[1];
+ String table1 = args[2];
+ String table2 = args[3];
+
+ Job job = new Job(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+ job.setJarByClass(this.getClass());
+
+ job.setInputFormatClass(AccumuloMultiTableInputFormat.class);
+
+ AccumuloMultiTableInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
+
+ TableQueryConfig tableConfig1 = new TableQueryConfig(table1);
+ TableQueryConfig tableConfig2 = new TableQueryConfig(table2);
+
+ AccumuloMultiTableInputFormat.setBatchScanConfigs(job, tableConfig1, tableConfig2);
+ AccumuloMultiTableInputFormat.setMockInstance(job, INSTANCE_NAME);
+
+ job.setMapperClass(TestMapper.class);
+ job.setMapOutputKeyClass(Key.class);
+ job.setMapOutputValueClass(Value.class);
+ job.setOutputFormatClass(NullOutputFormat.class);
+
+ job.setNumReduceTasks(0);
+
+ job.waitForCompletion(true);
+
+ return job.isSuccessful() ? 0 : 1;
+ }
+
+ public static void main(String[] args) throws Exception {
+ assertEquals(0, ToolRunner.run(CachedConfiguration.getInstance(), new MRTester(), args));
+ }
+ }
+
+ /**
+ * Generate incrementing counts and attach table name to the key/value so that order and multi-table data can be verified.
+ */
+ @Test
+ public void testMap() throws Exception {
+ MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+ Connector c = mockInstance.getConnector("root", new PasswordToken(""));
+ c.tableOperations().create(TEST_TABLE_1);
+ c.tableOperations().create(TEST_TABLE_2);
+ BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
+ BatchWriter bw2 = c.createBatchWriter(TEST_TABLE_2, new BatchWriterConfig());
+ for (int i = 0; i < 100; i++) {
+ Mutation t1m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_1, i + 1)));
+ t1m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_1, i).getBytes()));
+ bw.addMutation(t1m);
+ Mutation t2m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_2, i + 1)));
+ t2m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_2, i).getBytes()));
+ bw2.addMutation(t2m);
+ }
+ bw.close();
+ bw2.close();
+
+ MRTester.main(new String[] {"root", "", TEST_TABLE_1, TEST_TABLE_2});
+ assertNull(e1);
+ assertNull(e2);
+ }
+
+ /**
+ * Verify {@link TableQueryConfig} objects get correctly serialized in the JobContext.
+ */
+ @Test
+ public void testTableQueryConfigSerialization() throws IOException {
+
+ Job job = new Job();
+
+ TableQueryConfig table1 = new TableQueryConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
+ .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
+ .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
+
+ TableQueryConfig table2 = new TableQueryConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
+ .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
+ .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
+
+ AccumuloMultiTableInputFormat.setBatchScanConfigs(job, table1, table2);
+
+ assertEquals(table1, AccumuloMultiTableInputFormat.getBatchScanConfig(job, TEST_TABLE_1));
+ assertEquals(table2, AccumuloMultiTableInputFormat.getBatchScanConfig(job, TEST_TABLE_2));
+ }
+
+}
[4/5] git commit: ACCUMULO-391 AbstractRecordReader created to help
common functionality.
Posted by cj...@apache.org.
ACCUMULO-391 AbstractRecordReader created to help common functionality.
Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/ebd11205
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/ebd11205
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/ebd11205
Branch: refs/heads/master
Commit: ebd112056017d3bbe32c62329ca31fecf6c22fea
Parents: f819767
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Mon Oct 14 01:24:37 2013 -0400
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Tue Oct 15 20:22:07 2013 -0400
----------------------------------------------------------------------
.../client/mapreduce/AbstractInputFormat.java | 47 +++++++-------------
.../client/mapreduce/AccumuloInputFormat.java | 7 ++-
.../AccumuloMultiTableInputFormat.java | 13 +++++-
.../mapreduce/AccumuloRowInputFormat.java | 3 ++
.../core/client/mapreduce/BatchScanConfig.java | 1 -
.../core/client/mapreduce/InputFormatBase.java | 17 ++++++-
6 files changed, 53 insertions(+), 35 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebd11205/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 503718e..d86e111 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -1,7 +1,5 @@
package org.apache.accumulo.core.client.mapreduce;
-import static com.google.common.base.Preconditions.checkNotNull;
-
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
@@ -21,7 +19,6 @@ import org.apache.accumulo.core.client.ClientSideIteratorScanner;
import org.apache.accumulo.core.client.Connector;
import org.apache.accumulo.core.client.Instance;
import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.RowIterator;
import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.TableDeletedException;
@@ -58,6 +55,11 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
+/**
+ *
+ * @param <K>
+ * @param <V>
+ */
public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
protected static final Class<?> CLASS = AccumuloInputFormat.class;
@@ -328,27 +330,12 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
* <li>int {@link #numKeysRead} (used for progress reporting)</li>
* </ul>
*/
- protected abstract static class RecordReaderBase<K,V> extends RecordReader<K,V> {
+ protected abstract static class AbstractRecordReader<K,V> extends RecordReader<K,V> {
protected long numKeysRead;
protected Iterator<Map.Entry<Key,Value>> scannerIterator;
protected RangeInputSplit split;
- /**
- * Apply the configured iterators from the configuration to the scanner. This applies both the default iterators and the per-table iterators.
- *
- * @param context
- * the Hadoop context for the configured job
- * @param scanner
- * the scanner to configure
- * @param tableName
- * the table name for which to set up the iterators
- */
- protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName) {
- BatchScanConfig config = getBatchScanConfig(context, tableName);
- List<IteratorSetting> iterators = config.getIterators();
- for (IteratorSetting iterator : iterators)
- scanner.addScanIterator(iterator);
- }
+ protected abstract void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName);
/**
* Initialize a scanner over the given input split using this task attempt configuration.
@@ -546,18 +533,18 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
/**
* Gets the splits of the tables that have been set on the job.
*
- * @param conf
+ * @param context
* the configuration of the job
* @return the splits from the tables based on the ranges.
* @throws java.io.IOException
* if a table set on the job doesn't exist or an error occurs initializing the tablet locator
*/
- public List<InputSplit> getSplits(JobContext conf) throws IOException {
- log.setLevel(getLogLevel(conf));
- validateOptions(conf);
+ public List<InputSplit> getSplits(JobContext context) throws IOException {
+ log.setLevel(getLogLevel(context));
+ validateOptions(context);
LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
- List<BatchScanConfig> tableConfigs = getBatchScanConfigs(conf);
+ List<BatchScanConfig> tableConfigs = getBatchScanConfigs(context);
for (BatchScanConfig tableConfig : tableConfigs) {
boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
@@ -573,19 +560,19 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
TabletLocator tl;
try {
if (tableConfig.isOfflineScan()) {
- binnedRanges = binOfflineTable(conf, tableConfig.getTableName(), ranges);
+ binnedRanges = binOfflineTable(context, tableConfig.getTableName(), ranges);
while (binnedRanges == null) {
// Some tablets were still online, try again
UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
- binnedRanges = binOfflineTable(conf, tableConfig.getTableName(), ranges);
+ binnedRanges = binOfflineTable(context, tableConfig.getTableName(), ranges);
}
} else {
- Instance instance = getInstance(conf);
- tl = getTabletLocator(conf, tableConfig.getTableName());
+ Instance instance = getInstance(context);
+ tl = getTabletLocator(context, tableConfig.getTableName());
// its possible that the cache could contain complete, but old information about a tables tablets... so clear it
tl.invalidateCache();
- Credentials creds = new Credentials(getPrincipal(conf), getAuthenticationToken(conf));
+ Credentials creds = new Credentials(getPrincipal(context), getAuthenticationToken(context));
while (!tl.binRanges(creds, ranges, binnedRanges).isEmpty()) {
if (!(instance instanceof MockInstance)) {
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebd11205/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
index c7dfda5..cf4b376 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
@@ -17,8 +17,11 @@
package org.apache.accumulo.core.client.mapreduce;
import java.io.IOException;
+import java.util.List;
import java.util.Map.Entry;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Value;
@@ -38,19 +41,19 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
*
* <ul>
* <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, String)}
* <li>{@link AccumuloInputFormat#setScanAuthorizations(Job, Authorizations)}
* <li>{@link AccumuloInputFormat#setZooKeeperInstance(Job, String, String)} OR {@link AccumuloInputFormat#setMockInstance(Job, String)}
- * <li>{@link AccumuloInputFormat#setTableQueryConfigs(Job, BatchScanConfig...)}
* </ul>
*
* Other static methods are optional.
*/
public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
+
@Override
public RecordReader<Key,Value> createRecordReader(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
log.setLevel(getLogLevel(context));
return new RecordReaderBase<Key,Value>() {
+
@Override
public boolean nextKeyValue() throws IOException, InterruptedException {
if (scannerIterator.hasNext()) {
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebd11205/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
index f92de4b..7f8b47a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
@@ -1,5 +1,7 @@
package org.apache.accumulo.core.client.mapreduce;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Value;
@@ -10,6 +12,7 @@ import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import java.io.IOException;
+import java.util.List;
import java.util.Map;
import static com.google.common.base.Preconditions.checkNotNull;
@@ -33,7 +36,7 @@ public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value
@Override
public RecordReader<Key, Value> createRecordReader(InputSplit inputSplit, TaskAttemptContext context) throws IOException, InterruptedException {
log.setLevel(getLogLevel(context));
- return new RecordReaderBase<Key,Value>() {
+ return new AbstractRecordReader<Key, Value>() {
@Override
public boolean nextKeyValue() throws IOException, InterruptedException {
if (scannerIterator.hasNext()) {
@@ -47,6 +50,14 @@ public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value
}
return false;
}
+
+ @Override
+ protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName) {
+ List<IteratorSetting> iterators = getBatchScanConfig(context, tableName).getIterators();
+ for(IteratorSetting setting : iterators) {
+ scanner.addScanIterator(setting);
+ }
+ }
};
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebd11205/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
index 992990d..92ceec1 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
@@ -17,9 +17,12 @@
package org.apache.accumulo.core.client.mapreduce;
import java.io.IOException;
+import java.util.List;
import java.util.Map.Entry;
+import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.RowIterator;
+import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Value;
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebd11205/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
index d084b1a..eae973d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
@@ -172,7 +172,6 @@ public class BatchScanConfig implements Writable {
* @return true if the feature is enabled, false otherwise
* @since 1.6.0
* @see #setUseLocalIterators(boolean)
- * @deprecated since 1.6.0
*/
public boolean shouldUseLocalIterators() {
return useLocalIterators;
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ebd11205/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index ad9c454..f7057f1 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -24,6 +24,7 @@ import java.util.Set;
import org.apache.accumulo.core.client.ClientSideIteratorScanner;
import org.apache.accumulo.core.client.IsolatedScanner;
import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.TableNotFoundException;
import org.apache.accumulo.core.client.impl.TabletLocator;
import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
@@ -304,9 +305,23 @@ public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
* @throws org.apache.accumulo.core.client.TableNotFoundException
* if the table name set on the configuration doesn't exist
* @since 1.5.0
+ * @deprecated since 1.6.0
*/
+ @Deprecated
protected static TabletLocator getTabletLocator(JobContext context) throws TableNotFoundException {
return InputConfigurator.getTabletLocator(CLASS, getConfiguration(context), InputConfigurator.getInputTableName(CLASS, getConfiguration(context)));
}
-
+
+ protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
+ @Override
+ protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName) {
+ setupIterators(context, scanner);
+ }
+
+ protected void setupIterators(TaskAttemptContext context, Scanner scanner) {
+ List<IteratorSetting> iterators = getIterators(context);
+ for (IteratorSetting iterator : iterators)
+ scanner.addScanIterator(iterator);
+ }
+ }
}
[5/5] git commit: ACCUMULO-391 setters and getters for
BatchScanConfigs on jobs now use Map instead of a vararg.
Posted by cj...@apache.org.
ACCUMULO-391 setters and getters for BatchScanConfigs on jobs now use Map<String,BatchScanConfig> instead of a vararg.
Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/9a63ff4e
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/9a63ff4e
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/9a63ff4e
Branch: refs/heads/master
Commit: 9a63ff4ecf4b479403d16f1ee44b4f552f71719d
Parents: ebd1120
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Mon Oct 14 21:02:58 2013 -0400
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Tue Oct 15 20:22:07 2013 -0400
----------------------------------------------------------------------
.../core/client/mapred/InputFormatBase.java | 27 +--
.../client/mapreduce/AbstractInputFormat.java | 48 ++---
.../AccumuloMultiTableInputFormat.java | 22 ++-
.../core/client/mapreduce/BatchScanConfig.java | 87 ++++-----
.../core/client/mapreduce/InputFormatBase.java | 19 +-
.../mapreduce/lib/util/InputConfigurator.java | 181 ++++++++++---------
.../client/mapred/AccumuloInputFormatTest.java | 44 +++--
.../AccumuloMultiTableInputFormatTest.java | 28 ++-
.../core/conf/TableQueryConfigTest.java | 3 +-
9 files changed, 262 insertions(+), 197 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9a63ff4e/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
index 258a13a..0b13966 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
@@ -536,9 +536,9 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
* the table query configs to be set on the configuration.
* @since 1.6.0
*/
- public static void setTableQueryConfigs(JobConf job, BatchScanConfig... configs) {
+ public static void setTableQueryConfigs(JobConf job, Map<String, BatchScanConfig> configs) {
checkNotNull(configs);
- InputConfigurator.setTableQueryConfigs(CLASS, job, configs);
+ InputConfigurator.setBatchScanConfigs(CLASS, job, configs);
}
/**
@@ -554,8 +554,8 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
* @return
* @since 1.6.0
*/
- public static List<BatchScanConfig> getTableQueryConfigs(JobConf job) {
- return InputConfigurator.getTableQueryConfigs(CLASS, job);
+ public static Map<String,BatchScanConfig> getTableQueryConfigs(JobConf job) {
+ return InputConfigurator.getBatchScanConfigs(CLASS, job);
}
/**
@@ -828,9 +828,10 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
validateOptions(job);
LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
- List<BatchScanConfig> tableConfigs = getTableQueryConfigs(job);
- for (BatchScanConfig tableConfig : tableConfigs) {
-
+ Map<String, BatchScanConfig> tableConfigs = getTableQueryConfigs(job);
+ for (Entry<String, BatchScanConfig> tableConfigEntry : tableConfigs.entrySet()) {
+ String tableName = tableConfigEntry.getKey();
+ BatchScanConfig tableConfig = tableConfigEntry.getValue();
boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
String tableId = null;
List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
@@ -844,15 +845,15 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
TabletLocator tl;
try {
if (tableConfig.isOfflineScan()) {
- binnedRanges = binOfflineTable(job, tableConfig.getTableName(), ranges);
+ binnedRanges = binOfflineTable(job, tableName, ranges);
while (binnedRanges == null) {
// Some tablets were still online, try again
UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
- binnedRanges = binOfflineTable(job, tableConfig.getTableName(), ranges);
+ binnedRanges = binOfflineTable(job, tableName, ranges);
}
} else {
Instance instance = getInstance(job);
- tl = getTabletLocator(job, tableConfig.getTableName());
+ tl = getTabletLocator(job, tableName);
// its possible that the cache could contain complete, but old information about a tables tablets... so clear it
tl.invalidateCache();
Credentials creds = new Credentials(getPrincipal(job), getAuthenticationToken(job));
@@ -863,7 +864,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
throw new TableDeletedException(tableId);
if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
throw new TableOfflineException(instance, tableId);
- tableId = Tables.getTableId(instance, tableConfig.getTableName());
+ tableId = Tables.getTableId(instance, tableName);
}
binnedRanges.clear();
log.warn("Unable to locate bins for specified ranges. Retrying.");
@@ -894,7 +895,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
for (Range r : extentRanges.getValue()) {
if (autoAdjust) {
// divide ranges into smaller ranges, based on the tablets
- splits.add(new RangeInputSplit(tableConfig.getTableName(), tableId, ke.clip(r), new String[] {location}));
+ splits.add(new RangeInputSplit(tableName, tableId, ke.clip(r), new String[] {location}));
} else {
// don't divide ranges
ArrayList<String> locations = splitsToAdd.get(r);
@@ -909,7 +910,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
if (!autoAdjust)
for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
- splits.add(new RangeInputSplit(tableConfig.getTableName(), tableId, entry.getKey(), entry.getValue().toArray(new String[0])));
+ splits.add(new RangeInputSplit(tableName, tableId, entry.getKey(), entry.getValue().toArray(new String[0])));
}
return splits.toArray(new InputSplit[splits.size()]);
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9a63ff4e/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index d86e111..ed29dda 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -55,11 +55,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
-/**
- *
- * @param <K>
- * @param <V>
- */
public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
protected static final Class<?> CLASS = AccumuloInputFormat.class;
@@ -262,13 +257,13 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
/**
* Fetches all {@link BatchScanConfig}s that have been set on the given Hadoop configuration.
*
- * @param job
+ * @param context
* the Hadoop job instance to be configured
- * @return
+ * @return the {@link BatchScanConfig} objects for the job
* @since 1.6.0
*/
- protected static List<BatchScanConfig> getBatchScanConfigs(JobContext job) {
- return InputConfigurator.getTableQueryConfigs(CLASS, getConfiguration(job));
+ protected static Map<String, BatchScanConfig> getBatchScanConfigs(JobContext context) {
+ return InputConfigurator.getBatchScanConfigs(CLASS, getConfiguration(context));
}
/**
@@ -277,15 +272,15 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
* <p>
* null is returned in the event that the table doesn't exist.
*
- * @param job
+ * @param context
* the Hadoop job instance to be configured
* @param tableName
* the table name for which to grab the config object
* @return the {@link BatchScanConfig} for the given table
* @since 1.6.0
*/
- protected static BatchScanConfig getBatchScanConfig(JobContext job, String tableName) {
- return InputConfigurator.getTableQueryConfig(CLASS, getConfiguration(job), tableName);
+ protected static BatchScanConfig getBatchScanConfig(JobContext context, String tableName) {
+ return InputConfigurator.getTableQueryConfig(CLASS, getConfiguration(context), tableName);
}
/**
@@ -335,6 +330,15 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
protected Iterator<Map.Entry<Key,Value>> scannerIterator;
protected RangeInputSplit split;
+ /**
+ * Configures the iterators on a scanner for the given table name.
+ * @param context
+ * the Hadoop context for the configured job
+ * @param scanner
+ * the scanner for which to configure the iterators
+ * @param tableName
+ * the table name for which the scanner is configured
+ */
protected abstract void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName);
/**
@@ -544,8 +548,11 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
validateOptions(context);
LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
- List<BatchScanConfig> tableConfigs = getBatchScanConfigs(context);
- for (BatchScanConfig tableConfig : tableConfigs) {
+ Map<String, BatchScanConfig> tableConfigs = getBatchScanConfigs(context);
+ for (Map.Entry<String, BatchScanConfig> tableConfigEntry : tableConfigs.entrySet()) {
+
+ String tableName = tableConfigEntry.getKey();
+ BatchScanConfig tableConfig = tableConfigEntry.getValue();
boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
String tableId = null;
@@ -560,16 +567,16 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
TabletLocator tl;
try {
if (tableConfig.isOfflineScan()) {
- binnedRanges = binOfflineTable(context, tableConfig.getTableName(), ranges);
+ binnedRanges = binOfflineTable(context,tableName, ranges);
while (binnedRanges == null) {
// Some tablets were still online, try again
UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
- binnedRanges = binOfflineTable(context, tableConfig.getTableName(), ranges);
+ binnedRanges = binOfflineTable(context, tableName, ranges);
}
} else {
Instance instance = getInstance(context);
- tl = getTabletLocator(context, tableConfig.getTableName());
+ tl = getTabletLocator(context, tableName);
// its possible that the cache could contain complete, but old information about a tables tablets... so clear it
tl.invalidateCache();
Credentials creds = new Credentials(getPrincipal(context), getAuthenticationToken(context));
@@ -580,7 +587,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
throw new TableDeletedException(tableId);
if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
throw new TableOfflineException(instance, tableId);
- tableId = Tables.getTableId(instance, tableConfig.getTableName());
+ tableId = Tables.getTableId(instance, tableName);
}
binnedRanges.clear();
log.warn("Unable to locate bins for specified ranges. Retrying.");
@@ -611,7 +618,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
for (Range r : extentRanges.getValue()) {
if (autoAdjust) {
// divide ranges into smaller ranges, based on the tablets
- splits.add(new RangeInputSplit(tableConfig.getTableName(), tableId, ke.clip(r), new String[] {location}));
+ splits.add(new RangeInputSplit(tableName, tableId, ke.clip(r), new String[] {location}));
} else {
// don't divide ranges
ArrayList<String> locations = splitsToAdd.get(r);
@@ -626,7 +633,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
if (!autoAdjust)
for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
- splits.add(new RangeInputSplit(tableConfig.getTableName(), tableId, entry.getKey(), entry.getValue().toArray(new String[0])));
+ splits.add(new RangeInputSplit(tableName, tableId, entry.getKey(), entry.getValue().toArray(new String[0])));
}
return splits;
}
@@ -786,5 +793,4 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
throw new RuntimeException(e);
}
}
-
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9a63ff4e/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
index 7f8b47a..4208ff4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
@@ -17,6 +17,24 @@ import java.util.Map;
import static com.google.common.base.Preconditions.checkNotNull;
+/**
+ * This class allows MapReduce jobs to use multiple Accumulo tables as the source of data. This {@link org.apache.hadoop.mapreduce.InputFormat} provides keys
+ * and values of
+ * type {@link Key} and
+ * {@link Value} to the Map function.
+ *
+ * The user must specify the following via static configurator methods:
+ *
+ * <ul>
+ * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(Job, String, org.apache.accumulo.core.client.security.tokens.AuthenticationToken)}
+ * <li>{@link AccumuloMultiTableInputFormat#setScanAuthorizations(Job, org.apache.accumulo.core.security.Authorizations)}
+ * <li>{@link AccumuloMultiTableInputFormat#setZooKeeperInstance(Job, String, String)} OR {@link AccumuloInputFormat#setMockInstance(Job, String)}
+ * <li>{@link AccumuloMultiTableInputFormat#setBatchScanConfigs(org.apache.hadoop.mapreduce.Job, Map<String,BatchScanConfig>)}
+ * </ul>
+ *
+ * Other static methods are optional.
+ */
+
public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value>{
/**
@@ -28,9 +46,9 @@ public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value
* the table query configs to be set on the configuration.
* @since 1.6.0
*/
- public static void setBatchScanConfigs(Job job, BatchScanConfig... configs) {
+ public static void setBatchScanConfigs(Job job, Map<String, BatchScanConfig> configs) {
checkNotNull(configs);
- InputConfigurator.setTableQueryConfigs(CLASS, getConfiguration(job), configs);
+ InputConfigurator.setBatchScanConfigs(CLASS, getConfiguration(job), configs);
}
@Override
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9a63ff4e/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
index eae973d..feb49bb 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/BatchScanConfig.java
@@ -33,36 +33,31 @@ import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
/**
- * This class to holds a query configuration for a table. It contains all the properties needed to specify how rows should be returned from the table.
+ * This class to holds a batch scan configuration for a table. It contains all the properties needed to specify how rows should be returned from the table.
*/
public class BatchScanConfig implements Writable {
-
- private String tableName;
+
private List<IteratorSetting> iterators;
private List<Range> ranges;
private Set<Pair<Text,Text>> columns;
-
+
private boolean autoAdjustRanges = true;
private boolean useLocalIterators = false;
private boolean useIsolatedScanners = false;
private boolean offlineScan = false;
-
- public BatchScanConfig(String tableName) {
- checkNotNull(tableName);
- this.tableName = tableName;
- }
-
- public BatchScanConfig(DataInput input) throws IOException {
- readFields(input);
- }
+
+ public BatchScanConfig() {}
/**
- * Returns the table name associated with this configuration
+ * Creates a batch scan config object out of a previously serialized batch scan config object.
+ * @param input
+ * the data input of the serialized batch scan config
+ * @throws IOException
*/
- public String getTableName() {
- return tableName;
+ public BatchScanConfig(DataInput input) throws IOException {
+ readFields(input);
}
-
+
/**
* Sets the input ranges to scan for all tables associated with this job. This will be added to any per-table ranges that have been set using
*
@@ -74,14 +69,14 @@ public class BatchScanConfig implements Writable {
this.ranges = ranges;
return this;
}
-
+
/**
* Returns the ranges to be queried in the configuration
*/
public List<Range> getRanges() {
return ranges != null ? ranges : new ArrayList<Range>();
}
-
+
/**
* Restricts the columns that will be mapped over for this job for the default input table.
*
@@ -94,14 +89,14 @@ public class BatchScanConfig implements Writable {
this.columns = columns;
return this;
}
-
+
/**
* Returns the columns to be fetched for this configuration
*/
public Set<Pair<Text,Text>> getFetchedColumns() {
return columns != null ? columns : new HashSet<Pair<Text,Text>>();
}
-
+
/**
* Set iterators on to be used in the query.
*
@@ -113,14 +108,14 @@ public class BatchScanConfig implements Writable {
this.iterators = iterators;
return this;
}
-
+
/**
* Returns the iterators to be set on this configuration
*/
public List<IteratorSetting> getIterators() {
return iterators != null ? iterators : new ArrayList<IteratorSetting>();
}
-
+
/**
* Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
* Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
@@ -137,7 +132,7 @@ public class BatchScanConfig implements Writable {
this.autoAdjustRanges = autoAdjustRanges;
return this;
}
-
+
/**
* Determines whether a configuration has auto-adjust ranges enabled.
*
@@ -148,7 +143,7 @@ public class BatchScanConfig implements Writable {
public boolean shouldAutoAdjustRanges() {
return autoAdjustRanges;
}
-
+
/**
* Controls the use of the {@link org.apache.accumulo.core.client.ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack
* to be constructed within the Map task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be
@@ -165,7 +160,7 @@ public class BatchScanConfig implements Writable {
this.useLocalIterators = useLocalIterators;
return this;
}
-
+
/**
* Determines whether a configuration uses local iterators.
*
@@ -176,7 +171,7 @@ public class BatchScanConfig implements Writable {
public boolean shouldUseLocalIterators() {
return useLocalIterators;
}
-
+
/**
* <p>
* Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
@@ -210,7 +205,7 @@ public class BatchScanConfig implements Writable {
this.offlineScan = offlineScan;
return this;
}
-
+
/**
* Determines whether a configuration has the offline table scan feature enabled.
*
@@ -221,7 +216,7 @@ public class BatchScanConfig implements Writable {
public boolean isOfflineScan() {
return offlineScan;
}
-
+
/**
* Controls the use of the {@link org.apache.accumulo.core.client.IsolatedScanner} in this job.
*
@@ -236,7 +231,7 @@ public class BatchScanConfig implements Writable {
this.useIsolatedScanners = useIsolatedScanners;
return this;
}
-
+
/**
* Determines whether a configuration has isolation enabled.
*
@@ -247,10 +242,15 @@ public class BatchScanConfig implements Writable {
public boolean shouldUseIsolatedScanners() {
return useIsolatedScanners;
}
-
+
+ /**
+ * Writes the state for the current object out to the specified {@see DataOutput}
+ * @param dataOutput
+ * the output for which to write the object's state
+ * @throws IOException
+ */
@Override
public void write(DataOutput dataOutput) throws IOException {
- dataOutput.writeUTF(tableName);
if (iterators != null) {
dataOutput.writeInt(iterators.size());
for (IteratorSetting setting : iterators)
@@ -284,10 +284,15 @@ public class BatchScanConfig implements Writable {
dataOutput.writeBoolean(useLocalIterators);
dataOutput.writeBoolean(useIsolatedScanners);
}
-
+
+ /**
+ * Reads the fields in the {@see DataInput} into the current object
+ * @param dataInput
+ * the input fields to read into the current object
+ * @throws IOException
+ */
@Override
public void readFields(DataInput dataInput) throws IOException {
- this.tableName = dataInput.readUTF();
// load iterators
long iterSize = dataInput.readInt();
if (iterSize > 0)
@@ -323,16 +328,16 @@ public class BatchScanConfig implements Writable {
useLocalIterators = dataInput.readBoolean();
useIsolatedScanners = dataInput.readBoolean();
}
-
+
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
-
+
BatchScanConfig that = (BatchScanConfig) o;
-
+
if (autoAdjustRanges != that.autoAdjustRanges)
return false;
if (offlineScan != that.offlineScan)
@@ -347,16 +352,12 @@ public class BatchScanConfig implements Writable {
return false;
if (ranges != null ? !ranges.equals(that.ranges) : that.ranges != null)
return false;
- if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null)
- return false;
-
return true;
}
-
+
@Override
public int hashCode() {
- int result = tableName != null ? tableName.hashCode() : 0;
- result = 31 * result + (iterators != null ? iterators.hashCode() : 0);
+ int result = 31 * (iterators != null ? iterators.hashCode() : 0);
result = 31 * result + (ranges != null ? ranges.hashCode() : 0);
result = 31 * result + (columns != null ? columns.hashCode() : 0);
result = 31 * result + (autoAdjustRanges ? 1 : 0);
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9a63ff4e/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index f7057f1..af893a8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -99,7 +99,6 @@ public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
* @return the ranges
* @since 1.5.0
* @see #setRanges(Job, Collection)
- * @see #setRanges(org.apache.hadoop.mapreduce.Job, java.util.Collection)
*/
protected static List<Range> getRanges(JobContext context) throws IOException {
return InputConfigurator.getRanges(CLASS, getConfiguration(context));
@@ -313,11 +312,29 @@ public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
}
protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
+
+ /**
+ * Apply the configured iterators from the configuration to the scanner for the specified table name
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @param scanner
+ * the scanner to configure
+ * @since 1.6.0
+ */
@Override
protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName) {
setupIterators(context, scanner);
}
+ /**
+ * Apply the configured iterators from the configuration to the scanner.
+ *
+ * @param context
+ * the Hadoop context for the configured job
+ * @param scanner
+ * the scanner to configure
+ */
protected void setupIterators(TaskAttemptContext context, Scanner scanner) {
List<IteratorSetting> iterators = getIterators(context);
for (IteratorSetting iterator : iterators)
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9a63ff4e/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
index 1b76051..4dac750 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
@@ -25,11 +25,14 @@ import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import java.util.StringTokenizer;
+import com.google.common.collect.Maps;
import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -53,23 +56,25 @@ import org.apache.accumulo.core.util.Pair;
import org.apache.accumulo.core.util.TextUtil;
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.MapWritable;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
import org.apache.hadoop.util.StringUtils;
/**
* @since 1.5.0
*/
public class InputConfigurator extends ConfiguratorBase {
-
+
/**
* Configuration keys for {@link Scanner}.
*
* @since 1.5.0
*/
- public static enum ScanOpts {
+ public static enum ScanOpts {
TABLE_NAME, AUTHORIZATIONS, RANGES, COLUMNS, ITERATORS, TABLE_CONFIGS
}
-
+
/**
* Configuration keys for various features.
*
@@ -78,7 +83,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static enum Features {
AUTO_ADJUST_RANGES, SCAN_ISOLATION, USE_LOCAL_ITERATORS, SCAN_OFFLINE
}
-
+
/**
* Sets the name of the input table, over which this job will scan.
*
@@ -94,7 +99,7 @@ public class InputConfigurator extends ConfiguratorBase {
notNull(tableName);
conf.set(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME), tableName);
}
-
+
/**
* Sets the name of the input table, over which this job will scan.
*
@@ -107,7 +112,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static String getInputTableName(Class<?> implementingClass, Configuration conf) {
return conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME));
}
-
+
/**
* Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
*
@@ -123,7 +128,7 @@ public class InputConfigurator extends ConfiguratorBase {
if (auths != null && !auths.isEmpty())
conf.set(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS), auths.serialize());
}
-
+
/**
* Gets the authorizations to set for the scans from the configuration.
*
@@ -139,7 +144,7 @@ public class InputConfigurator extends ConfiguratorBase {
String authString = conf.get(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS));
return authString == null ? Authorizations.EMPTY : new Authorizations(authString.getBytes());
}
-
+
/**
* Sets the input ranges to scan on all input tables for this job. If not set, the entire table will be scanned.
*
@@ -155,7 +160,7 @@ public class InputConfigurator extends ConfiguratorBase {
*/
public static void setRanges(Class<?> implementingClass, Configuration conf, Collection<Range> ranges) {
notNull(ranges);
-
+
ArrayList<String> rangeStrings = new ArrayList<String>(ranges.size());
try {
for (Range r : ranges) {
@@ -168,7 +173,7 @@ public class InputConfigurator extends ConfiguratorBase {
throw new IllegalArgumentException("Unable to encode ranges to Base64", ex);
}
}
-
+
/**
* Gets the ranges to scan over from a job.
*
@@ -183,7 +188,7 @@ public class InputConfigurator extends ConfiguratorBase {
* @see #setRanges(Class, Configuration, Collection)
*/
public static List<Range> getRanges(Class<?> implementingClass, Configuration conf) throws IOException {
-
+
Collection<String> encodedRanges = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.RANGES));
List<Range> ranges = new ArrayList<Range>();
for (String rangeString : encodedRanges) {
@@ -194,7 +199,7 @@ public class InputConfigurator extends ConfiguratorBase {
}
return ranges;
}
-
+
/**
* Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
*
@@ -208,11 +213,11 @@ public class InputConfigurator extends ConfiguratorBase {
*/
public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
String iterators = conf.get(enumToConfKey(implementingClass, ScanOpts.ITERATORS));
-
+
// If no iterators are present, return an empty list
if (iterators == null || iterators.isEmpty())
return new ArrayList<IteratorSetting>();
-
+
// Compose the set of iterators encoded in the job configuration
StringTokenizer tokens = new StringTokenizer(iterators, StringUtils.COMMA_STR);
List<IteratorSetting> list = new ArrayList<IteratorSetting>();
@@ -228,7 +233,7 @@ public class InputConfigurator extends ConfiguratorBase {
}
return list;
}
-
+
/**
* Restricts the columns that will be mapped over for the single input table on this job.
*
@@ -247,10 +252,10 @@ public class InputConfigurator extends ConfiguratorBase {
notNull(columnFamilyColumnQualifierPairs);
ArrayList<String> columnStrings = new ArrayList<String>();
for (Pair<Text,Text> column : columnFamilyColumnQualifierPairs) {
-
+
if (column.getFirst() == null)
throw new IllegalArgumentException("Column family can not be null");
-
+
String col = new String(Base64.encodeBase64(TextUtil.getBytes(column.getFirst())), Constants.UTF8);
if (column.getSecond() != null)
col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())), Constants.UTF8);
@@ -258,7 +263,7 @@ public class InputConfigurator extends ConfiguratorBase {
}
conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings.toArray(new String[0]));
}
-
+
/**
* Gets the columns to be mapped over from this job.
*
@@ -280,7 +285,7 @@ public class InputConfigurator extends ConfiguratorBase {
}
return columns;
}
-
+
/**
* Encode an iterator on the input for the single input table associated with this job.
*
@@ -304,7 +309,7 @@ public class InputConfigurator extends ConfiguratorBase {
} catch (IOException e) {
throw new IllegalArgumentException("unable to serialize IteratorSetting");
}
-
+
String confKey = enumToConfKey(implementingClass, ScanOpts.ITERATORS);
String iterators = conf.get(confKey);
// No iterators specified yet, create a new string
@@ -317,7 +322,7 @@ public class InputConfigurator extends ConfiguratorBase {
// Store the iterators w/ the job
conf.set(confKey, iterators);
}
-
+
/**
* Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
* Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
@@ -337,7 +342,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static void setAutoAdjustRanges(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
conf.setBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), enableFeature);
}
-
+
/**
* Determines whether a configuration has auto-adjust ranges enabled.
*
@@ -352,7 +357,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static Boolean getAutoAdjustRanges(Class<?> implementingClass, Configuration conf) {
return conf.getBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), true);
}
-
+
/**
* Controls the use of the {@link IsolatedScanner} in this job.
*
@@ -370,7 +375,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static void setScanIsolation(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), enableFeature);
}
-
+
/**
* Determines whether a configuration has isolation enabled.
*
@@ -385,7 +390,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static Boolean isIsolated(Class<?> implementingClass, Configuration conf) {
return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), false);
}
-
+
/**
* Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack to be constructed within the Map
* task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be available on the classpath for the task.
@@ -404,7 +409,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static void setLocalIterators(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
conf.setBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), enableFeature);
}
-
+
/**
* Determines whether a configuration uses local iterators.
*
@@ -419,7 +424,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static Boolean usesLocalIterators(Class<?> implementingClass, Configuration conf) {
return conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false);
}
-
+
/**
* <p>
* Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
@@ -456,7 +461,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static void setOfflineTableScan(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), enableFeature);
}
-
+
/**
* Determines whether a configuration has the offline table scan feature enabled.
*
@@ -471,7 +476,7 @@ public class InputConfigurator extends ConfiguratorBase {
public static Boolean isOfflineScan(Class<?> implementingClass, Configuration conf) {
return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), false);
}
-
+
/**
* Sets configurations for multiple tables at a time.
*
@@ -479,25 +484,26 @@ public class InputConfigurator extends ConfiguratorBase {
* the class whose name will be used as a prefix for the property configuration key
* @param conf
* the Hadoop configuration object to configure
- * @param tconf
+ * @param configs
* an array of {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} objects to associate with the job
* @since 1.6.0
*/
- public static void setTableQueryConfigs(Class<?> implementingClass, Configuration conf, BatchScanConfig... tconf) {
- List<String> tableQueryConfigStrings = new ArrayList<String>();
- for (BatchScanConfig queryConfig : tconf) {
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- try {
- queryConfig.write(new DataOutputStream(baos));
- } catch (IOException e) {
- throw new IllegalStateException("Configuration for " + queryConfig.getTableName() + " could not be serialized.");
- }
- tableQueryConfigStrings.add(new String(Base64.encodeBase64(baos.toByteArray())));
+ public static void setBatchScanConfigs(Class<?> implementingClass, Configuration conf, Map<String,BatchScanConfig> configs) {
+ MapWritable mapWritable = new MapWritable();
+ for (Map.Entry<String,BatchScanConfig> tableConfig : configs.entrySet())
+ mapWritable.put(new Text(tableConfig.getKey()), tableConfig.getValue());
+
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ try {
+ mapWritable.write(new DataOutputStream(baos));
+ } catch (IOException e) {
+ throw new IllegalStateException("Table configuration could not be serialized.");
}
+
String confKey = enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS);
- conf.setStrings(confKey, tableQueryConfigStrings.toArray(new String[0]));
+ conf.set(confKey, new String(Base64.encodeBase64(baos.toByteArray())));
}
-
+
/**
* Returns all {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} objects associated with this job.
*
@@ -508,33 +514,29 @@ public class InputConfigurator extends ConfiguratorBase {
* @return all of the table query configs for the job
* @since 1.6.0
*/
- public static List<BatchScanConfig> getTableQueryConfigs(Class<?> implementingClass, Configuration conf) {
- List<BatchScanConfig> configs = new ArrayList<BatchScanConfig>();
- Collection<String> configStrings = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS));
- if (configStrings != null) {
- for (String str : configStrings) {
- try {
- byte[] bytes = Base64.decodeBase64(str.getBytes());
- ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
- configs.add(new BatchScanConfig(new DataInputStream(bais)));
- bais.close();
- } catch (IOException e) {
- throw new IllegalStateException("The table query configurations could not be deserialized from the given configuration");
- }
+ public static Map<String,BatchScanConfig> getBatchScanConfigs(Class<?> implementingClass, Configuration conf) {
+ Map<String,BatchScanConfig> configs = new HashMap<String,BatchScanConfig>();
+ Map.Entry<String, BatchScanConfig> defaultConfig = getDefaultBatchScanConfig(implementingClass, conf);
+ if(defaultConfig != null)
+ configs.put(defaultConfig.getKey(), defaultConfig.getValue());
+ String configString = conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS));
+ MapWritable mapWritable = new MapWritable();
+ if (configString != null) {
+ try {
+ byte[] bytes = Base64.decodeBase64(configString.getBytes());
+ ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+ mapWritable.readFields(new DataInputStream(bais));
+ bais.close();
+ } catch (IOException e) {
+ throw new IllegalStateException("The table query configurations could not be deserialized from the given configuration");
}
}
- BatchScanConfig defaultQueryConfig;
- try {
- defaultQueryConfig = getDefaultTableConfig(implementingClass, conf);
- } catch (IOException e) {
- throw new IllegalStateException("There was an error deserializing the default table configuration.");
- }
- if (defaultQueryConfig != null)
- configs.add(defaultQueryConfig);
-
+ for (Map.Entry<Writable,Writable> entry : mapWritable.entrySet())
+ configs.put(((Text) entry.getKey()).toString(), (BatchScanConfig) entry.getValue());
+
return configs;
}
-
+
/**
* Returns the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} for the given table
*
@@ -548,15 +550,10 @@ public class InputConfigurator extends ConfiguratorBase {
* @since 1.6.0
*/
public static BatchScanConfig getTableQueryConfig(Class<?> implementingClass, Configuration conf, String tableName) {
- List<BatchScanConfig> queryConfigs = getTableQueryConfigs(implementingClass, conf);
- for (BatchScanConfig queryConfig : queryConfigs) {
- if (queryConfig.getTableName().equals(tableName)) {
- return queryConfig;
- }
- }
- return null;
+ Map<String,BatchScanConfig> queryConfigs = getBatchScanConfigs(implementingClass, conf);
+ return queryConfigs.get(tableName);
}
-
+
/**
* Initializes an Accumulo {@link TabletLocator} based on the configuration.
*
@@ -578,7 +575,7 @@ public class InputConfigurator extends ConfiguratorBase {
Instance instance = getInstance(implementingClass, conf);
return TabletLocator.getLocator(instance, new Text(Tables.getTableId(instance, tableName)));
}
-
+
// InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
/**
* Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
@@ -592,6 +589,8 @@ public class InputConfigurator extends ConfiguratorBase {
* @since 1.5.0
*/
public static void validateOptions(Class<?> implementingClass, Configuration conf) throws IOException {
+
+ Map<String, BatchScanConfig> batchScanConfigs = getBatchScanConfigs(implementingClass, conf);
if (!isConnectorInfoSet(implementingClass, conf))
throw new IOException("Input info has not been set.");
String instanceKey = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
@@ -605,18 +604,20 @@ public class InputConfigurator extends ConfiguratorBase {
if (!c.securityOperations().authenticateUser(principal, token))
throw new IOException("Unable to authenticate user");
- for (BatchScanConfig tableConfig : getTableQueryConfigs(implementingClass, conf)) {
- if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), tableConfig.getTableName(), TablePermission.READ))
+ if(getBatchScanConfigs(implementingClass, conf).size() == 0)
+ throw new IOException("No table set.");
+
+ for (Map.Entry<String, BatchScanConfig> tableConfig : batchScanConfigs.entrySet()) {
+ if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), tableConfig.getKey(), TablePermission.READ))
throw new IOException("Unable to access table");
}
-
- for (BatchScanConfig tableConfig : getTableQueryConfigs(implementingClass, conf)) {
+ for (Map.Entry<String, BatchScanConfig> tableConfigEntry : batchScanConfigs.entrySet()) {
+ BatchScanConfig tableConfig = tableConfigEntry.getValue();
if (!tableConfig.shouldUseLocalIterators()) {
if (tableConfig.getIterators() != null) {
for (IteratorSetting iter : tableConfig.getIterators()) {
- if (!c.tableOperations().testClassLoad(tableConfig.getTableName(), iter.getIteratorClass(), SortedKeyValueIterator.class.getName()))
+ if (!c.tableOperations().testClassLoad(tableConfigEntry.getKey(), iter.getIteratorClass(), SortedKeyValueIterator.class.getName()))
throw new AccumuloException("Servers are unable to load " + iter.getIteratorClass() + " as a " + SortedKeyValueIterator.class.getName());
-
}
}
}
@@ -629,9 +630,10 @@ public class InputConfigurator extends ConfiguratorBase {
throw new IOException(e);
}
}
-
+
/**
- * Returns the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} for the configuration based on the properties set using the single-table input methods.
+ * Returns the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} for the configuration based on the properties set using the single-table
+ * input methods.
*
* @param implementingClass
* the class whose name will be used as a prefix for the property configuration key
@@ -641,23 +643,28 @@ public class InputConfigurator extends ConfiguratorBase {
* @throws IOException
* @since 1.6.0
*/
- protected static BatchScanConfig getDefaultTableConfig(Class<?> implementingClass, Configuration conf) throws IOException {
+ protected static Map.Entry<String, BatchScanConfig> getDefaultBatchScanConfig(Class<?> implementingClass, Configuration conf) {
String tableName = getInputTableName(implementingClass, conf);
if (tableName != null) {
- BatchScanConfig queryConfig = new BatchScanConfig(getInputTableName(implementingClass, conf));
+ BatchScanConfig queryConfig = new BatchScanConfig();
List<IteratorSetting> itrs = getIterators(implementingClass, conf);
if (itrs != null)
queryConfig.setIterators(itrs);
Set<Pair<Text,Text>> columns = getFetchedColumns(implementingClass, conf);
if (columns != null)
queryConfig.fetchColumns(columns);
- List<Range> ranges = getRanges(implementingClass, conf);
+ List<Range> ranges = null;
+ try {
+ ranges = getRanges(implementingClass, conf);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
if (ranges != null)
queryConfig.setRanges(ranges);
-
+
queryConfig.setAutoAdjustRanges(getAutoAdjustRanges(implementingClass, conf)).setUseIsolatedScanners(isIsolated(implementingClass, conf))
.setUseLocalIterators(usesLocalIterators(implementingClass, conf)).setOfflineScan(isOfflineScan(implementingClass, conf));
- return queryConfig;
+ return Maps.immutableEntry(tableName, queryConfig);
}
return null;
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9a63ff4e/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
index 70cf60d..6e44222 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
@@ -24,7 +24,9 @@ import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import org.apache.accumulo.core.client.BatchWriter;
import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -254,10 +256,14 @@ public class AccumuloInputFormatTest {
AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
- BatchScanConfig tableConfig1 = new BatchScanConfig(table1);
- BatchScanConfig tableConfig2 = new BatchScanConfig(table2);
+ BatchScanConfig tableConfig1 = new BatchScanConfig();
+ BatchScanConfig tableConfig2 = new BatchScanConfig();
+
+ Map<String, BatchScanConfig> configMap = new HashMap<String, BatchScanConfig>();
+ configMap.put(table1, tableConfig1);
+ configMap.put(table2, tableConfig2);
- AccumuloInputFormat.setTableQueryConfigs(job, tableConfig1, tableConfig2);
+ AccumuloInputFormat.setTableQueryConfigs(job, configMap);
job.setMapperClass(TestMapper.class);
job.setMapOutputKeyClass(Key.class);
@@ -306,16 +312,19 @@ public class AccumuloInputFormatTest {
JobConf job = new JobConf();
- BatchScanConfig table1 = new BatchScanConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
+ BatchScanConfig table1 = new BatchScanConfig().setRanges(Collections.singletonList(new Range("a", "b")))
.fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
.setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
- BatchScanConfig table2 = new BatchScanConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
+ BatchScanConfig table2 = new BatchScanConfig().setRanges(Collections.singletonList(new Range("a", "b")))
.fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
.setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
- AccumuloInputFormat.setTableQueryConfigs(job, table1, table2);
-
+ Map<String, BatchScanConfig> configMap = new HashMap<String, BatchScanConfig>();
+ configMap.put(TEST_TABLE_1, table1);
+ configMap.put(TEST_TABLE_2, table2);
+ AccumuloInputFormat.setTableQueryConfigs(job, configMap);
+
assertEquals(table1, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_1));
assertEquals(table2, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_2));
}
@@ -328,21 +337,20 @@ public class AccumuloInputFormatTest {
JobConf job = new JobConf();
- BatchScanConfig table1 = new BatchScanConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
+ BatchScanConfig tableConfig = new BatchScanConfig().setRanges(Collections.singletonList(new Range("a", "b")))
.fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
.setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
- BatchScanConfig table2 = new BatchScanConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
- .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
- .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
+ Map<String, BatchScanConfig> configMap = new HashMap<String, BatchScanConfig>();
+ configMap.put(TEST_TABLE_1, tableConfig);
- AccumuloInputFormat.setTableQueryConfigs(job, table1);
- AccumuloInputFormat.setInputTableName(job, table2.getTableName());
- AccumuloInputFormat.setRanges(job, table2.getRanges());
- AccumuloInputFormat.fetchColumns(job, table2.getFetchedColumns());
- AccumuloInputFormat.addIterator(job, table2.getIterators().get(0));
+ AccumuloInputFormat.setTableQueryConfigs(job, configMap);
+ AccumuloInputFormat.setInputTableName(job, TEST_TABLE_2);
+ AccumuloInputFormat.setRanges(job, tableConfig.getRanges());
+ AccumuloInputFormat.fetchColumns(job, tableConfig.getFetchedColumns());
+ AccumuloInputFormat.addIterator(job, tableConfig.getIterators().get(0));
- assertEquals(table1, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_1));
- assertEquals(table2, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_2));
+ assertEquals(tableConfig, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_1));
+ assertEquals(tableConfig, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_2));
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9a63ff4e/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
index 6b2eba1..6bbb1d6 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
@@ -23,6 +23,8 @@ import org.junit.Test;
import java.io.IOException;
import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
@@ -87,10 +89,14 @@ public class AccumuloMultiTableInputFormatTest {
AccumuloMultiTableInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
- BatchScanConfig tableConfig1 = new BatchScanConfig(table1);
- BatchScanConfig tableConfig2 = new BatchScanConfig(table2);
+ BatchScanConfig tableConfig1 = new BatchScanConfig();
+ BatchScanConfig tableConfig2 = new BatchScanConfig();
- AccumuloMultiTableInputFormat.setBatchScanConfigs(job, tableConfig1, tableConfig2);
+ Map<String, BatchScanConfig> configMap = new HashMap<String, BatchScanConfig>();
+ configMap.put(table1, tableConfig1);
+ configMap.put(table2, tableConfig2);
+
+ AccumuloMultiTableInputFormat.setBatchScanConfigs(job, configMap);
AccumuloMultiTableInputFormat.setMockInstance(job, INSTANCE_NAME);
job.setMapperClass(TestMapper.class);
@@ -145,18 +151,20 @@ public class AccumuloMultiTableInputFormatTest {
Job job = new Job();
- BatchScanConfig table1 = new BatchScanConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
- .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
- .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
- BatchScanConfig table2 = new BatchScanConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
+
+ BatchScanConfig tableConfig= new BatchScanConfig().setRanges(Collections.singletonList(new Range("a", "b")))
.fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
.setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
+
+ Map<String, BatchScanConfig> configMap = new HashMap<String, BatchScanConfig>();
+ configMap.put(TEST_TABLE_1, tableConfig);
+ configMap.put(TEST_TABLE_2, tableConfig);
- AccumuloMultiTableInputFormat.setBatchScanConfigs(job, table1, table2);
+ AccumuloMultiTableInputFormat.setBatchScanConfigs(job, configMap);
- assertEquals(table1, AccumuloMultiTableInputFormat.getBatchScanConfig(job, TEST_TABLE_1));
- assertEquals(table2, AccumuloMultiTableInputFormat.getBatchScanConfig(job, TEST_TABLE_2));
+ assertEquals(tableConfig, AccumuloMultiTableInputFormat.getBatchScanConfig(job, TEST_TABLE_1));
+ assertEquals(tableConfig, AccumuloMultiTableInputFormat.getBatchScanConfig(job, TEST_TABLE_2));
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9a63ff4e/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java b/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
index 9910182..a2b0db0 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
@@ -38,12 +38,11 @@ import org.junit.Test;
public class TableQueryConfigTest {
- private static final String TEST_TABLE = "TEST_TABLE";
private BatchScanConfig tableQueryConfig;
@Before
public void setUp() {
- tableQueryConfig = new BatchScanConfig(TEST_TABLE);
+ tableQueryConfig = new BatchScanConfig();
}
@Test
[3/5] git commit: ACCUMULO-1751 Adding tests for autoAdjustRanges and
offlineTable paths through getSplits().
Posted by cj...@apache.org.
ACCUMULO-1751 Adding tests for autoAdjustRanges and offlineTable paths through getSplits().
Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/f8197677
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/f8197677
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/f8197677
Branch: refs/heads/master
Commit: f8197677aa482d8118f884f3ea3a08495a4344ed
Parents: 87b104d
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Mon Oct 14 01:23:47 2013 -0400
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Tue Oct 15 20:22:07 2013 -0400
----------------------------------------------------------------------
.../test/functional/AccumuloInputFormatIT.java | 67 +++++++++++++++-----
1 file changed, 51 insertions(+), 16 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f8197677/test/src/test/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java b/test/src/test/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
index b4258c5..77769a5 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
@@ -2,8 +2,10 @@ package org.apache.accumulo.test.functional;
import static java.lang.System.currentTimeMillis;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.TreeSet;
@@ -16,48 +18,81 @@ import org.apache.accumulo.core.client.TableNotFoundException;
import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
import org.apache.accumulo.core.client.security.tokens.PasswordToken;
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.util.UtilWaitThread;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
+import org.junit.Before;
import org.junit.Test;
public class AccumuloInputFormatIT extends SimpleMacIT {
+ AccumuloInputFormat inputFormat;
+
+ @Before
+ public void before() {
+ inputFormat = new AccumuloInputFormat();
+ }
+
+ /**
+ * Tests several different paths through the getSplits() method by setting different properties and verifying the results.
+ */
@Test
public void testGetSplits() throws IOException, AccumuloSecurityException, AccumuloException, TableNotFoundException, TableExistsException {
-
String table = getTableNames(1)[0];
getConnector().tableOperations().create(table);
-
- // add data
-
- // add splits
+ insertData(table, currentTimeMillis());
Job job = new Job();
-
- // set up the job here
AccumuloInputFormat.setInputTableName(job, table);
AccumuloInputFormat.setZooKeeperInstance(job, getConnector().getInstance().getInstanceName(), getConnector().getInstance().getZooKeepers());
AccumuloInputFormat.setConnectorInfo(job, "root", new PasswordToken(ROOT_PASSWORD));
- AccumuloInputFormat inputFormat = new AccumuloInputFormat();
-
- insertData(table, currentTimeMillis());
-
+ // split table
TreeSet<Text> splitsToAdd = new TreeSet<Text>();
for (int i = 0; i < 10000; i += 1000)
splitsToAdd.add(new Text(String.format("%09d", i)));
-
getConnector().tableOperations().addSplits(table, splitsToAdd);
+ UtilWaitThread.sleep(500); // wait for splits to be propagated
- UtilWaitThread.sleep(5000);
+ // get splits without setting any range
Collection<Text> actualSplits = getConnector().tableOperations().listSplits(table);
-
List<InputSplit> splits = inputFormat.getSplits(job);
- assertEquals(actualSplits.size()+1, splits.size());
-
+ assertEquals(actualSplits.size() + 1, splits.size()); // No ranges set on the job so it'll start with -inf
+
+ // set ranges and get splits
+ List<Range> ranges = new ArrayList<Range>();
+ for (Text text : actualSplits)
+ ranges.add(new Range(text));
+ AccumuloInputFormat.setRanges(job, ranges);
+ splits = inputFormat.getSplits(job);
+ assertEquals(actualSplits.size(), splits.size());
+
+ // offline mode
+ AccumuloInputFormat.setOfflineTableScan(job, true);
+ try {
+ inputFormat.getSplits(job);
+ fail("An exception should have been thrown");
+ } catch (Exception e) {}
+
+ getConnector().tableOperations().offline(table);
+ splits = inputFormat.getSplits(job);
+ assertEquals(actualSplits.size(), splits.size());
+>>>>>>> ACCUMULO-1751 Adding tests for autoAdjustRanges and offlineTable paths through getSplits().
+
+ // auto adjust ranges
+ ranges = new ArrayList<Range>();
+ for (int i = 0; i < 5; i++) // overlapping ranges
+ ranges.add(new Range(String.format("%09d", i), String.format("%09d", i + 2)));
+ AccumuloInputFormat.setRanges(job, ranges);
+ splits = inputFormat.getSplits(job);
+ assertEquals(2, splits.size());
+
+ AccumuloInputFormat.setAutoAdjustRanges(job, false);
+ splits = inputFormat.getSplits(job);
+ assertEquals(ranges.size(), splits.size());
}
private void insertData(String tableName, long ts) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {