Merge branch '2.1'
diff --git a/LICENSE b/LICENSE
index 1c80cd5..22519c1 100644
--- a/LICENSE
+++ b/LICENSE
@@ -291,7 +291,8 @@
Copyright (c) 2007-2014 IOLA and Ole Laursen
- Available under the MIT License (see above and https://github.com/flot/flot/blob/master/LICENSE.txt)
+ Available under the MIT License
+ (see server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/external/flot/LICENSE.txt)
Flot bundles additional works:
diff --git a/assemble/bin/accumulo-cluster b/assemble/bin/accumulo-cluster
index d31d659..f525093 100755
--- a/assemble/bin/accumulo-cluster
+++ b/assemble/bin/accumulo-cluster
@@ -49,14 +49,6 @@
function parse_config {
- for file in slaves tservers monitor tracers gc managers masters; do
- if [[ -f ${conf}/${file} ]]; then
- echo "ERROR: A '${file}' file was found in ${conf}/"
- echo "Accumulo now uses cluster host configuration information from 'cluster.yaml' and requires that the '${file}' file not be present to reduce confusion."
- exit 1
- fi
- done
-
if [[ ! -f ${conf}/cluster.yaml ]]; then
echo "ERROR: A 'cluster.yaml' file was not found at ${conf}/cluster.yaml"
echo "Please make sure it exists and is configured with the host information. Run 'accumulo-cluster create-config' to create an example configuration."
@@ -149,16 +141,16 @@
# using the value of $host
#
if [[ $# -gt 3 ]]; then
- ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-a" "$host" "${@:4}"
+ ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-o" "general.process.bind.addr=$host" "${@:4}"
else
- ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-a" "$host"
+ ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-o" "general.process.bind.addr=$host"
fi
else
if [[ $# -gt 3 ]]; then
EXTRA_ARGS="${*:4}"
- $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-a\" \"$host\" $EXTRA_ARGS '"
+ $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-o\" \"general.process.bind.addr=$host\" $EXTRA_ARGS '"
else
- $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-a\" \"$host\"'"
+ $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-o\" \"general.process.bind.addr=$host\"'"
fi
fi
done
@@ -204,7 +196,7 @@
for group in $SSERVER_GROUPS; do
G="SSERVER_HOSTS_${group}"
for sserver in ${!G}; do
- start_service "$sserver" sserver "-g" "$group"
+ start_service "$sserver" sserver "-o" "sserver.group=$group"
done
done
@@ -215,7 +207,7 @@
for queue in $COMPACTION_QUEUES; do
Q="COMPACTOR_HOSTS_${queue}"
for compactor in ${!Q}; do
- start_service "$compactor" compactor "-q" "$queue"
+ start_service "$compactor" compactor "-o" "compactor.queue=$queue"
done
done
@@ -341,7 +333,7 @@
for group in $SSERVER_GROUPS; do
G="SSERVER_HOSTS_${group}"
for sserver in ${!G}; do
- kill_service "$sserver" sserver "-g" "$group"
+ kill_service "$sserver" sserver "-o" "sserver.group=$group"
done
done
@@ -356,7 +348,7 @@
for queue in $COMPACTION_QUEUES; do
Q="COMPACTOR_HOSTS_${queue}"
for compactor in ${!Q}; do
- kill_service "$compactor" compactor "-q" "$queue"
+ kill_service "$compactor" compactor "-o" "compactor.queue=$queue"
done
done
@@ -397,7 +389,7 @@
for group in $SSERVER_GROUPS; do
G="SSERVER_HOSTS_${group}"
for sserver in ${!G}; do
- end_service $end_cmd "$sserver" sserver "-g" "$group"
+ end_service $end_cmd "$sserver" sserver "-o" "sserver.group=$group"
done
done
@@ -408,7 +400,7 @@
for queue in $COMPACTION_QUEUES; do
Q="COMPACTOR_HOSTS_${queue}"
for compactor in ${!Q}; do
- end_service $end_cmd "$compactor" compactor "-q" "$queue"
+ end_service $end_cmd "$compactor" compactor "-o" "compactor.queue=$queue"
done
done
@@ -445,13 +437,13 @@
for group in $SSERVER_GROUPS; do
G="SSERVER_HOSTS_${group}"
for sserver in ${!G}; do
- end_service $end_cmd "$sserver" sserver "-g" "$group"
+ end_service $end_cmd "$sserver" sserver "-o" "sserver.group=$group"
done
done
for queue in $COMPACTION_QUEUES; do
Q="COMPACTOR_HOSTS_${queue}"
for compactor in ${!Q}; do
- end_service $end_cmd "$host" compactor "-q" "$queue"
+ end_service $end_cmd "$host" compactor "-o" "compactor.queue=$queue"
done
done
done
diff --git a/assemble/bin/accumulo-service b/assemble/bin/accumulo-service
index 552c2b2..c501876 100755
--- a/assemble/bin/accumulo-service
+++ b/assemble/bin/accumulo-service
@@ -26,7 +26,6 @@
gc Accumulo garbage collector
monitor Accumulo monitor
manager Accumulo manager
- master Accumulo master (Deprecated)
tserver Accumulo tserver
compaction-coordinator Accumulo compaction coordinator (experimental)
compactor Accumulo compactor (experimental)
@@ -143,11 +142,6 @@
fi
service="$1"
- if [[ $service == "master" ]]; then
- echo "WARN : Use of 'master' service name is deprecated; use 'manager' instead."
- service="manager"
- fi
-
pid_file="${ACCUMULO_PID_DIR}/accumulo-${service}${ACCUMULO_SERVICE_INSTANCE}.pid"
case "$service" in
gc | manager | monitor | tserver | compaction-coordinator | compactor | sserver)
diff --git a/assemble/conf/accumulo-env.sh b/assemble/conf/accumulo-env.sh
index bcde690..9e4acc1 100644
--- a/assemble/conf/accumulo-env.sh
+++ b/assemble/conf/accumulo-env.sh
@@ -90,7 +90,7 @@
# cmd is set by calling script that sources this env file
#shellcheck disable=SC2154
case "$cmd" in
- manager | master) JAVA_OPTS=('-Xmx512m' '-Xms512m' "${JAVA_OPTS[@]}") ;;
+ manager) JAVA_OPTS=('-Xmx512m' '-Xms512m' "${JAVA_OPTS[@]}") ;;
monitor) JAVA_OPTS=('-Xmx256m' '-Xms256m' "${JAVA_OPTS[@]}") ;;
gc) JAVA_OPTS=('-Xmx256m' '-Xms256m' "${JAVA_OPTS[@]}") ;;
tserver) JAVA_OPTS=('-Xmx768m' '-Xms768m' "${JAVA_OPTS[@]}") ;;
@@ -118,7 +118,7 @@
#JAVA_OPTS=('-javaagent:path/to/opentelemetry-javaagent-all.jar' "${JAVA_OPTS[@]}")
case "$cmd" in
- monitor | gc | manager | master | tserver | compaction-coordinator | compactor | sserver)
+ monitor | gc | manager | tserver | compaction-coordinator | compactor | sserver)
JAVA_OPTS=('-Dlog4j.configurationFile=log4j2-service.properties' "${JAVA_OPTS[@]}")
;;
*)
diff --git a/assemble/pom.xml b/assemble/pom.xml
index d24a608..4b7fe16 100644
--- a/assemble/pom.xml
+++ b/assemble/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.accumulo</groupId>
<artifactId>accumulo-project</artifactId>
- <version>2.1.3-SNAPSHOT</version>
+ <version>3.1.0-SNAPSHOT</version>
</parent>
<artifactId>accumulo</artifactId>
<packaging>pom</packaging>
@@ -92,11 +92,6 @@
<optional>true</optional>
</dependency>
<dependency>
- <groupId>com.google.protobuf</groupId>
- <artifactId>protobuf-java</artifactId>
- <optional>true</optional>
- </dependency>
- <dependency>
<groupId>com.lmax</groupId>
<artifactId>disruptor</artifactId>
<optional>true</optional>
@@ -188,6 +183,11 @@
</dependency>
<dependency>
<groupId>org.apache.accumulo</groupId>
+ <artifactId>accumulo-access</artifactId>
+ <optional>true</optional>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.accumulo</groupId>
<artifactId>accumulo-compaction-coordinator</artifactId>
<optional>true</optional>
</dependency>
@@ -278,11 +278,6 @@
<optional>true</optional>
</dependency>
<dependency>
- <groupId>org.apache.commons</groupId>
- <artifactId>commons-vfs2</artifactId>
- <optional>true</optional>
- </dependency>
- <dependency>
<groupId>org.apache.datasketches</groupId>
<artifactId>datasketches-java</artifactId>
<optional>true</optional>
diff --git a/assemble/src/main/resources/LICENSE b/assemble/src/main/resources/LICENSE
index e01f6ae..9bf302b 100644
--- a/assemble/src/main/resources/LICENSE
+++ b/assemble/src/main/resources/LICENSE
@@ -290,7 +290,8 @@
Copyright (c) 2007-2014 IOLA and Ole Laursen
- Available under the MIT License (see above and https://github.com/flot/flot/blob/master/LICENSE.txt)
+ Available under the MIT License
+ (see org/apache/accumulo/monitor/resources/external/flot/LICENSE.txt in the accumulo-monitor jar)
Flot bundles additional works:
diff --git a/core/pom.xml b/core/pom.xml
index 9822afe..fcaecd8 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.accumulo</groupId>
<artifactId>accumulo-project</artifactId>
- <version>2.1.3-SNAPSHOT</version>
+ <version>3.1.0-SNAPSHOT</version>
</parent>
<artifactId>accumulo-core</artifactId>
<name>Apache Accumulo Core</name>
@@ -62,10 +62,6 @@
</exclusions>
</dependency>
<dependency>
- <groupId>com.google.protobuf</groupId>
- <artifactId>protobuf-java</artifactId>
- </dependency>
- <dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
@@ -87,6 +83,10 @@
</dependency>
<dependency>
<groupId>org.apache.accumulo</groupId>
+ <artifactId>accumulo-access</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.accumulo</groupId>
<artifactId>accumulo-start</artifactId>
</dependency>
<dependency>
@@ -209,7 +209,6 @@
<exclude>src/main/java/org/apache/accumulo/core/bloomfilter/*.java</exclude>
<exclude>src/main/java/org/apache/accumulo/core/util/HostAndPort.java</exclude>
<exclude>src/test/resources/*.jceks</exclude>
- <exclude>src/test/resources/org/apache/accumulo/core/file/rfile/*.rf</exclude>
</excludes>
</licenseSet>
</licenseSets>
@@ -222,7 +221,6 @@
<excludes>
<exclude>src/main/java/org/apache/accumulo/core/bloomfilter/*.java</exclude>
<exclude>src/test/resources/*.jceks</exclude>
- <exclude>src/test/resources/org/apache/accumulo/core/file/rfile/*.rf</exclude>
</excludes>
</configuration>
</plugin>
@@ -274,6 +272,8 @@
<allow>javax[.]security[.]auth[.]DestroyFailedException</allow>
<!-- allow questionable Hadoop exceptions for mapreduce -->
<allow>org[.]apache[.]hadoop[.]mapred[.](FileAlreadyExistsException|InvalidJobConfException)</allow>
+ <!-- allow the following types from the visibility API -->
+ <allow>org[.]apache[.]accumulo[.]access[.].*</allow>
</allows>
</configuration>
</execution>
@@ -368,7 +368,7 @@
<classpathScope>test</classpathScope>
<arguments>
<argument>--generate-markdown</argument>
- <argument>${project.build.directory}/generated-docs/server-properties.md</argument>
+ <argument>${project.build.directory}/generated-docs/server-properties3.md</argument>
</arguments>
</configuration>
</execution>
@@ -383,7 +383,7 @@
<classpathScope>test</classpathScope>
<arguments>
<argument>--generate-markdown</argument>
- <argument>${project.build.directory}/generated-docs/client-properties.md</argument>
+ <argument>${project.build.directory}/generated-docs/client-properties3.md</argument>
</arguments>
</configuration>
</execution>
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index c31e205..411af75 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -35,6 +35,7 @@
// Zookeeper locations
public static final String ZROOT = "/accumulo";
public static final String ZINSTANCES = "/instances";
+ public static final String ZUSERS = "/users";
public static final String ZTABLES = "/tables";
public static final byte[] ZTABLES_INITIAL_ID = {'0'};
@@ -52,7 +53,6 @@
public static final String ZMANAGERS = "/managers";
public static final String ZMANAGER_LOCK = ZMANAGERS + "/lock";
public static final String ZMANAGER_GOAL_STATE = ZMANAGERS + "/goal_state";
- public static final String ZMANAGER_REPLICATION_COORDINATOR_ADDR = ZMANAGERS + "/repl_coord_addr";
public static final String ZMANAGER_TICK = ZMANAGERS + "/tick";
public static final String ZGC = "/gc";
@@ -62,8 +62,6 @@
public static final String ZMONITOR_LOCK = ZMONITOR + "/lock";
public static final String ZMONITOR_HTTP_ADDR = ZMONITOR + "/http_addr";
- // used by < 2.1 table and namespace configurations
- public static final String ZCONF_LEGACY = "/conf";
public static final String ZCONFIG = "/config";
public static final String ZTSERVERS = "/tservers";
@@ -78,8 +76,6 @@
public static final String ZDEAD = "/dead";
public static final String ZDEADTSERVERS = ZDEAD + "/tservers";
- public static final String ZTRACERS = "/tracers";
-
public static final String ZPROBLEMS = "/problems";
public static final String BULK_ARBITRATOR_TYPE = "bulkTx";
@@ -88,9 +84,6 @@
public static final String ZNEXT_FILE = "/next_file";
- // TODO: Remove when Property.TSERV_WORKQ_THREADS is removed
- public static final String ZBULK_FAILED_COPYQ = "/bulk_failed_copyq";
-
public static final String ZHDFS_RESERVATIONS = "/hdfs_reservations";
public static final String ZRECOVERY = "/recovery";
@@ -117,7 +110,6 @@
public static final int MAX_DATA_TO_PRINT = 64;
public static final String CORE_PACKAGE_NAME = "org.apache.accumulo.core";
- public static final String MAPFILE_EXTENSION = "map";
public static final String GENERATED_TABLET_DIRECTORY_PREFIX = "t-";
public static final String EXPORT_METADATA_FILE = "metadata.bin";
diff --git a/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java b/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java
index 4928767..dc84a63 100644
--- a/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java
@@ -45,7 +45,7 @@
// load the default implementation
LOG.info("Using default {}, which is subject to change in a future release",
ContextClassLoaderFactory.class.getName());
- FACTORY = new DefaultContextClassLoaderFactory(conf);
+ FACTORY = new URLContextClassLoaderFactory();
} else {
// load user's selected implementation and provide it with the service environment
try {
@@ -73,12 +73,15 @@
FACTORY = null;
}
- @SuppressWarnings("deprecation")
+ public static ClassLoader getClassLoader() {
+ return getClassLoader(null);
+ }
+
public static ClassLoader getClassLoader(String context) {
if (context != null && !context.isEmpty()) {
return FACTORY.getClassLoader(context);
} else {
- return org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.getClassLoader();
+ return ClassLoader.getSystemClassLoader();
}
}
@@ -114,9 +117,8 @@
/**
* Retrieve the classloader context from a table's configuration.
*/
- @SuppressWarnings("removal")
public static String tableContext(AccumuloConfiguration conf) {
- return conf.get(conf.resolve(Property.TABLE_CLASSLOADER_CONTEXT, Property.TABLE_CLASSPATH));
+ return conf.get(Property.TABLE_CLASSLOADER_CONTEXT);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java b/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
deleted file mode 100644
index 152c2b9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
+++ /dev/null
@@ -1,101 +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
- *
- * https://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.classloader;
-
-import static java.util.concurrent.TimeUnit.MINUTES;
-
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory;
-import org.apache.accumulo.core.util.threads.ThreadPools;
-import org.apache.accumulo.core.util.threads.Threads;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The default implementation of ContextClassLoaderFactory. This implementation is subject to change
- * over time. It currently implements the legacy context class loading behavior based on Accumulo's
- * custom class loaders and commons-vfs2. In future, it may simply return the system class loader
- * for all requested contexts. This class is used internally to Accumulo only, and should not be
- * used by users in their configuration.
- */
-public class DefaultContextClassLoaderFactory implements ContextClassLoaderFactory {
-
- private static final AtomicBoolean isInstantiated = new AtomicBoolean(false);
- private static final Logger LOG = LoggerFactory.getLogger(DefaultContextClassLoaderFactory.class);
- private static final String className = DefaultContextClassLoaderFactory.class.getName();
-
- @SuppressWarnings("removal")
- private static final Property VFS_CONTEXT_CLASSPATH_PROPERTY =
- Property.VFS_CONTEXT_CLASSPATH_PROPERTY;
-
- public DefaultContextClassLoaderFactory(final AccumuloConfiguration accConf) {
- if (!isInstantiated.compareAndSet(false, true)) {
- throw new IllegalStateException("Can only instantiate " + className + " once");
- }
- Supplier<Map<String,String>> contextConfigSupplier =
- () -> accConf.getAllPropertiesWithPrefix(VFS_CONTEXT_CLASSPATH_PROPERTY);
- setContextConfig(contextConfigSupplier);
- LOG.debug("ContextManager configuration set");
- startCleanupThread(accConf, contextConfigSupplier);
- }
-
- @SuppressWarnings("deprecation")
- private static void setContextConfig(Supplier<Map<String,String>> contextConfigSupplier) {
- org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader
- .setContextConfig(contextConfigSupplier);
- }
-
- private static void startCleanupThread(final AccumuloConfiguration conf,
- final Supplier<Map<String,String>> contextConfigSupplier) {
- ScheduledFuture<?> future = ThreadPools.getClientThreadPools((t, e) -> {
- LOG.error("context classloader cleanup thread has failed.", e);
- }).createGeneralScheduledExecutorService(conf)
- .scheduleWithFixedDelay(Threads.createNamedRunnable(className + "-cleanup", () -> {
- LOG.trace("{}-cleanup thread, properties: {}", className, conf);
- Set<String> contextsInUse = contextConfigSupplier.get().keySet().stream()
- .map(p -> p.substring(VFS_CONTEXT_CLASSPATH_PROPERTY.getKey().length()))
- .collect(Collectors.toSet());
- LOG.trace("{}-cleanup thread, contexts in use: {}", className, contextsInUse);
- removeUnusedContexts(contextsInUse);
- }), 1, 1, MINUTES);
- ThreadPools.watchNonCriticalScheduledTask(future);
- LOG.debug("Context cleanup timer started at 60s intervals");
- }
-
- @SuppressWarnings("deprecation")
- private static void removeUnusedContexts(Set<String> contextsInUse) {
- org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader
- .removeUnusedContexts(contextsInUse);
- }
-
- @SuppressWarnings("deprecation")
- @Override
- public ClassLoader getClassLoader(String contextName) {
- return org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader
- .getContextClassLoader(contextName);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/classloader/URLContextClassLoaderFactory.java b/core/src/main/java/org/apache/accumulo/core/classloader/URLContextClassLoaderFactory.java
new file mode 100644
index 0000000..26b06e3
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/classloader/URLContextClassLoaderFactory.java
@@ -0,0 +1,75 @@
+/*
+ * 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
+ *
+ * https://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.classloader;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+
+/**
+ * The default implementation of ContextClassLoaderFactory. This classloader returns a
+ * URLClassLoader based on the given context value which is a CSV list of URLs. For example,
+ * file://path/one/jar1.jar,file://path/two/jar2.jar
+ */
+public class URLContextClassLoaderFactory implements ContextClassLoaderFactory {
+
+ private static final AtomicBoolean isInstantiated = new AtomicBoolean(false);
+ private static final Logger LOG = LoggerFactory.getLogger(URLContextClassLoaderFactory.class);
+ private static final String className = URLContextClassLoaderFactory.class.getName();
+
+ // Cache the class loaders for re-use
+ // WeakReferences are used so that the class loaders can be cleaned up when no longer needed
+ // Classes that are loaded contain a reference to the class loader used to load them
+ // so the class loader will be garbage collected when no more classes are loaded that reference it
+ private final Cache<String,URLClassLoader> classloaders =
+ Caffeine.newBuilder().weakValues().build();
+
+ public URLContextClassLoaderFactory() {
+ if (!isInstantiated.compareAndSet(false, true)) {
+ throw new IllegalStateException("Can only instantiate " + className + " once");
+ }
+ }
+
+ @Override
+ public ClassLoader getClassLoader(String context) {
+ if (context == null) {
+ throw new IllegalArgumentException("Unknown context");
+ }
+
+ return classloaders.get(context, k -> {
+ LOG.debug("Creating URLClassLoader for context, uris: {}", context);
+ return new URLClassLoader(Arrays.stream(context.split(",")).map(url -> {
+ try {
+ return new URL(url);
+ } catch (MalformedURLException e) {
+ throw new RuntimeException(e);
+ }
+ }).toArray(URL[]::new), ClassLoader.getSystemClassLoader());
+ });
+ }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
index 25e6ab3..5b82d0e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
@@ -51,11 +51,11 @@
* </pre>
*
* <p>
- * If migrating code from Connector to AccumuloClient an important difference to consider is that
- * AccumuloClient is closable and Connector is not. Connector uses static resources and therefore
- * creating them is cheap. AccumuloClient attempts to clean up resources on close, so constantly
- * creating them could perform worse than Connector. Therefore, it would be better to create an
- * AccumuloClient and pass it around.
+ * An important difference with the legacy Connector to consider is that Connector reused global
+ * static resources. AccumuloClient, however, attempts to clean up its resources on close. So,
+ * creating many AccumuloClient objects will perform worse than creating many Connectors did.
+ * Therefore, it is suggested to reuse AccumuloClient instances where possible, rather than create
+ * many of them.
*
* <p>
* AccumuloClient objects are intended to be thread-safe, and can be used by multiple threads.
@@ -292,14 +292,6 @@
InstanceOperations instanceOperations();
/**
- * Retrieves a ReplicationOperations object to manage replication configuration.
- *
- * @return an object to modify replication configuration
- */
- @Deprecated(since = "2.1.0")
- org.apache.accumulo.core.client.admin.ReplicationOperations replicationOperations();
-
- /**
* @return All {@link Properties} used to create client except 'auth.token'
*/
Properties properties();
diff --git a/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java
index 68dbb5f..2bf7fcd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java
@@ -23,7 +23,7 @@
/**
* Send Mutations to a single Table in Accumulo.
* <p>
- * When the user uses a Connector to create a BatchWriter, they specify how much memory and how many
+ * When the user uses a client to create a BatchWriter, they specify how much memory and how many
* threads it should use. As the user adds mutations to the batch writer, it buffers them. Once the
* buffered mutations have used half of the user specified buffer, the mutations are dumped into the
* background to be written by a thread pool. If the user specified memory completely fills up, then
diff --git a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
index c570934..d3a8f7d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
@@ -122,7 +122,6 @@
} else {
// make small, positive values that truncate to 0 when converted use the minimum millis
// instead
-
this.maxLatency = Math.max(1, timeUnit.toMillis(maxLatency));
}
return this;
@@ -157,7 +156,6 @@
} else {
// make small, positive values that truncate to 0 when converted use the minimum millis
// instead
-
this.timeout = Math.max(1, timeUnit.toMillis(timeout));
}
return this;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
deleted file mode 100644
index 89bab19..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
+++ /dev/null
@@ -1,547 +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
- *
- * https://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;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.PropertyType;
-import org.apache.commons.configuration2.CompositeConfiguration;
-import org.apache.commons.configuration2.Configuration;
-import org.apache.commons.configuration2.MapConfiguration;
-import org.apache.commons.configuration2.PropertiesConfiguration;
-import org.apache.commons.configuration2.ex.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * Contains a list of property keys recognized by the Accumulo client and convenience methods for
- * setting them.
- *
- * @since 1.6.0
- * @deprecated since 2.0.0, replaced by {@link Accumulo#newClient()}
- */
-@Deprecated(since = "2.0.0")
-public class ClientConfiguration {
- private static final Logger log = LoggerFactory.getLogger(ClientConfiguration.class);
-
- public static final String USER_ACCUMULO_DIR_NAME = ".accumulo";
- public static final String USER_CONF_FILENAME = "config";
- public static final String GLOBAL_CONF_FILENAME = "client.conf";
-
- private final CompositeConfiguration compositeConfig;
-
- public enum ClientProperty {
- // SSL
- RPC_SSL_TRUSTSTORE_PATH(Property.RPC_SSL_TRUSTSTORE_PATH),
- RPC_SSL_TRUSTSTORE_PASSWORD(Property.RPC_SSL_TRUSTSTORE_PASSWORD),
- RPC_SSL_TRUSTSTORE_TYPE(Property.RPC_SSL_TRUSTSTORE_TYPE),
- RPC_SSL_KEYSTORE_PATH(Property.RPC_SSL_KEYSTORE_PATH),
- RPC_SSL_KEYSTORE_PASSWORD(Property.RPC_SSL_KEYSTORE_PASSWORD),
- RPC_SSL_KEYSTORE_TYPE(Property.RPC_SSL_KEYSTORE_TYPE),
- RPC_USE_JSSE(Property.RPC_USE_JSSE),
- GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS),
- INSTANCE_RPC_SSL_CLIENT_AUTH(Property.INSTANCE_RPC_SSL_CLIENT_AUTH),
- INSTANCE_RPC_SSL_ENABLED(Property.INSTANCE_RPC_SSL_ENABLED),
-
- // ZooKeeper
- INSTANCE_ZK_HOST(Property.INSTANCE_ZK_HOST),
- INSTANCE_ZK_TIMEOUT(Property.INSTANCE_ZK_TIMEOUT),
-
- // Instance information
- INSTANCE_NAME("instance.name", null, PropertyType.STRING,
- "Name of Accumulo instance to connect to"),
- INSTANCE_ID("instance.id", null, PropertyType.STRING,
- "UUID of Accumulo instance to connect to"),
-
- // Tracing
- @Deprecated(since = "2.1.0")
- TRACE_SPAN_RECEIVERS(Property.TRACE_SPAN_RECEIVERS),
- @Deprecated(since = "2.1.0")
- TRACE_SPAN_RECEIVER_PREFIX(Property.TRACE_SPAN_RECEIVER_PREFIX),
- @Deprecated(since = "2.1.0")
- TRACE_ZK_PATH(Property.TRACE_ZK_PATH),
-
- // SASL / GSSAPI(Kerberos)
- /**
- * @since 1.7.0
- */
- INSTANCE_RPC_SASL_ENABLED(Property.INSTANCE_RPC_SASL_ENABLED),
- /**
- * @since 1.7.0
- */
- RPC_SASL_QOP(Property.RPC_SASL_QOP),
- /**
- * @since 1.7.0
- */
- KERBEROS_SERVER_PRIMARY("kerberos.server.primary", "accumulo", PropertyType.STRING,
- "The first component of the Kerberos principal, the 'primary', "
- + "that Accumulo servers use to login");
-
- private String key;
- private String defaultValue;
- private PropertyType type;
- private String description;
-
- private ClientProperty(Property prop) {
- this(prop.getKey(), prop.getDefaultValue(), prop.getType(), prop.getDescription());
- }
-
- private ClientProperty(String key, String defaultValue, PropertyType type, String description) {
- this.key = key;
- this.defaultValue = defaultValue;
- this.type = type;
- this.description = description;
- }
-
- public String getKey() {
- return key;
- }
-
- public String getDefaultValue() {
- return defaultValue;
- }
-
- private PropertyType getType() {
- return type;
- }
-
- public String getDescription() {
- return description;
- }
-
- public static ClientProperty getPropertyByKey(String key) {
- for (ClientProperty prop : ClientProperty.values()) {
- if (prop.getKey().equals(key)) {
- return prop;
- }
- }
- return null;
- }
- }
-
- private ClientConfiguration(List<? extends Configuration> configs) {
- compositeConfig = new CompositeConfiguration(configs);
- }
-
- /**
- * Attempts to load a configuration file from the system using the default search paths. Uses the
- * <em>ACCUMULO_CLIENT_CONF_PATH</em> environment variable, split on <em>File.pathSeparator</em>,
- * for a list of target files.
- * <p>
- * If <em>ACCUMULO_CLIENT_CONF_PATH</em> is not set, uses the following in this order:
- * <ul>
- * <li>~/.accumulo/config
- * <li><em>$ACCUMULO_CONF_DIR</em>/client.conf, if <em>$ACCUMULO_CONF_DIR</em> is defined.
- * <li>/etc/accumulo/client.conf
- * <li>/etc/accumulo/conf/client.conf
- * </ul>
- * <p>
- * A client configuration will then be read from each location using
- * <em>PropertiesConfiguration</em> to construct a configuration. That means the latest item will
- * be the one in the configuration.
- *
- * @see PropertiesConfiguration
- * @see File#pathSeparator
- */
- public static ClientConfiguration loadDefault() {
- return loadFromSearchPath(getDefaultSearchPath());
- }
-
- /**
- * Initializes an empty configuration object to be further configured with other methods on the
- * class.
- *
- * @since 1.9.0
- */
- public static ClientConfiguration create() {
- return new ClientConfiguration(Collections.emptyList());
- }
-
- /**
- * Initializes a configuration object from the contents of a configuration file. Currently
- * supports Java "properties" files. The returned object can be further configured with subsequent
- * calls to other methods on this class.
- *
- * @param file the path to the configuration file
- * @since 1.9.0
- */
- public static ClientConfiguration fromFile(File file) {
- var config = new PropertiesConfiguration();
- try (var reader = new FileReader(file, UTF_8)) {
- config.read(reader);
- } catch (ConfigurationException | IOException e) {
- throw new IllegalArgumentException("Bad configuration file: " + file, e);
- }
- return new ClientConfiguration(Collections.singletonList(config));
- }
-
- /**
- * Initializes a configuration object from the contents of a map. The returned object can be
- * further configured with subsequent calls to other methods on this class.
- *
- * @param properties a map containing the configuration properties to use
- * @since 1.9.0
- */
- public static ClientConfiguration fromMap(Map<String,String> properties) {
- MapConfiguration mapConf = new MapConfiguration(properties);
- return new ClientConfiguration(Collections.singletonList(mapConf));
- }
-
- @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
- justification = "process runs in same security context as user who provided path")
- private static ClientConfiguration loadFromSearchPath(List<String> paths) {
- List<Configuration> configs = new LinkedList<>();
- for (String path : paths) {
- File conf = new File(path);
- if (conf.isFile() && conf.canRead()) {
- var config = new PropertiesConfiguration();
- try (var reader = new FileReader(conf, UTF_8)) {
- config.read(reader);
- } catch (ConfigurationException | IOException e) {
- throw new IllegalStateException("Error loading client configuration file " + conf, e);
- }
- configs.add(config);
- log.info("Loaded client configuration file {}", conf);
- }
- }
- // We couldn't find the client configuration anywhere
- if (configs.isEmpty()) {
- log.debug(
- "Found no client.conf in default paths. Using default client configuration values.");
- }
- return new ClientConfiguration(configs);
- }
-
- public static ClientConfiguration deserialize(String serializedConfig) {
- var propConfig = new PropertiesConfiguration();
- try {
- propConfig.read(new StringReader(serializedConfig));
- } catch (ConfigurationException | IOException e) {
- throw new IllegalArgumentException(
- "Error deserializing client configuration: " + serializedConfig, e);
- }
- return new ClientConfiguration(Collections.singletonList(propConfig));
- }
-
- /**
- * Muck the value of {@code clientConfPath} if it points to a directory by appending
- * {@code client.conf} to the end of the file path. This is a no-op if the value is not a
- * directory on the filesystem.
- *
- * @param clientConfPath The value of ACCUMULO_CLIENT_CONF_PATH.
- */
- @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
- justification = "process runs in same security context as user who provided path")
- static String getClientConfPath(String clientConfPath) {
- if (clientConfPath == null) {
- return null;
- }
- File filePath = new File(clientConfPath);
- // If clientConfPath is a directory, tack on the default client.conf file name.
- if (filePath.exists() && filePath.isDirectory()) {
- return new File(filePath, "client.conf").toString();
- }
- return clientConfPath;
- }
-
- private static List<String> getDefaultSearchPath() {
- String clientConfSearchPath = getClientConfPath(System.getenv("ACCUMULO_CLIENT_CONF_PATH"));
- List<String> clientConfPaths;
- if (clientConfSearchPath != null) {
- clientConfPaths = Arrays.asList(clientConfSearchPath.split(File.pathSeparator));
- } else {
- // if $ACCUMULO_CLIENT_CONF_PATH env isn't set, priority from top to bottom is:
- // ~/.accumulo/config
- // $ACCUMULO_CONF_DIR/client.conf
- // /etc/accumulo/client.conf
- // /etc/accumulo/conf/client.conf
- clientConfPaths = new LinkedList<>();
- clientConfPaths.add(System.getProperty("user.home") + File.separator + USER_ACCUMULO_DIR_NAME
- + File.separator + USER_CONF_FILENAME);
- if (System.getenv("ACCUMULO_CONF_DIR") != null) {
- clientConfPaths
- .add(System.getenv("ACCUMULO_CONF_DIR") + File.separator + GLOBAL_CONF_FILENAME);
- }
- clientConfPaths.add("/etc/accumulo/" + GLOBAL_CONF_FILENAME);
- clientConfPaths.add("/etc/accumulo/conf/" + GLOBAL_CONF_FILENAME);
- }
- return clientConfPaths;
- }
-
- public String serialize() {
- var propConfig = new PropertiesConfiguration();
- propConfig.copy(compositeConfig);
- StringWriter writer = new StringWriter();
- try {
- propConfig.write(writer);
- } catch (ConfigurationException | IOException e) {
- // this should never happen
- throw new IllegalStateException(e);
- }
- return writer.toString();
- }
-
- /**
- * Returns the value for prop, the default value if not present.
- *
- */
- public String get(ClientProperty prop) {
- if (compositeConfig.containsKey(prop.getKey())) {
- return compositeConfig.getString(prop.getKey());
- } else {
- return prop.getDefaultValue();
- }
- }
-
- private void checkType(ClientProperty property, PropertyType type) {
- if (!property.getType().equals(type)) {
- String msg = "Configuration method intended for type " + type + " called with a "
- + property.getType() + " argument (" + property.getKey() + ")";
- throw new IllegalArgumentException(msg);
- }
- }
-
- /**
- * Gets all properties under the given prefix in this configuration.
- *
- * @param property prefix property, must be of type PropertyType.PREFIX
- * @return a map of property keys to values
- * @throws IllegalArgumentException if property is not a prefix
- */
- public Map<String,String> getAllPropertiesWithPrefix(ClientProperty property) {
- checkType(property, PropertyType.PREFIX);
-
- Map<String,String> propMap = new HashMap<>();
- String prefix = property.getKey();
- if (prefix.endsWith(".")) {
- prefix = prefix.substring(0, prefix.length() - 1);
- }
- Iterator<?> iter = compositeConfig.getKeys(prefix);
- while (iter.hasNext()) {
- String p = (String) iter.next();
- propMap.put(p, compositeConfig.getString(p));
- }
- return propMap;
- }
-
- /**
- * Sets the value of property to value
- *
- */
- public void setProperty(ClientProperty prop, String value) {
- with(prop, value);
- }
-
- /**
- * Same as {@link #setProperty(ClientProperty, String)} but returns the ClientConfiguration for
- * chaining purposes
- */
- public ClientConfiguration with(ClientProperty prop, String value) {
- return with(prop.getKey(), value);
- }
-
- /**
- * Sets the value of property to value
- *
- * @since 1.9.0
- */
- public void setProperty(String prop, String value) {
- with(prop, value);
- }
-
- /**
- * Same as {@link #setProperty(String, String)} but returns the ClientConfiguration for chaining
- * purposes
- *
- * @since 1.9.0
- */
- public ClientConfiguration with(String prop, String value) {
- compositeConfig.setProperty(prop, value);
- return this;
- }
-
- /**
- * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_NAME
- *
- */
- public ClientConfiguration withInstance(String instanceName) {
- checkArgument(instanceName != null, "instanceName is null");
- return with(ClientProperty.INSTANCE_NAME, instanceName);
- }
-
- /**
- * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ID
- *
- */
- public ClientConfiguration withInstance(UUID instanceId) {
- checkArgument(instanceId != null, "instanceId is null");
- return with(ClientProperty.INSTANCE_ID, instanceId.toString());
- }
-
- /**
- * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ZK_HOST
- *
- */
- public ClientConfiguration withZkHosts(String zooKeepers) {
- checkArgument(zooKeepers != null, "zooKeepers is null");
- return with(ClientProperty.INSTANCE_ZK_HOST, zooKeepers);
- }
-
- /**
- * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ZK_TIMEOUT
- *
- */
- public ClientConfiguration withZkTimeout(int timeout) {
- return with(ClientProperty.INSTANCE_ZK_TIMEOUT, String.valueOf(timeout));
- }
-
- /**
- * Same as {@link #withSsl(boolean, boolean)} with useJsseConfig set to false
- *
- */
- public ClientConfiguration withSsl(boolean sslEnabled) {
- return withSsl(sslEnabled, false);
- }
-
- /**
- * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SSL_ENABLED and
- * ClientProperty.RPC_USE_JSSE
- *
- */
- public ClientConfiguration withSsl(boolean sslEnabled, boolean useJsseConfig) {
- return with(ClientProperty.INSTANCE_RPC_SSL_ENABLED, String.valueOf(sslEnabled))
- .with(ClientProperty.RPC_USE_JSSE, String.valueOf(useJsseConfig));
- }
-
- /**
- * Same as {@link #withTruststore(String, String, String)} with password null and type null
- *
- */
- public ClientConfiguration withTruststore(String path) {
- return withTruststore(path, null, null);
- }
-
- /**
- * Same as {@link #with(ClientProperty, String)} for ClientProperty.RPC_SSL_TRUSTORE_PATH,
- * ClientProperty.RPC_SSL_TRUSTORE_PASSWORD, and ClientProperty.RPC_SSL_TRUSTORE_TYPE
- *
- */
- public ClientConfiguration withTruststore(String path, String password, String type) {
- checkArgument(path != null, "path is null");
- setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PATH, path);
- if (password != null) {
- setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD, password);
- }
- if (type != null) {
- setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_TYPE, type);
- }
- return this;
- }
-
- /**
- * Same as {@link #withKeystore(String, String, String)} with password null and type null
- *
- */
- public ClientConfiguration withKeystore(String path) {
- return withKeystore(path, null, null);
- }
-
- /**
- * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
- * ClientProperty.RPC_SSL_KEYSTORE_PATH, ClientProperty.RPC_SSL_KEYSTORE_PASSWORD, and
- * ClientProperty.RPC_SSL_KEYSTORE_TYPE
- *
- */
- public ClientConfiguration withKeystore(String path, String password, String type) {
- checkArgument(path != null, "path is null");
- setProperty(ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH, "true");
- setProperty(ClientProperty.RPC_SSL_KEYSTORE_PATH, path);
- if (password != null) {
- setProperty(ClientProperty.RPC_SSL_KEYSTORE_PASSWORD, password);
- }
- if (type != null) {
- setProperty(ClientProperty.RPC_SSL_KEYSTORE_TYPE, type);
- }
- return this;
- }
-
- /**
- * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SASL_ENABLED.
- *
- * @since 1.7.0
- */
- public ClientConfiguration withSasl(boolean saslEnabled) {
- return with(ClientProperty.INSTANCE_RPC_SASL_ENABLED, String.valueOf(saslEnabled));
- }
-
- /**
- * Show whether SASL has been set on this configuration.
- *
- * @since 1.9.0
- */
- public boolean hasSasl() {
- return compositeConfig.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(),
- Boolean.parseBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getDefaultValue()));
- }
-
- /**
- * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SASL_ENABLED and
- * ClientProperty.GENERAL_KERBEROS_PRINCIPAL.
- *
- * @param saslEnabled Should SASL(kerberos) be enabled
- * @param kerberosServerPrimary The 'primary' component of the Kerberos principal Accumulo servers
- * use to login (e.g. 'accumulo' in 'accumulo/_HOST@REALM')
- * @since 1.7.0
- */
- public ClientConfiguration withSasl(boolean saslEnabled, String kerberosServerPrimary) {
- return withSasl(saslEnabled).with(ClientProperty.KERBEROS_SERVER_PRIMARY,
- kerberosServerPrimary);
- }
-
- public boolean containsKey(String key) {
- return compositeConfig.containsKey(key);
- }
-
- public Iterator<String> getKeys() {
- return compositeConfig.getKeys();
- }
-
- public String getString(String key) {
- return compositeConfig.getString(key);
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
index 362c4e8..7e300e9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
@@ -22,6 +22,7 @@
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.Collection;
import java.util.Iterator;
import java.util.Map;
@@ -52,7 +53,6 @@
import org.apache.accumulo.core.iteratorsImpl.IteratorBuilder;
import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil;
import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.spi.common.ServiceEnvironment;
import org.apache.hadoop.io.Text;
/**
@@ -134,12 +134,6 @@
return samplerConfig;
}
- @Deprecated(since = "2.1.0")
- @Override
- public ServiceEnvironment getServiceEnv() {
- return new ClientServiceEnvironmentImpl(context.get());
- }
-
@Override
public PluginEnvironment getPluginEnv() {
return new ClientServiceEnvironmentImpl(context.get());
@@ -303,7 +297,7 @@
skvi = IteratorConfigUtil.loadIterators(smi, ib);
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
final Set<ByteSequence> colfs = new TreeSet<>();
@@ -314,7 +308,7 @@
try {
skvi.seek(range, colfs, true);
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
return new IteratorAdapter(skvi);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
deleted file mode 100644
index 990824d..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ /dev/null
@@ -1,283 +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
- *
- * https://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;
-
-import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.client.admin.NamespaceOperations;
-import org.apache.accumulo.core.client.admin.ReplicationOperations;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ConnectorImpl;
-import org.apache.accumulo.core.security.Authorizations;
-
-/**
- * Connector connects to an Accumulo instance and allows the user to request readers and writers for
- * the instance as well as various objects that permit administrative operations.
- *
- * The Connector enforces security on the client side by forcing all API calls to be accompanied by
- * user credentials.
- *
- * @deprecated since 2.0.0. Use {@link AccumuloClient} for writing new code. Connector is available
- * for existing code. Use {@link #from(AccumuloClient)} as a bridge between the two.
- */
-@Deprecated(since = "2.0.0")
-public abstract class Connector {
-
- /**
- * Factory method to create a BatchScanner connected to Accumulo.
- *
- * @param tableName the name of the table to query
- * @param authorizations A set of authorization labels that will be checked against the column
- * visibility of each key in order to filter data. The authorizations passed in must be a
- * subset of the accumulo user's set of authorizations. If the accumulo user has
- * authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
- * be thrown.
- * @param numQueryThreads the number of concurrent threads to spawn for querying
- *
- * @return BatchScanner object for configuring and querying
- * @throws TableNotFoundException when the specified table doesn't exist
- */
- public abstract BatchScanner createBatchScanner(String tableName, Authorizations authorizations,
- int numQueryThreads) throws TableNotFoundException;
-
- /**
- * Factory method to create a BatchDeleter connected to Accumulo.
- *
- * @param tableName the name of the table to query and delete from
- * @param authorizations A set of authorization labels that will be checked against the column
- * visibility of each key in order to filter data. The authorizations passed in must be a
- * subset of the accumulo user's set of authorizations. If the accumulo user has
- * authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
- * be thrown.
- * @param numQueryThreads the number of concurrent threads to spawn for querying
- * @param maxMemory size in bytes of the maximum memory to batch before writing
- * @param maxLatency size in milliseconds; set to 0 or Long.MAX_VALUE to allow the maximum time to
- * hold a batch before writing
- * @param maxWriteThreads the maximum number of threads to use for writing data to the tablet
- * servers
- *
- * @return BatchDeleter object for configuring and deleting
- * @throws TableNotFoundException when the specified table doesn't exist
- * @deprecated since 1.5.0; Use
- * {@link #createBatchDeleter(String, Authorizations, int, BatchWriterConfig)}
- * instead.
- */
- @Deprecated(since = "1.5.0")
- public abstract BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
- int numQueryThreads, long maxMemory, long maxLatency, int maxWriteThreads)
- throws TableNotFoundException;
-
- /**
- * Factory method to create BatchDeleter
- *
- * @param tableName the name of the table to query and delete from
- * @param authorizations A set of authorization labels that will be checked against the column
- * visibility of each key in order to filter data. The authorizations passed in must be a
- * subset of the accumulo user's set of authorizations. If the accumulo user has
- * authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
- * be thrown.
- * @param numQueryThreads the number of concurrent threads to spawn for querying
- * @param config configuration used to create batch writer. This config takes precedence. Any
- * unset values will be merged with config set when the Connector was created. If no config
- * was set during Connector creation, BatchWriterConfig defaults will be used.
- * @return BatchDeleter object for configuring and deleting
- * @since 1.5.0
- */
- public abstract BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
- int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException;
-
- /**
- * Factory method to create a BatchWriter connected to Accumulo.
- *
- * @param tableName the name of the table to insert data into
- * @param maxMemory size in bytes of the maximum memory to batch before writing
- * @param maxLatency time in milliseconds; set to 0 or Long.MAX_VALUE to allow the maximum time to
- * hold a batch before writing
- * @param maxWriteThreads the maximum number of threads to use for writing data to the tablet
- * servers
- *
- * @return BatchWriter object for configuring and writing data to
- * @throws TableNotFoundException when the specified table doesn't exist
- * @deprecated since 1.5.0; Use {@link #createBatchWriter(String, BatchWriterConfig)} instead.
- */
- @Deprecated(since = "1.5.0")
- public abstract BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency,
- int maxWriteThreads) throws TableNotFoundException;
-
- /**
- * Factory method to create a BatchWriter connected to Accumulo.
- *
- * @param tableName the name of the table to insert data into
- * @param config configuration used to create batch writer. This config will take precedence. Any
- * unset values will merged with config set when the Connector was created. If no config
- * was set during Connector creation, BatchWriterConfig defaults will be used.
- * @return BatchWriter object for configuring and writing data to
- * @since 1.5.0
- */
- public abstract BatchWriter createBatchWriter(String tableName, BatchWriterConfig config)
- throws TableNotFoundException;
-
- /**
- * Factory method to create a Multi-Table BatchWriter connected to Accumulo. Multi-table batch
- * writers can queue data for multiple tables, which is good for ingesting data into multiple
- * tables from the same source
- *
- * @param maxMemory size in bytes of the maximum memory to batch before writing
- * @param maxLatency size in milliseconds; set to 0 or Long.MAX_VALUE to allow the maximum time to
- * hold a batch before writing
- * @param maxWriteThreads the maximum number of threads to use for writing data to the tablet
- * servers
- *
- * @return MultiTableBatchWriter object for configuring and writing data to
- * @deprecated since 1.5.0; Use {@link #createMultiTableBatchWriter(BatchWriterConfig)} instead.
- */
- @Deprecated(since = "1.5.0")
- public abstract MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency,
- int maxWriteThreads);
-
- /**
- * Factory method to create a Multi-Table BatchWriter connected to Accumulo. Multi-table batch
- * writers can queue data for multiple tables. Also data for multiple tables can be sent to a
- * server in a single batch. Its an efficient way to ingest data into multiple tables from a
- * single process.
- *
- * @param config configuration used to create multi-table batch writer. This config will take
- * precedence. Any unset values will merged with config set when the Connector was created.
- * If no config was set during Connector creation, BatchWriterConfig defaults will be used.
- * @return MultiTableBatchWriter object for configuring and writing data to
- * @since 1.5.0
- */
- public abstract MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config);
-
- /**
- * Factory method to create a Scanner connected to Accumulo.
- *
- * @param tableName the name of the table to query data from
- * @param authorizations A set of authorization labels that will be checked against the column
- * visibility of each key in order to filter data. The authorizations passed in must be a
- * subset of the accumulo user's set of authorizations. If the accumulo user has
- * authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
- * be thrown.
- *
- * @return Scanner object for configuring and querying data with
- * @throws TableNotFoundException when the specified table doesn't exist
- */
- public abstract Scanner createScanner(String tableName, Authorizations authorizations)
- throws TableNotFoundException;
-
- /**
- * Factory method to create a ConditionalWriter connected to Accumulo.
- *
- * @param tableName the name of the table to query data from
- * @param config configuration used to create conditional writer
- *
- * @return ConditionalWriter object for writing ConditionalMutations
- * @throws TableNotFoundException when the specified table doesn't exist
- * @since 1.6.0
- */
- public abstract ConditionalWriter createConditionalWriter(String tableName,
- ConditionalWriterConfig config) throws TableNotFoundException;
-
- /**
- * Accessor method for internal instance object.
- *
- * @return the internal instance object
- */
- public abstract Instance getInstance();
-
- /**
- * Get the current user for this connector
- *
- * @return the user name
- */
- public abstract String whoami();
-
- /**
- * Retrieves a TableOperations object to perform table functions, such as create and delete.
- *
- * @return an object to manipulate tables
- */
- public abstract TableOperations tableOperations();
-
- /**
- * Retrieves a NamespaceOperations object to perform namespace functions, such as create and
- * delete.
- *
- * @return an object to manipulate namespaces
- */
- public abstract NamespaceOperations namespaceOperations();
-
- /**
- * Retrieves a SecurityOperations object to perform user security operations, such as creating
- * users.
- *
- * @return an object to modify users and permissions
- */
- public abstract SecurityOperations securityOperations();
-
- /**
- * Retrieves an InstanceOperations object to modify instance configuration.
- *
- * @return an object to modify instance configuration
- */
- public abstract InstanceOperations instanceOperations();
-
- /**
- * Retrieves a ReplicationOperations object to manage replication configuration.
- *
- * @return an object to modify replication configuration
- * @since 1.7.0
- */
- public abstract ReplicationOperations replicationOperations();
-
- /**
- * Creates a Connector from an AccumuloClient. This Connector will no longer work after the
- * AccumuloClient is closed. Also anything derived from the Connector (like a Scanner for example)
- * is unlikely to work after the AccumuloClient is closed.
- *
- * @since 2.0
- */
- public static Connector from(AccumuloClient client)
- throws AccumuloSecurityException, AccumuloException {
- return new ConnectorImpl((ClientContext) client);
- }
-
- /**
- * Creates a new Accumulo Client from a Connector. The returned client should be closed and
- * closing it will not affect the Connector from which it was derived. This method is useful for
- * cases where code written using Connector must call code written using AccumuloClient. Below is
- * an example.
- *
- * <pre>
- * <code>
- * Connector conn = getMyConnector();
- * try(AccumuloClient client = Connector.newClient(conn) {
- * doSomething(client);
- * }
- * </code>
- * </pre>
- *
- * @since 2.1.0
- */
- public static AccumuloClient newClient(Connector conn) {
- return Accumulo.newClient().from(((ConnectorImpl) conn).getAccumuloClient().getProperties())
- .build();
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
deleted file mode 100644
index 947f8b1..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ /dev/null
@@ -1,139 +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
- *
- * https://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;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-
-/**
- * This class represents the information a client needs to know to connect to an instance of
- * accumulo.
- *
- * @deprecated since 2.0.0, use {@link Accumulo#newClient()} and {@link java.util.Properties}
- * instead
- */
-@Deprecated(since = "2.0.0")
-public interface Instance {
- /**
- * Returns the location of the tablet server that is serving the root tablet.
- *
- * @return location in "hostname:port" form
- */
- String getRootTabletLocation();
-
- /**
- * Returns the location(s) of the accumulo manager and any redundant servers.
- *
- * @return a list of locations in "hostname:port" form
- */
- List<String> getMasterLocations();
-
- /**
- * Returns a unique string that identifies this instance of accumulo.
- *
- * @return a UUID
- */
- String getInstanceID();
-
- /**
- * Returns the instance name given at system initialization time.
- *
- * @return current instance name
- */
- String getInstanceName();
-
- /**
- * Returns a comma-separated list of zookeeper servers the instance is using.
- *
- * @return the zookeeper servers this instance is using in "hostname:port" form
- */
- String getZooKeepers();
-
- /**
- * Returns the zookeeper connection timeout.
- *
- * @return the configured timeout to connect to zookeeper
- */
- int getZooKeepersSessionTimeOut();
-
- /**
- * Returns a connection to accumulo.
- *
- * @param user a valid accumulo user
- * @param pass A UTF-8 encoded password. The password may be cleared after making this call.
- * @return the accumulo Connector
- * @throws AccumuloException when a generic exception occurs
- * @throws AccumuloSecurityException when a user's credentials are invalid
- * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with
- * {@link PasswordToken}
- */
- @Deprecated(since = "1.5.0")
- default Connector getConnector(String user, byte[] pass)
- throws AccumuloException, AccumuloSecurityException {
- return getConnector(user, new PasswordToken(pass));
- }
-
- /**
- * Returns a connection to accumulo.
- *
- * @param user a valid accumulo user
- * @param pass A UTF-8 encoded password. The password may be cleared after making this call.
- * @return the accumulo Connector
- * @throws AccumuloException when a generic exception occurs
- * @throws AccumuloSecurityException when a user's credentials are invalid
- * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with
- * {@link PasswordToken}
- */
- @Deprecated(since = "1.5.0")
- default Connector getConnector(String user, ByteBuffer pass)
- throws AccumuloException, AccumuloSecurityException {
- return getConnector(user, new PasswordToken(pass));
- }
-
- /**
- * Returns a connection to this instance of accumulo.
- *
- * @param user a valid accumulo user
- * @param pass If a mutable CharSequence is passed in, it may be cleared after this call.
- * @return the accumulo Connector
- * @throws AccumuloException when a generic exception occurs
- * @throws AccumuloSecurityException when a user's credentials are invalid
- * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with
- * {@link PasswordToken}
- */
- @Deprecated(since = "1.5.0")
- default Connector getConnector(String user, CharSequence pass)
- throws AccumuloException, AccumuloSecurityException {
- return getConnector(user, new PasswordToken(pass));
- }
-
- /**
- * Returns a connection to this instance of accumulo.
- *
- * @param principal a valid accumulo user
- * @param token Use the token type configured for the Accumulo instance you are connecting to. An
- * Accumulo instance with default configurations will use {@link PasswordToken}
- * @since 1.5.0
- */
- Connector getConnector(String principal, AuthenticationToken token)
- throws AccumuloException, AccumuloSecurityException;
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
index 518737b..c3ead89 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
@@ -18,8 +18,8 @@
*/
package org.apache.accumulo.core.client;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
import java.util.ArrayList;
import java.util.Iterator;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
index 30e6fbd..1603268 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
@@ -45,34 +45,6 @@
private final int unknownErrors;
/**
- *
- * @param cvsList list of constraint violations
- * @param hashMap authorization failures
- * @param serverSideErrors server side errors
- * @param unknownErrors number of unknown errors
- *
- * @since 1.7.0
- * @deprecated since 2.0.0, replaced by
- * {@link #MutationsRejectedException(AccumuloClient, List, Map, Collection, int, Throwable)}
- */
- @Deprecated(since = "2.0.0")
- public MutationsRejectedException(Instance instance, List<ConstraintViolationSummary> cvsList,
- Map<TabletId,Set<SecurityErrorCode>> hashMap, Collection<String> serverSideErrors,
- int unknownErrors, Throwable cause) {
- super(
- "constraint violation codes : "
- + cvsList.stream().map(ConstraintViolationSummary::getViolationCode)
- .collect(Collectors.toSet())
- + " security codes: " + hashMap.toString() + " # server errors "
- + serverSideErrors.size() + " # exceptions " + unknownErrors,
- cause);
- this.cvsl.addAll(cvsList);
- this.af.putAll(hashMap);
- this.es.addAll(serverSideErrors);
- this.unknownErrors = unknownErrors;
- }
-
- /**
* Creates Mutations rejected exception
*
* @param client AccumuloClient
diff --git a/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java b/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
index 6653c80..4372a15 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
@@ -163,7 +163,7 @@
* @param className Fully qualified name of the class.
* @param base The expected super type of the class.
*/
- <T> T instantiate(String className, Class<T> base) throws Exception;
+ <T> T instantiate(String className, Class<T> base) throws ReflectiveOperationException;
/**
* Instantiate a class using Accumulo's per table classloader. The class must have a no argument
@@ -172,5 +172,6 @@
* @param className Fully qualified name of the class.
* @param base The expected super type of the class.
*/
- <T> T instantiate(TableId tableId, String className, Class<T> base) throws Exception;
+ <T> T instantiate(TableId tableId, String className, Class<T> base)
+ throws ReflectiveOperationException;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java b/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
index 2dbeb14..bd0e266 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
@@ -25,14 +25,6 @@
private static final long serialVersionUID = 1L;
/**
- * @deprecated since 2.0.0, replaced by {@link #TableOfflineException(String)}
- */
- @Deprecated(since = "2.0.0")
- public TableOfflineException(Instance instance, String tableId) {
- super("Table with ID (" + tableId + ") is offline");
- }
-
- /**
* @since 2.0.0
*/
public TableOfflineException(String msg) {
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
deleted file mode 100644
index aa96404..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ /dev/null
@@ -1,203 +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
- *
- * https://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;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.util.concurrent.TimeUnit.SECONDS;
-
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.clientImpl.ClientConfConverter;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.clientImpl.ClientInfoImpl;
-import org.apache.accumulo.core.clientImpl.InstanceOperationsImpl;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ZooCache;
-import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
-import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
-import org.apache.accumulo.core.singletons.SingletonReservation;
-import org.apache.accumulo.core.util.OpTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An implementation of instance that looks in zookeeper to find information needed to connect to an
- * instance of accumulo.
- *
- * <p>
- * The advantage of using zookeeper to obtain information about accumulo is that zookeeper is highly
- * available, very responsive, and supports caching.
- *
- * <p>
- * Because it is possible for multiple instances of accumulo to share a single set of zookeeper
- * servers, all constructors require an accumulo instance name.
- *
- * If you do not know the instance names then run accumulo
- * org.apache.accumulo.server.util.ListInstances on an accumulo server.
- *
- * @deprecated since 2.0.0, Use {@link Accumulo#newClient()} instead
- */
-@Deprecated(since = "2.0.0")
-public class ZooKeeperInstance implements Instance {
-
- private static final Logger log = LoggerFactory.getLogger(ZooKeeperInstance.class);
-
- private String instanceId = null;
- private String instanceName = null;
-
- private final ZooCache zooCache;
-
- private final String zooKeepers;
-
- private final int zooKeepersSessionTimeOut;
-
- private ClientConfiguration clientConf;
-
- /**
- *
- * @param instanceName The name of specific accumulo instance. This is set at initialization time.
- * @param zooKeepers A comma separated list of zoo keeper server locations. Each location can
- * contain an optional port, of the format host:port.
- */
- public ZooKeeperInstance(String instanceName, String zooKeepers) {
- this(ClientConfiguration.loadDefault().withInstance(instanceName).withZkHosts(zooKeepers));
- }
-
- ZooKeeperInstance(ClientConfiguration config, ZooCacheFactory zcf) {
- checkArgument(config != null, "config is null");
- // Enable singletons before before getting a zoocache
- SingletonManager.setMode(Mode.CONNECTOR);
- this.clientConf = config;
- this.instanceId = clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_ID);
- this.instanceName = clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_NAME);
- if ((instanceId == null) == (instanceName == null)) {
- throw new IllegalArgumentException(
- "Expected exactly one of instanceName and instanceId to be set; "
- + (instanceName == null ? "neither" : "both") + " were set");
- }
- this.zooKeepers = clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST);
- this.zooKeepersSessionTimeOut = (int) ConfigurationTypeHelper
- .getTimeInMillis(clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT));
- zooCache = zcf.getZooCache(zooKeepers, zooKeepersSessionTimeOut);
- if (instanceName != null) {
- // Validates that the provided instanceName actually exists
- getInstanceID();
- }
- }
-
- /**
- * @param config Client configuration for specifying connection options. See
- * {@link ClientConfiguration} which extends Configuration with convenience methods
- * specific to Accumulo.
- * @since 1.9.0
- */
- public ZooKeeperInstance(ClientConfiguration config) {
- this(config, new ZooCacheFactory());
- }
-
- @Override
- public String getInstanceID() {
- if (instanceId == null) {
- instanceId = ClientContext.getInstanceID(zooCache, instanceName).canonical();
- }
- ClientContext.verifyInstanceId(zooCache, instanceId, instanceName);
- return instanceId;
- }
-
- @Override
- public List<String> getMasterLocations() {
- return ClientContext.getManagerLocations(zooCache, getInstanceID());
- }
-
- @Override
- public String getRootTabletLocation() {
- OpTimer timer = null;
-
- if (log.isTraceEnabled()) {
- log.trace("tid={} Looking up root tablet location in zookeeper.",
- Thread.currentThread().getId());
- timer = new OpTimer().start();
- }
-
- Location loc = TabletsMetadata
- .getRootMetadata(Constants.ZROOT + "/" + getInstanceID(), zooCache).getLocation();
-
- if (timer != null) {
- timer.stop();
- log.trace("tid={} Found root tablet at {} in {}", Thread.currentThread().getId(), loc,
- String.format("%.3f secs", timer.scale(SECONDS)));
- }
-
- if (loc == null || loc.getType() != LocationType.CURRENT) {
- return null;
- }
-
- return loc.getHostPort();
- }
-
- @Override
- public String getInstanceName() {
- if (instanceName == null) {
- instanceName =
- InstanceOperationsImpl.lookupInstanceName(zooCache, InstanceId.of(getInstanceID()));
- }
-
- return instanceName;
- }
-
- @Override
- public String getZooKeepers() {
- return zooKeepers;
- }
-
- @Override
- public int getZooKeepersSessionTimeOut() {
- return zooKeepersSessionTimeOut;
- }
-
- @Override
- public Connector getConnector(String principal, AuthenticationToken token)
- throws AccumuloException, AccumuloSecurityException {
- Properties properties = ClientConfConverter.toProperties(clientConf);
- properties.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), principal);
- properties.setProperty(ClientProperty.INSTANCE_NAME.getKey(), getInstanceName());
- ClientInfo info = new ClientInfoImpl(properties, token);
- AccumuloConfiguration serverConf = ClientConfConverter.toAccumuloConf(properties);
- return new org.apache.accumulo.core.clientImpl.ConnectorImpl(
- new ClientContext(SingletonReservation.noop(), info, serverConf, null));
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder(64);
- sb.append("ZooKeeperInstance: ").append(getInstanceName()).append(" ").append(getZooKeepers());
- return sb.toString();
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
index 45a31e9..dd02f6b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
@@ -29,17 +29,12 @@
*/
public abstract class ActiveCompaction {
- public static enum CompactionType {
+ public enum CompactionType {
/**
* compaction to flush a tablets memory
*/
MINOR,
/**
- * Accumulo no longer does merging minor compactions.
- */
- @Deprecated(since = "2.1.0", forRemoval = true)
- MERGE,
- /**
* compaction that merges a subset of a tablets files into one file
*/
MAJOR,
@@ -49,7 +44,7 @@
FULL
}
- public static enum CompactionReason {
+ public enum CompactionReason {
/**
* compaction initiated by user
*/
@@ -59,8 +54,10 @@
*/
SYSTEM,
/**
- * Compaction initiated by merge operation
+ * @deprecated Chop compactions no longer occur and it's not expected that listing compaction
+ * would ever return this.
*/
+ @Deprecated(since = "3.1")
CHOP,
/**
* idle compaction
@@ -125,6 +122,12 @@
public abstract long getEntriesWritten();
/**
+ * @return the number of times the server paused a compaction
+ * @since 3.0.0
+ */
+ public abstract long getPausedCount();
+
+ /**
* @return the per compaction iterators configured
*/
public abstract List<IteratorSetting> getIterators();
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java b/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java
index 64392a1..1600ea0 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java
@@ -19,7 +19,6 @@
package org.apache.accumulo.core.client.admin;
import static java.util.Objects.requireNonNull;
-import static org.apache.accumulo.core.clientImpl.CompactionStrategyConfigUtil.DEFAULT_STRATEGY;
import java.util.Collections;
import java.util.List;
@@ -29,7 +28,6 @@
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer;
import org.apache.accumulo.core.client.admin.compaction.CompactionSelector;
-import org.apache.accumulo.core.clientImpl.CompactionStrategyConfigUtil;
import org.apache.accumulo.core.clientImpl.UserCompactionUtils;
import org.apache.hadoop.io.Text;
@@ -40,7 +38,6 @@
*
* @since 1.7.0
*/
-@SuppressWarnings("removal")
public class CompactionConfig {
private Text start = null;
@@ -48,7 +45,6 @@
private boolean flush = true;
private boolean wait = true;
private List<IteratorSetting> iterators = Collections.emptyList();
- private CompactionStrategyConfig compactionStrategy = DEFAULT_STRATEGY;
private Map<String,String> hints = Map.of();
private PluginConfig selectorConfig = UserCompactionUtils.DEFAULT_SELECTOR;
private PluginConfig configurerConfig = UserCompactionUtils.DEFAULT_CONFIGURER;
@@ -144,35 +140,6 @@
}
/**
- * @param csConfig configures the strategy that will be used by each tablet to select files. If no
- * strategy is set, then all files will be compacted.
- * @return this
- * @deprecated since 2.1.0 use {@link #setSelector(PluginConfig)} and
- * {@link #setConfigurer(PluginConfig)} instead. See {@link CompactionStrategyConfig}
- * for details about why this was deprecated.
- */
- @Deprecated(since = "2.1.0", forRemoval = true)
- public CompactionConfig setCompactionStrategy(CompactionStrategyConfig csConfig) {
- requireNonNull(csConfig);
- Preconditions.checkArgument(!csConfig.getClassName().isBlank());
- Preconditions.checkState(
- selectorConfig.getClassName().isEmpty() && configurerConfig.getClassName().isEmpty());
- this.compactionStrategy = csConfig;
- return this;
- }
-
- /**
- * @return The previously set compaction strategy. Defaults to a configuration of
- * org.apache.accumulo.tserver.compaction.EverythingCompactionStrategy which always
- * compacts all files.
- * @deprecated since 2.1.0
- */
- @Deprecated(since = "2.1.0", forRemoval = true)
- public CompactionStrategyConfig getCompactionStrategy() {
- return compactionStrategy;
- }
-
- /**
* Configure a {@link CompactionSelector} plugin to run for this compaction. Specify the class
* name and options here.
*
@@ -180,7 +147,6 @@
* @since 2.1.0
*/
public CompactionConfig setSelector(PluginConfig selectorConfig) {
- Preconditions.checkState(compactionStrategy.getClassName().isEmpty());
Preconditions.checkArgument(!selectorConfig.getClassName().isBlank());
this.selectorConfig = requireNonNull(selectorConfig);
return this;
@@ -197,9 +163,6 @@
* @since 2.1.0
*/
public CompactionConfig setExecutionHints(Map<String,String> hints) {
- if (!hints.isEmpty()) {
- Preconditions.checkState(compactionStrategy.getClassName().isEmpty());
- }
this.hints = Map.copyOf(hints);
return this;
}
@@ -218,7 +181,6 @@
* @since 2.1.0
*/
public CompactionConfig setConfigurer(PluginConfig configurerConfig) {
- Preconditions.checkState(compactionStrategy.getClassName().isEmpty());
this.configurerConfig = configurerConfig;
return this;
}
@@ -251,8 +213,6 @@
prefix = append(sb, prefix, () -> !flush, "flush", flush);
prefix = append(sb, prefix, () -> !wait, "wait", wait);
prefix = append(sb, prefix, () -> !iterators.isEmpty(), "iterators", iterators);
- prefix = append(sb, prefix, () -> !CompactionStrategyConfigUtil.isDefault(compactionStrategy),
- "strategy", compactionStrategy);
prefix = append(sb, prefix, () -> !UserCompactionUtils.isDefault(selectorConfig), "selector",
selectorConfig);
prefix = append(sb, prefix, () -> !UserCompactionUtils.isDefault(configurerConfig),
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionStrategyConfig.java b/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionStrategyConfig.java
deleted file mode 100644
index 2f880a6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionStrategyConfig.java
+++ /dev/null
@@ -1,96 +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
- *
- * https://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.admin;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.Map;
-
-/**
- * Configuration object which describes how a Compaction is run. Configuration objects are dependent
- * upon the CompactionStrategy running insider the server. This class is used in conjunction with
- * {@link CompactionConfig}.
- *
- * @since 1.7.0
- * @deprecated since 2.1.0 CompactionStrategies were deprecated for multiple reasons. First, they do
- * not support the new compaction execution model. Second, they bind selection and
- * output file configuration into a single entity when users need to configure these
- * independently. Third, they use internal Accumulo types and ensuring their stability
- * requires manual effort that may never happen. Fourth, writing a correct compaction
- * strategy was exceedingly difficult as it required knowledge of internal tablet server
- * synchronization in order to avoid causing scans to hang. Fifth although measure were
- * taken to execute compaction strategies in the same manner as before, their execution
- * in the new model has subtle differences that may result in suboptimal compactions.
- * Please migrate to using {@link CompactionConfig#setSelector(PluginConfig)} and
- * {@link CompactionConfig#setConfigurer(PluginConfig)} as soon as possible.
- */
-@Deprecated(since = "2.1.0", forRemoval = true)
-public class CompactionStrategyConfig {
- private String className;
- private Map<String,String> options = Map.of();
-
- /**
- * @param className The name of a class that implements
- * org.apache.accumulo.tserver.compaction.CompactionStrategy. This class must exist on
- * tservers.
- */
- public CompactionStrategyConfig(String className) {
- this.className = requireNonNull(className);
- }
-
- /**
- * @return the class name passed to the constructor.
- */
- public String getClassName() {
- return className;
- }
-
- /**
- * @param opts The options that will be passed to the init() method of the compaction strategy
- * when its instantiated on a tserver. This method will copy the map. The default is an
- * empty map.
- * @return this
- */
- public CompactionStrategyConfig setOptions(Map<String,String> opts) {
- this.options = Map.copyOf(opts);
- return this;
- }
-
- /**
- * @return The previously set options. Returns an unmodifiable map. The default is an empty map.
- */
- public Map<String,String> getOptions() {
- return options;
- }
-
- @Override
- public int hashCode() {
- return className.hashCode() + options.hashCode();
- }
-
- @Override
- public boolean equals(Object o) {
- if (o instanceof CompactionStrategyConfig) {
- CompactionStrategyConfig ocsc = (CompactionStrategyConfig) o;
- return className.equals(ocsc.className) && options.equals(ocsc.options);
- }
-
- return false;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java b/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java
index 2caba85..a953d7d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java
@@ -67,7 +67,7 @@
if (ba.length == startOS.size()) {
if (ba[0] != 0) {
- throw new RuntimeException();
+ throw new IllegalStateException();
}
// big int added a zero so it would not be negative, drop it
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
index 2e1c0c9..4cb3f6d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
@@ -87,22 +87,22 @@
* {@code
* AccumuloClient client = getClient();
* Map<String,String> acceptedProps = client.instanceOperations().modifyProperties(currProps -> {
- * var planner = currProps.get("tserver.compaction.major.service.default.planner");
+ * var planner = currProps.get("compaction.service.default.planner");
* //This code will only change the compaction planner if its currently set to default settings.
* //The endsWith() function was used to make the example short, would be better to use equals().
* if(planner != null && planner.endsWith("DefaultCompactionPlanner") {
* // tservers will eventually see these compaction planner changes and when they do they will see all of the changes at once
* currProps.keySet().removeIf(
- * prop -> prop.startsWith("tserver.compaction.major.service.default.planner.opts."));
- * currProps.put("tserver.compaction.major.service.default.planner","MyPlannerClassName");
- * currProps.put("tserver.compaction.major.service.default.planner.opts.myOpt1","val1");
- * currProps.put("tserver.compaction.major.service.default.planner.opts.myOpt2","val2");
+ * prop -> prop.startsWith("compaction.service.default.planner.opts."));
+ * currProps.put("compaction.service.default.planner","MyPlannerClassName");
+ * currProps.put("compaction.service.default.planner.opts.myOpt1","val1");
+ * currProps.put("compaction.service.default.planner.opts.myOpt2","val2");
* }
* });
*
* // Since three properties were set may want to check for the values of all
* // three, just checking one in this example to keep it short.
- * if("MyPlannerClassName".equals(acceptedProps.get("tserver.compaction.major.service.default.planner"))){
+ * if("MyPlannerClassName".equals(acceptedProps.get("compaction.service.default.planner"))){
* // the compaction planner change was accepted or already existed, so take action for that outcome
* } else {
* // the compaction planner change was not done, so take action for that outcome
@@ -140,7 +140,17 @@
void removeProperty(final String property) throws AccumuloException, AccumuloSecurityException;
/**
- * Retrieve the system-wide configuration.
+ * Retrieve the system-wide, merged view of the system configuration. Accumulo has multiple layers
+ * of properties, in order of precedence (highest - lowest):
+ * <ul>
+ * <li>the properties set in Zookeeper</li>
+ * <li>the properties set in the site configuration file</li>
+ * <li>the default properties</li>
+ * </ul>
+ * The properties returned is the merged view of these properties. The properties that are stored
+ * in ZooKeeper can be modified with {@link #modifyProperties modifyProperties},
+ * {@link #setProperty setProperty} and {@link #removeProperty removeProperty}. Properties can be
+ * further refined by namesapce {@link NamespaceOperations} and by table {@link TableOperations}.
*
* @return A map of system properties set in zookeeper. If a property is not set in zookeeper,
* then it will return the value set in accumulo.properties on some server. If nothing is
@@ -157,6 +167,16 @@
Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException;
/**
+ * Retrieve a map of the system properties set in Zookeeper. Note that this does not return a
+ * merged view of the properties from its parent configuration. See
+ * {@link #getSystemConfiguration} for a merged view.
+ *
+ * @return A map of the system properties set in Zookeeper only.
+ * @since 3.1
+ */
+ Map<String,String> getSystemProperties() throws AccumuloException, AccumuloSecurityException;
+
+ /**
* Returns the location(s) of the accumulo manager and any redundant servers.
*
* @return a list of locations in <code>hostname:port</code> form.
@@ -237,17 +257,6 @@
void waitForBalance() throws AccumuloException;
/**
- * Returns a unique string that identifies this instance of accumulo.
- *
- * @return a String
- * @since 2.0.0
- *
- * @deprecated in 2.1.0 Use {@link #getInstanceId()}
- */
- @Deprecated(since = "2.1.0")
- String getInstanceID();
-
- /**
* Returns a unique ID object that identifies this instance of accumulo.
*
* @return an InstanceId
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
deleted file mode 100644
index 2815174..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
+++ /dev/null
@@ -1,84 +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
- *
- * https://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.admin;
-
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.replication.PeerExistsException;
-import org.apache.accumulo.core.client.replication.PeerNotFoundException;
-
-/**
- * Supports replication configuration
- *
- * @since 1.7.0
- */
-@Deprecated
-public interface ReplicationOperations {
-
- /**
- * Defines a cluster with the given name and the given name system.
- *
- * @param name Unique name for the cluster
- * @param replicaType Class name to use to replicate the data
- */
- void addPeer(String name, String replicaType)
- throws AccumuloException, AccumuloSecurityException, PeerExistsException;
-
- /**
- * Removes a cluster with the given name.
- *
- * @param name Name of the cluster to remove
- */
- void removePeer(String name)
- throws AccumuloException, AccumuloSecurityException, PeerNotFoundException;
-
- /**
- * Waits for a table to be fully replicated, given the state of files pending replication for the
- * provided table at the point in time which this method is invoked.
- *
- * @param tableName The table to wait for
- */
- void drain(String tableName)
- throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-
- /**
- * Given the provided set of files that are pending replication for a table, wait for those files
- * to be fully replicated to all configured peers. This allows for the accurate calculation when a
- * table, at a given point in time, has been fully replicated.
- *
- * @param tableName The table to wait for
- */
- void drain(String tableName, Set<String> files)
- throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-
- /**
- * Gets all of the referenced files for a table from the metadata table. The result of this method
- * is intended to be directly supplied to {@link #drain(String, Set)}. This helps determine when
- * all data from a given point in time has been fully replicated.
- * <p>
- * This also allows callers to get the {@link Set} of files for a table at some time, and later
- * provide that {@link Set} to {@link #drain(String,Set)} to wait for all of those files to be
- * replicated.
- */
- Set<String> referencedFiles(String tableName)
- throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
index eabafaa..fef5599 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
@@ -88,47 +88,6 @@
throws AccumuloException, AccumuloSecurityException, TableExistsException;
/**
- * @param tableName the name of the table
- * @param limitVersion Enables/disables the versioning iterator, which will limit the number of
- * Key versions kept.
- * @throws AccumuloException if a general error occurs
- * @throws AccumuloSecurityException if the user does not have permission
- * @throws TableExistsException if the table already exists
- * @deprecated since 1.7.0; use {@link #create(String, NewTableConfiguration)} instead.
- */
- @Deprecated(since = "1.7.0")
- default void create(String tableName, boolean limitVersion)
- throws AccumuloException, AccumuloSecurityException, TableExistsException {
- if (limitVersion) {
- create(tableName);
- } else {
- create(tableName, new NewTableConfiguration().withoutDefaultIterators());
- }
- }
-
- /**
- * @param tableName the name of the table
- * @param versioningIter Enables/disables the versioning iterator, which will limit the number of
- * Key versions kept.
- * @param timeType specifies logical or real-time based time recording for entries in the table
- * @throws AccumuloException if a general error occurs
- * @throws AccumuloSecurityException if the user does not have permission
- * @throws TableExistsException if the table already exists
- * @deprecated since 1.7.0; use {@link #create(String, NewTableConfiguration)} instead.
- */
- @Deprecated(since = "1.7.0")
- default void create(String tableName, boolean versioningIter, TimeType timeType)
- throws AccumuloException, AccumuloSecurityException, TableExistsException {
- NewTableConfiguration ntc = new NewTableConfiguration().setTimeType(timeType);
-
- if (versioningIter) {
- create(tableName, ntc);
- } else {
- create(tableName, ntc.withoutDefaultIterators());
- }
- }
-
- /**
* Create a table with specified configuration. A safe way to ignore tables that do exist would be
* to do something like the following:
*
@@ -228,21 +187,6 @@
* @param tableName the name of the table
* @return the split points (end-row names) for the table's current split profile
* @throws TableNotFoundException if the table does not exist
- * @deprecated since 1.5.0; use {@link #listSplits(String)} instead.
- */
- @Deprecated(since = "1.5.0")
- default Collection<Text> getSplits(String tableName) throws TableNotFoundException {
- try {
- return listSplits(tableName);
- } catch (AccumuloSecurityException | AccumuloException e) {
- throw new RuntimeException(e);
- }
- }
-
- /**
- * @param tableName the name of the table
- * @return the split points (end-row names) for the table's current split profile
- * @throws TableNotFoundException if the table does not exist
* @throws AccumuloException if a general error occurs
* @throws AccumuloSecurityException if the user does not have permission
* @since 1.5.0
@@ -253,23 +197,6 @@
/**
* @param tableName the name of the table
* @param maxSplits specifies the maximum number of splits to return
- * @return the split points (end-row names) for the table's current split profile, grouped into
- * fewer splits so as not to exceed maxSplits
- * @deprecated since 1.5.0; use {@link #listSplits(String, int)} instead.
- */
- @Deprecated(since = "1.5.0")
- default Collection<Text> getSplits(String tableName, int maxSplits)
- throws TableNotFoundException {
- try {
- return listSplits(tableName, maxSplits);
- } catch (AccumuloSecurityException | AccumuloException e) {
- throw new RuntimeException(e);
- }
- }
-
- /**
- * @param tableName the name of the table
- * @param maxSplits specifies the maximum number of splits to return
* @throws AccumuloException if a general error occurs
* @throws AccumuloSecurityException if the user does not have permission
* @return the split points (end-row names) for the table's current split profile, grouped into
@@ -363,8 +290,8 @@
/**
* Starts a full major compaction of the tablets in the range (start, end]. If the config does not
- * specify a compaction selector (or a deprecated strategy), then all files in a tablet are
- * compacted. The compaction is performed even for tablets that have only one file.
+ * specify a compaction selector, then all files in a tablet are compacted. The compaction is
+ * performed even for tablets that have only one file.
*
* <p>
* The following optional settings can only be set by one compact call per table at the same time.
@@ -374,8 +301,6 @@
* <li>Selector : {@link CompactionConfig#setSelector(PluginConfig)}</li>
* <li>Configurer : {@link CompactionConfig#setConfigurer(PluginConfig)}</li>
* <li>Iterators : {@link CompactionConfig#setIterators(List)}</li>
- * <li>Compaction strategy (deprecated) :
- * {@code CompactionConfig.setCompactionStrategy(CompactionStrategyConfig)}</li>
* </ul>
*
* <p>
@@ -640,27 +565,6 @@
throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
/**
- * Bulk import all the files in a directory into a table. Files can be created using
- * {@link RFile#newWriter()}
- *
- * @param tableName the name of the table
- * @param dir the HDFS directory to find files for importing
- * @param failureDir the HDFS directory to place files that failed to be imported, must exist and
- * be empty
- * @param setTime override the time values in the input files, and use the current time for all
- * mutations
- * @throws IOException when there is an error reading/writing to HDFS
- * @throws AccumuloException when there is a general accumulo error
- * @throws AccumuloSecurityException when the user does not have the proper permissions
- * @throws TableNotFoundException when the table no longer exists
- *
- * @deprecated since 2.0.0 use {@link #importDirectory(String)} instead.
- */
- @Deprecated(since = "2.0.0")
- void importDirectory(String tableName, String dir, String failureDir, boolean setTime)
- throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException;
-
- /**
* @since 2.0.0
*/
interface ImportOptions {
@@ -749,11 +653,9 @@
* Bulk import the files in a directory into a table. Files can be created using
* {@link RFile#newWriter()}.
* <p>
- * This new method of bulk import examines files in the current process outside of holding a table
- * lock. The old bulk import method ({@link #importDirectory(String, String, String, boolean)})
- * examines files on the server side while holding a table read lock.
- * <p>
- * This API supports adding files to online and offline tables.
+ * This API supports adding files to online and offline tables. The files are examined on the
+ * client side to determine destination tablets. This examination will use memory and cpu within
+ * the process calling this API.
* <p>
* For example, to bulk import files from the directory 'dir1' into the table 'table1' use the
* following code.
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java
index baf5f04..36b2d7b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java
@@ -20,6 +20,7 @@
import java.net.URI;
+import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.metadata.CompactableFileImpl;
/**
@@ -33,6 +34,16 @@
public URI getUri();
+ /**
+ * @return A range associated with the file. If a file has an associated range then Accumulo will
+ * limit reads to within the range. Not all files have an associated range, it a file does
+ * not have a range then an infinite range is returned. The URI plus this range uniquely
+ * identify a file.
+ *
+ * @since 3.1.0
+ */
+ public Range getRange();
+
public long getEstimatedSize();
public long getEstimatedEntries();
@@ -41,4 +52,12 @@
return new CompactableFileImpl(uri, estimatedSize, estimatedEntries);
}
+ /**
+ * Creates a new CompactableFile object that implements this interface.
+ *
+ * @since 3.1.0
+ */
+ static CompactableFile create(URI uri, Range range, long estimatedSize, long estimatedEntries) {
+ return new CompactableFileImpl(uri, range, estimatedSize, estimatedEntries);
+ }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
index 6a58704..1e0e47c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
@@ -20,9 +20,11 @@
import java.util.Collection;
import java.util.Map;
+import java.util.Set;
import org.apache.accumulo.core.client.PluginEnvironment;
import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
/**
* Enables dynamically overriding of per table properties used to create the output file for a
@@ -50,8 +52,36 @@
public interface InputParameters {
TableId getTableId();
+ /**
+ * @return the id of the tablet being compacted
+ * @since 3.0.0
+ */
+ TabletId getTabletId();
+
public Collection<CompactableFile> getInputFiles();
+ /**
+ * For user and selector compactions:
+ * <ul>
+ * <li>Returns the selected set of files to be compacted.</li>
+ * <li>When getInputFiles() (inputFiles) and getSelectedFiles() (selectedFiles) are equal, then
+ * this is the final compaction.</li>
+ * <li>When they are not equal, this is an intermediate compaction.</li>
+ * <li>Intermediate compactions are compactions whose resultant RFile will be consumed by
+ * another compaction.</li>
+ * <li>inputFiles and selectedFiles can be compared using: <code>
+ * selectedFiles.equals(inputFiles instanceof Set ? inputFiles : Set.copyOf(inputFiles))
+ * </code></li>
+ * </ul>
+ * For system compactions:
+ * <ul>
+ * <li>There is no selected set of files so the empty set is returned.</li>
+ * </ul>
+ *
+ * @since 3.1
+ */
+ public Set<CompactableFile> getSelectedFiles();
+
PluginEnvironment getEnvironment();
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java
index d54f612..713a451 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java
@@ -30,12 +30,14 @@
import org.apache.accumulo.core.client.summary.Summary;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
/**
* This class selects which files a user compaction will compact. It can also be configured per
- * table to periodically select files to compact.
+ * table to periodically select files to compact, although per table functionality is deprecated.
+ * See {@link org.apache.accumulo.core.spi.compaction.CompactionKind#SELECTOR} for details.
*
* @since 2.1.0
*/
@@ -61,6 +63,12 @@
TableId getTableId();
+ /**
+ * @return the tablet id of the tablet being compacted
+ * @since 3.0.0
+ */
+ TabletId getTabletId();
+
Optional<SortedKeyValueIterator<Key,Value>> getSample(CompactableFile cf,
SamplerConfiguration sc);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
index 1d468e4..9195684 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
@@ -22,6 +22,7 @@
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.math.BigInteger;
import org.apache.accumulo.core.clientImpl.lexicoder.FixedByteArrayOutputStream;
@@ -59,7 +60,7 @@
return ret;
} catch (IOException ioe) {
- throw new RuntimeException(ioe);
+ throw new UncheckedIOException(ioe);
}
}
@@ -87,7 +88,7 @@
return new BigInteger(bytes);
} catch (IOException ioe) {
- throw new RuntimeException(ioe);
+ throw new UncheckedIOException(ioe);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
index 590c370..e5172a5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
@@ -80,7 +80,7 @@
byte[][] fields = split(data, offset, len);
if (fields.length != 2) {
- throw new RuntimeException("Data does not have 2 fields, it has " + fields.length);
+ throw new IllegalArgumentException("Data does not have 2 fields, it has " + fields.length);
}
return new ComparablePair<>(firstLexicoder.decode(unescape(fields[0])),
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
index b3a7bc2..fa1f3fd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
@@ -22,6 +22,7 @@
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.UUID;
import org.apache.accumulo.core.clientImpl.lexicoder.FixedByteArrayOutputStream;
@@ -52,7 +53,7 @@
return ret;
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
}
@@ -69,7 +70,7 @@
DataInputStream in = new DataInputStream(new ByteArrayInputStream(b, offset, len));
return new UUID(in.readLong() ^ 0x8000000000000000L, in.readLong() ^ 0x8000000000000000L);
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
deleted file mode 100644
index 088a065..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ /dev/null
@@ -1,753 +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
- *
- * https://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.mapred;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.SecureRandom;
-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.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-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.ScannerBase;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.OfflineScanner;
-import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.security.token.Token;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * An abstract input format to provide shared methods common to all other input format classes. At
- * the very least, any classes inheriting from this class will need to define their own
- * {@link RecordReader}.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
-
- private static final SecureRandom random = new SecureRandom();
-
- protected static final Class<?> CLASS = AccumuloInputFormat.class;
- protected static final Logger log = Logger.getLogger(CLASS);
-
- /**
- * Sets the name of the classloader context on this scanner
- *
- * @param job the Hadoop job instance to be configured
- * @param context name of the classloader context
- * @since 1.8.0
- */
- public static void setClassLoaderContext(JobConf job, String context) {
- InputConfigurator.setClassLoaderContext(CLASS, job, context);
- }
-
- /**
- * Returns the name of the current classloader context set on this scanner
- *
- * @param job the Hadoop job instance to be configured
- * @return name of the current context
- * @since 1.8.0
- */
- public static String getClassLoaderContext(JobConf job) {
- return InputConfigurator.getClassLoaderContext(CLASS, job);
- }
-
- /**
- * Sets the connector information needed to communicate with Accumulo in this job.
- *
- * <p>
- * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the
- * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
- * encoded to provide a charset safe conversion to a string, but this conversion is not intended
- * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
- * {@link DelegationToken}s, acquired using
- * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
- * concern.
- *
- * @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
- * @since 1.5.0
- */
- public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token)
- throws AccumuloSecurityException {
- if (token instanceof KerberosToken) {
- log.info("Received KerberosToken, attempting to fetch DelegationToken");
- try {
- ClientContext client = InputConfigurator.client(CLASS, job);
- token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
- } catch (Exception e) {
- log.warn("Failed to automatically obtain DelegationToken, Mappers/Reducers will likely"
- + " fail to communicate with Accumulo", e);
- }
- }
- // DelegationTokens can be passed securely from user to task without serializing insecurely in
- // the configuration
- if (token instanceof DelegationTokenImpl) {
- DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
- // Convert it into a Hadoop Token
- AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
- Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
- delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
- // Add the Hadoop Token to the Job so it gets serialized and passed along.
- job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
- }
-
- InputConfigurator.setConnectorInfo(CLASS, job, 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
- * @since 1.6.0
- */
- public static void setConnectorInfo(JobConf job, String principal, String tokenFile)
- throws AccumuloSecurityException {
- InputConfigurator.setConnectorInfo(CLASS, job, principal, tokenFile);
- }
-
- /**
- * Determines if the connector has been configured.
- *
- * @param job the Hadoop context for the configured job
- * @return true if the connector has been configured, false otherwise
- * @since 1.5.0
- * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
- */
- protected static Boolean isConnectorInfoSet(JobConf job) {
- return InputConfigurator.isConnectorInfoSet(CLASS, job);
- }
-
- /**
- * Gets the user name from the configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return the user name
- * @since 1.5.0
- * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
- */
- protected static String getPrincipal(JobConf job) {
- return InputConfigurator.getPrincipal(CLASS, job);
- }
-
- /**
- * Gets the authenticated token from either the specified token file or directly from the
- * configuration, whichever was used when the job was configured.
- *
- * @param job the Hadoop context for the configured job
- * @return the principal's authentication token
- * @since 1.6.0
- * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
- * @see #setConnectorInfo(JobConf, String, String)
- */
- protected static AuthenticationToken getAuthenticationToken(JobConf job) {
- AuthenticationToken token = InputConfigurator.getAuthenticationToken(CLASS, job);
- return InputConfigurator.unwrapAuthenticationToken(job, token);
- }
-
- /**
- * 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
- * @deprecated since 1.6.0; Use
- * {@link #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * instead.
- */
- @Deprecated(since = "1.6.0")
- public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
- setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
- .withInstance(instanceName).withZkHosts(zooKeepers));
- }
-
- /**
- * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
- *
- * @param job the Hadoop job instance to be configured
- * @param clientConfig client configuration containing connection options
- * @since 1.6.0
- */
- public static void setZooKeeperInstance(JobConf job,
- org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
- InputConfigurator.setZooKeeperInstance(CLASS, job, clientConfig);
- }
-
- /**
- * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
- * configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return an Accumulo instance
- * @since 1.5.0
- * @see #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)
- */
- protected static org.apache.accumulo.core.client.Instance getInstance(JobConf job) {
- return InputConfigurator.getInstance(CLASS, job);
- }
-
- /**
- * 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(JobConf job, Level level) {
- InputConfigurator.setLogLevel(CLASS, job, level);
- }
-
- /**
- * Gets the log level from this configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return the log level
- * @since 1.5.0
- * @see #setLogLevel(JobConf, Level)
- */
- protected static Level getLogLevel(JobConf job) {
- return InputConfigurator.getLogLevel(CLASS, job);
- }
-
- /**
- * 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
- * @since 1.5.0
- */
- public static void setScanAuthorizations(JobConf job, Authorizations auths) {
- InputConfigurator.setScanAuthorizations(CLASS, job, auths);
- }
-
- /**
- * Gets the authorizations to set for the scans from the configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return the Accumulo scan authorizations
- * @since 1.5.0
- * @see #setScanAuthorizations(JobConf, Authorizations)
- */
- protected static Authorizations getScanAuthorizations(JobConf job) {
- return InputConfigurator.getScanAuthorizations(CLASS, job);
- }
-
- /**
- * Fetch the client configuration from the job.
- *
- * @param job The job
- * @return The client configuration for the job
- * @since 1.7.0
- */
- protected static org.apache.accumulo.core.client.ClientConfiguration
- getClientConfiguration(JobConf job) {
- return InputConfigurator.getClientConfiguration(CLASS, job);
- }
-
- // 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 InputFormat}.
- *
- * @param job 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(JobConf job) throws IOException {
- InputConfigurator.validatePermissions(CLASS, job);
- }
-
- /**
- * Fetches all {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig}s that have been
- * set on the given Hadoop job.
- *
- * @param job the Hadoop job instance to be configured
- * @return the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects set on
- * the job
- * @since 1.6.0
- */
- public static Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig>
- getInputTableConfigs(JobConf job) {
- return InputConfigurator.getInputTableConfigs(CLASS, job);
- }
-
- /**
- * Fetches a {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} 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 org.apache.accumulo.core.client.mapreduce.InputTableConfig} for the given
- * table
- * @since 1.6.0
- */
- public static org.apache.accumulo.core.client.mapreduce.InputTableConfig
- getInputTableConfig(JobConf job, String tableName) {
- return InputConfigurator.getInputTableConfig(CLASS, job, tableName);
- }
-
- /**
- * 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 #next(Object, Object)} to update key and value, and also to
- * update the following variables:
- * <ul>
- * <li>Key {@link #currentKey} (used for progress reporting)</li>
- * <li>int {@link #numKeysRead} (used for progress reporting)</li>
- * </ul>
- */
- protected abstract static class AbstractRecordReader<K,V> implements RecordReader<K,V> {
- protected long numKeysRead;
- protected Iterator<Map.Entry<Key,Value>> scannerIterator;
- protected RangeInputSplit split;
- private org.apache.accumulo.core.client.mapreduce.RangeInputSplit baseSplit;
- protected ScannerBase scannerBase;
-
- /**
- * Extracts Iterators settings from the context to be used by RecordReader.
- *
- * @param job the Hadoop job configuration
- * @param tableName the table name for which the scanner is configured
- * @return List of iterator settings for given table
- * @since 1.7.0
- */
- protected abstract List<IteratorSetting> jobIterators(JobConf job, String tableName);
-
- /**
- * Configures the iterators on a scanner for the given table name.
- *
- * @param job the Hadoop job configuration
- * @param scanner the scanner for which to configure the iterators
- * @param tableName the table name for which the scanner is configured
- * @since 1.7.0
- */
- private void setupIterators(JobConf job, ScannerBase scanner, String tableName,
- org.apache.accumulo.core.client.mapreduce.RangeInputSplit split) {
- List<IteratorSetting> iterators = null;
-
- if (split == null) {
- iterators = jobIterators(job, tableName);
- } else {
- iterators = split.getIterators();
- if (iterators == null) {
- iterators = jobIterators(job, tableName);
- }
- }
-
- for (IteratorSetting iterator : iterators) {
- scanner.addScanIterator(iterator);
- }
- }
-
- /**
- * Configures the iterators on a scanner for the given table name.
- *
- * @param job the Hadoop job configuration
- * @param scanner the scanner for which to configure the iterators
- * @param tableName the table name for which the scanner is configured
- * @since 1.6.0
- * @deprecated since 1.7.0; Use {@link #jobIterators} instead.
- */
- @Deprecated(since = "1.7.0")
- protected void setupIterators(JobConf job, Scanner scanner, String tableName,
- RangeInputSplit split) {
- setupIterators(job, (ScannerBase) scanner, tableName, split);
- }
-
- /**
- * Initialize a scanner over the given input split using this task attempt configuration.
- */
- public void initialize(InputSplit inSplit, JobConf job) throws IOException {
- baseSplit = (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) inSplit;
- log.debug("Initializing input split: " + baseSplit);
-
- Authorizations authorizations = baseSplit.getAuths();
- if (null == authorizations) {
- authorizations = getScanAuthorizations(job);
- }
-
- String classLoaderContext = getClassLoaderContext(job);
- String table = baseSplit.getTableName();
-
- // in case the table name changed, we can still use the previous name for terms of
- // configuration, but the scanner will use the table id resolved at job setup time
- org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig =
- getInputTableConfig(job, baseSplit.getTableName());
-
- ClientContext client = InputConfigurator.client(CLASS, baseSplit, job);
-
- log.debug("Created client with user: " + client.whoami());
- log.debug("Creating scanner for table: " + table);
- log.debug("Authorizations are: " + authorizations);
-
- if (baseSplit instanceof org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit) {
- BatchScanner scanner;
- org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit multiRangeSplit =
- (org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit) baseSplit;
-
- try {
- // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
- // will not span tablets
- int scanThreads = 1;
- scanner =
- client.createBatchScanner(baseSplit.getTableName(), authorizations, scanThreads);
- setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
- if (classLoaderContext != null) {
- scanner.setClassLoaderContext(classLoaderContext);
- }
- } catch (Exception e) {
- throw new IOException(e);
- }
-
- scanner.setRanges(multiRangeSplit.getRanges());
- scannerBase = scanner;
-
- } else if (baseSplit instanceof RangeInputSplit) {
- split = (RangeInputSplit) baseSplit;
- Boolean isOffline = baseSplit.isOffline();
- if (isOffline == null) {
- isOffline = tableConfig.isOfflineScan();
- }
-
- Boolean isIsolated = baseSplit.isIsolatedScan();
- if (isIsolated == null) {
- isIsolated = tableConfig.shouldUseIsolatedScanners();
- }
-
- Boolean usesLocalIterators = baseSplit.usesLocalIterators();
- if (usesLocalIterators == null) {
- usesLocalIterators = tableConfig.shouldUseLocalIterators();
- }
-
- Scanner scanner;
-
- try {
- if (isOffline) {
- scanner =
- new OfflineScanner(client, TableId.of(baseSplit.getTableId()), authorizations);
- } else {
- scanner = new ScannerImpl(client, TableId.of(baseSplit.getTableId()), authorizations);
- }
- if (isIsolated) {
- log.info("Creating isolated scanner");
- @SuppressWarnings("resource")
- var wrapped = new IsolatedScanner(scanner);
- scanner = wrapped;
- }
- if (usesLocalIterators) {
- log.info("Using local iterators");
- scanner = new ClientSideIteratorScanner(scanner);
- }
- setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
- } catch (Exception e) {
- throw new IOException(e);
- }
-
- scanner.setRange(baseSplit.getRange());
- scannerBase = scanner;
- } else {
- throw new IllegalArgumentException("Can not initialize from " + baseSplit.getClass());
- }
-
- Collection<Pair<Text,Text>> columns = baseSplit.getFetchedColumns();
- if (columns == null) {
- columns = tableConfig.getFetchedColumns();
- }
-
- // setup a scanner within the bounds of this split
- for (Pair<Text,Text> c : columns) {
- if (c.getSecond() != null) {
- log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
- scannerBase.fetchColumn(c.getFirst(), c.getSecond());
- } else {
- log.debug("Fetching column family " + c.getFirst());
- scannerBase.fetchColumnFamily(c.getFirst());
- }
- }
-
- SamplerConfiguration samplerConfig = baseSplit.getSamplerConfiguration();
- if (samplerConfig == null) {
- samplerConfig = tableConfig.getSamplerConfiguration();
- }
-
- if (samplerConfig != null) {
- scannerBase.setSamplerConfiguration(samplerConfig);
- }
-
- scannerIterator = scannerBase.iterator();
- numKeysRead = 0;
- }
-
- @Override
- public void close() {
- if (scannerBase != null) {
- scannerBase.close();
- }
- }
-
- @Override
- public long getPos() throws IOException {
- return numKeysRead;
- }
-
- @Override
- public float getProgress() throws IOException {
- if (numKeysRead > 0 && currentKey == null) {
- return 1.0f;
- }
- return baseSplit.getProgress(currentKey);
- }
-
- protected Key currentKey = null;
-
- }
-
- Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job, TableId tableId,
- List<Range> ranges)
- throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
- return InputConfigurator.binOffline(tableId, ranges, InputConfigurator.client(CLASS, job));
- }
-
- /**
- * Gets the splits of the tables that have been set on the job by reading the metadata table for
- * the specified ranges.
- *
- * @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
- */
- @Override
- public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
- Level logLevel = getLogLevel(job);
- log.setLevel(logLevel);
- validateOptions(job);
-
- LinkedList<InputSplit> splits = new LinkedList<>();
- Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfigs =
- getInputTableConfigs(job);
-
- for (Map.Entry<String,
- org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfigEntry : tableConfigs
- .entrySet()) {
-
- String tableName = tableConfigEntry.getKey();
- org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig =
- tableConfigEntry.getValue();
-
- ClientContext client;
- try {
- client = InputConfigurator.client(CLASS, job);
- } catch (AccumuloException | AccumuloSecurityException e) {
- throw new IOException(e);
- }
-
- TableId tableId;
- // resolve table name to id once, and use id from this point forward
- try {
- tableId = client.getTableId(tableName);
- } catch (TableNotFoundException e) {
- throw new IOException(e);
- }
-
- boolean batchScan = InputConfigurator.isBatchScan(CLASS, job);
- boolean supportBatchScan = !(tableConfig.isOfflineScan()
- || tableConfig.shouldUseIsolatedScanners() || tableConfig.shouldUseLocalIterators());
- if (batchScan && !supportBatchScan) {
- throw new IllegalArgumentException("BatchScanner optimization not available for offline"
- + " scan, isolated, or local iterators");
- }
-
- boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
- if (batchScan && !autoAdjust) {
- throw new IllegalArgumentException(
- "AutoAdjustRanges must be enabled when using BatchScanner optimization");
- }
-
- List<Range> ranges =
- autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
- if (ranges.isEmpty()) {
- ranges = new ArrayList<>(1);
- ranges.add(new Range());
- }
-
- // get the metadata information for these ranges
- Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
- TabletLocator tl;
- try {
- if (tableConfig.isOfflineScan()) {
- binnedRanges = binOfflineTable(job, tableId, ranges);
- while (binnedRanges == null) {
- // Some tablets were still online, try again
- // sleep randomly between 100 and 200 ms
- sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
- binnedRanges = binOfflineTable(job, tableId, ranges);
- }
- } else {
- tl = TabletLocator.getLocator(client, tableId);
- // its possible that the cache could contain complete, but old information about a
- // tables tablets... so clear it
- tl.invalidateCache();
-
- while (!tl.binRanges(client, ranges, binnedRanges).isEmpty()) {
- client.requireNotDeleted(tableId);
- client.requireNotOffline(tableId, tableName);
- binnedRanges.clear();
- log.warn("Unable to locate bins for specified ranges. Retrying.");
- // sleep randomly between 100 and 200 ms
- sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
- tl.invalidateCache();
- }
- }
- } catch (Exception e) {
- throw new IOException(e);
- }
-
- // all of this code will add either range per each locations or split ranges and add
- // range-location split
- // Map from Range to Array of Locations, we only use this if we're don't split
- HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
- if (!autoAdjust) {
- splitsToAdd = new HashMap<>();
- }
-
- HashMap<String,String> hostNameCache = new HashMap<>();
- 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.getCanonicalHostName();
- hostNameCache.put(ip, location);
- }
- for (Map.Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
- Range ke = extentRanges.getKey().toDataRange();
- if (batchScan) {
- // group ranges by tablet to be read by a BatchScanner
- ArrayList<Range> clippedRanges = new ArrayList<>();
- for (Range r : extentRanges.getValue()) {
- clippedRanges.add(ke.clip(r));
- }
- org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit split =
- new org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit(tableName, tableId,
- clippedRanges, new String[] {location});
- org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
- logLevel);
-
- splits.add(split);
- } else {
- // not grouping by tablet
- for (Range r : extentRanges.getValue()) {
- if (autoAdjust) {
- // divide ranges into smaller ranges, based on the tablets
- RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
- ke.clip(r), new String[] {location});
- org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split,
- tableConfig, logLevel);
- split.setOffline(tableConfig.isOfflineScan());
- split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
- split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
- splits.add(split);
- } else {
- // don't divide ranges
- ArrayList<String> locations = splitsToAdd.get(r);
- if (locations == null) {
- locations = new ArrayList<>(1);
- }
- locations.add(location);
- splitsToAdd.put(r, locations);
- }
- }
- }
- }
- }
-
- if (!autoAdjust) {
- for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
- RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
- entry.getKey(), entry.getValue().toArray(new String[0]));
- org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
- logLevel);
- split.setOffline(tableConfig.isOfflineScan());
- split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
- split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-
- splits.add(split);
- }
- }
- }
- return splits.toArray(new InputSplit[splits.size()]);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
deleted file mode 100644
index 75c11ad..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
+++ /dev/null
@@ -1,176 +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
- *
- * https://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.mapred;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.client.rfile.RFile;
-import org.apache.accumulo.core.client.rfile.RFileWriter;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.util.Progressable;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to write output in the Accumulo data file format.<br>
- * Care should be taken to write only sorted data (sorted by {@link Key}), as this is an important
- * requirement of Accumulo data files.
- *
- * <p>
- * The output path to be created must be specified via
- * {@link AccumuloFileOutputFormat#setOutputPath(JobConf, Path)}. This is inherited from
- * {@link FileOutputFormat#setOutputPath(JobConf, Path)}. Other methods from
- * {@link FileOutputFormat} are not supported and may be ignored or cause failures. Using other
- * Hadoop configuration options that affect the behavior of the underlying files directly in the
- * Job's configuration may work, but are not directly supported at this time.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
-
- private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
- protected static final Logger log = Logger.getLogger(CLASS);
-
- /**
- * Sets the compression type to use for data blocks. Specifying a compression may require
- * additional libraries to be available to your Job.
- *
- * @param job the Hadoop job instance to be configured
- * @param compressionType one of "none", "gz", "bzip2", "lzo", "lz4", "snappy", or "zstd"
- * @since 1.5.0
- */
- public static void setCompressionType(JobConf job, String compressionType) {
- FileOutputConfigurator.setCompressionType(CLASS, job, compressionType);
- }
-
- /**
- * Sets the size for data blocks within each file.<br>
- * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as
- * a group.
- *
- * <p>
- * Making this value smaller may increase seek performance, but at the cost of increasing the size
- * of the indexes (which can also affect seek performance).
- *
- * @param job the Hadoop job instance to be configured
- * @param dataBlockSize the block size, in bytes
- * @since 1.5.0
- */
- public static void setDataBlockSize(JobConf job, long dataBlockSize) {
- FileOutputConfigurator.setDataBlockSize(CLASS, job, dataBlockSize);
- }
-
- /**
- * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by
- * the underlying file system.
- *
- * @param job the Hadoop job instance to be configured
- * @param fileBlockSize the block size, in bytes
- * @since 1.5.0
- */
- public static void setFileBlockSize(JobConf job, long fileBlockSize) {
- FileOutputConfigurator.setFileBlockSize(CLASS, job, fileBlockSize);
- }
-
- /**
- * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy
- * within the file, while larger blocks mean a more shallow index hierarchy within the file. This
- * can affect the performance of queries.
- *
- * @param job the Hadoop job instance to be configured
- * @param indexBlockSize the block size, in bytes
- * @since 1.5.0
- */
- public static void setIndexBlockSize(JobConf job, long indexBlockSize) {
- FileOutputConfigurator.setIndexBlockSize(CLASS, job, indexBlockSize);
- }
-
- /**
- * Sets the file system replication factor for the resulting file, overriding the file system
- * default.
- *
- * @param job the Hadoop job instance to be configured
- * @param replication the number of replicas for produced files
- * @since 1.5.0
- */
- public static void setReplication(JobConf job, int replication) {
- FileOutputConfigurator.setReplication(CLASS, job, replication);
- }
-
- /**
- * Specify a sampler to be used when writing out data. This will result in the output file having
- * sample data.
- *
- * @param job The Hadoop job instance to be configured
- * @param samplerConfig The configuration for creating sample data in the output file.
- * @since 1.8.0
- */
-
- public static void setSampler(JobConf job, SamplerConfiguration samplerConfig) {
- FileOutputConfigurator.setSampler(CLASS, job, samplerConfig);
- }
-
- @Override
- public RecordWriter<Key,Value> getRecordWriter(FileSystem ignored, JobConf job, String name,
- Progressable progress) throws IOException {
- // get the path of the temporary output file
- final Configuration conf = job;
- final AccumuloConfiguration acuConf =
- FileOutputConfigurator.getAccumuloConfiguration(CLASS, job);
-
- final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
- final Path file =
- new Path(getWorkOutputPath(job), getUniqueName(job, "part") + "." + extension);
- final int visCacheSize = FileOutputConfigurator.getVisibilityCacheSize(conf);
-
- return new RecordWriter<>() {
- RFileWriter out = null;
-
- @Override
- public void close(Reporter reporter) throws IOException {
- if (out != null) {
- out.close();
- }
- }
-
- @Override
- public void write(Key key, Value value) throws IOException {
- if (out == null) {
- out = RFile.newWriter().to(file.toString()).withFileSystem(file.getFileSystem(conf))
- .withTableProperties(acuConf).withVisibilityCacheSize(visCacheSize).build();
- out.startDefaultLocalityGroup();
- }
- out.append(key, value);
- }
- };
- }
-}
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
deleted file mode 100644
index 4e59225..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
+++ /dev/null
@@ -1,107 +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
- *
- * https://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.mapred;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.log4j.Level;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link 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 AccumuloInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloInputFormat#setInputTableName(JobConf, String)}
- * <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
-
- @Override
- public RecordReader<Key,Value> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
- throws IOException {
- log.setLevel(getLogLevel(job));
-
- // Override the log level from the configuration as if the RangeInputSplit has one it's the more
- // correct one to use.
- if (split instanceof org.apache.accumulo.core.client.mapreduce.RangeInputSplit) {
- org.apache.accumulo.core.client.mapreduce.RangeInputSplit accSplit =
- (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
- Level level = accSplit.getLogLevel();
- if (level != null) {
- log.setLevel(level);
- }
- } else {
- throw new IllegalArgumentException("No RecordReader for " + split.getClass());
- }
-
- RecordReaderBase<Key,Value> recordReader = new RecordReaderBase<>() {
-
- @Override
- public boolean next(Key key, Value value) throws IOException {
- if (scannerIterator.hasNext()) {
- ++numKeysRead;
- Entry<Key,Value> entry = scannerIterator.next();
- key.set(currentKey = entry.getKey());
- value.set(entry.getValue().get());
- if (log.isTraceEnabled()) {
- log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
- }
- return true;
- }
- return false;
- }
-
- @Override
- public Key createKey() {
- return new Key();
- }
-
- @Override
- public Value createValue() {
- return new Value();
- }
-
- };
- recordReader.initialize(split, job);
- return recordReader;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
deleted file mode 100644
index d0680d0..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
+++ /dev/null
@@ -1,108 +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
- *
- * https://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.mapred;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This class allows MapReduce jobs to use multiple Accumulo tables as the source of data. This
- * {@link org.apache.hadoop.mapred.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(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloMultiTableInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloMultiTableInputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * <li>{@link AccumuloMultiTableInputFormat#setInputTableConfigs(org.apache.hadoop.mapred.JobConf, java.util.Map)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value> {
-
- /**
- * Sets the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} 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 setInputTableConfigs(JobConf job,
- Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configs) {
- org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator.setInputTableConfigs(CLASS,
- job, configs);
- }
-
- @Override
- public RecordReader<Key,Value> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
- throws IOException {
- log.setLevel(getLogLevel(job));
- InputFormatBase.RecordReaderBase<Key,Value> recordReader =
- new InputFormatBase.RecordReaderBase<>() {
-
- @Override
- public boolean next(Key key, Value value) throws IOException {
- if (scannerIterator.hasNext()) {
- ++numKeysRead;
- Map.Entry<Key,Value> entry = scannerIterator.next();
- key.set(currentKey = entry.getKey());
- value.set(entry.getValue().get());
- if (log.isTraceEnabled()) {
- log.trace(
- "Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
- }
- return true;
- }
- return false;
- }
-
- @Override
- public Key createKey() {
- return new Key();
- }
-
- @Override
- public Value createValue() {
- return new Value();
- }
-
- };
- recordReader.initialize(split, job);
- return recordReader;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
deleted file mode 100644
index 6dbecae..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ /dev/null
@@ -1,577 +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
- *
- * https://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.mapred;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.security.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputFormat;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Progressable;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the sink for data. This {@link OutputFormat}
- * accepts keys and values of type {@link Text} (for a table name) and {@link Mutation} from the Map
- * and Reduce functions.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
-
- private static final Class<?> CLASS = AccumuloOutputFormat.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> Some tokens, when serialized, divulge sensitive information in the
- * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
- * encoded to provide a charset safe conversion to a string, but this conversion is not intended
- * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
- * {@link DelegationToken}s, acquired using
- * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
- * concern.
- *
- * @param job the Hadoop job instance to be configured
- * @param principal a valid Accumulo user name (user must have Table.CREATE permission if
- * {@link #setCreateTables(JobConf, boolean)} is set to true)
- * @param token the user's password
- * @since 1.5.0
- */
- public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token)
- throws AccumuloSecurityException {
- if (token instanceof KerberosToken) {
- log.info("Received KerberosToken, attempting to fetch DelegationToken");
- try {
- ClientContext client = OutputConfigurator.client(CLASS, job);
- token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
- } catch (Exception e) {
- log.warn("Failed to automatically obtain DelegationToken, "
- + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
- }
- }
- // DelegationTokens can be passed securely from user to task without serializing insecurely in
- // the configuration
- if (token instanceof DelegationTokenImpl) {
- DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
- // Convert it into a Hadoop Token
- AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
- Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
- delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
- // Add the Hadoop Token to the Job so it gets serialized and passed along.
- job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
- }
-
- OutputConfigurator.setConnectorInfo(CLASS, job, 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 if
- * {@link #setCreateTables(JobConf, boolean)} is set to true)
- * @param tokenFile the path to the password file
- * @since 1.6.0
- */
- public static void setConnectorInfo(JobConf job, String principal, String tokenFile)
- throws AccumuloSecurityException {
- OutputConfigurator.setConnectorInfo(CLASS, job, principal, tokenFile);
- }
-
- /**
- * Determines if the connector has been configured.
- *
- * @param job the Hadoop context for the configured job
- * @return true if the connector has been configured, false otherwise
- * @since 1.5.0
- * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
- */
- protected static Boolean isConnectorInfoSet(JobConf job) {
- return OutputConfigurator.isConnectorInfoSet(CLASS, job);
- }
-
- /**
- * Gets the principal from the configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return the user name
- * @since 1.5.0
- * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
- */
- protected static String getPrincipal(JobConf job) {
- return OutputConfigurator.getPrincipal(CLASS, job);
- }
-
- /**
- * 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(JobConf)} instead.
- */
- @Deprecated(since = "1.6.0")
- protected static String getTokenClass(JobConf job) {
- return getAuthenticationToken(job).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(JobConf)} instead.
- */
- @Deprecated(since = "1.6.0")
- protected static byte[] getToken(JobConf job) {
- return AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
- }
-
- /**
- * Gets the authenticated token from either the specified token file or directly from the
- * configuration, whichever was used when the job was configured.
- *
- * @param job the Hadoop job instance to be configured
- * @return the principal's authentication token
- * @since 1.6.0
- * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
- * @see #setConnectorInfo(JobConf, String, String)
- */
- protected static AuthenticationToken getAuthenticationToken(JobConf job) {
- AuthenticationToken token = OutputConfigurator.getAuthenticationToken(CLASS, job);
- return OutputConfigurator.unwrapAuthenticationToken(job, token);
- }
-
- /**
- * 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
- * @deprecated since 1.6.0; Use
- * {@link #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * instead.
- */
- @Deprecated(since = "1.6.0")
- public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
- setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
- .withInstance(instanceName).withZkHosts(zooKeepers));
- }
-
- /**
- * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
- *
- * @param job the Hadoop job instance to be configured
- *
- * @param clientConfig client configuration for specifying connection timeouts, SSL connection
- * options, etc.
- * @since 1.6.0
- */
- public static void setZooKeeperInstance(JobConf job,
- org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
- OutputConfigurator.setZooKeeperInstance(CLASS, job, clientConfig);
- }
-
- /**
- * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
- * configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return an Accumulo instance
- * @since 1.5.0
- * @see #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)
- */
- protected static org.apache.accumulo.core.client.Instance getInstance(JobConf job) {
- return OutputConfigurator.getInstance(CLASS, job);
- }
-
- /**
- * 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(JobConf job, Level level) {
- OutputConfigurator.setLogLevel(CLASS, job, level);
- }
-
- /**
- * Gets the log level from this configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return the log level
- * @since 1.5.0
- * @see #setLogLevel(JobConf, Level)
- */
- protected static Level getLogLevel(JobConf job) {
- return OutputConfigurator.getLogLevel(CLASS, job);
- }
-
- /**
- * Sets the default table name to use if one emits a null in place of a table name for a given
- * mutation. Table names can only be alpha-numeric and underscores.
- *
- * @param job the Hadoop job instance to be configured
- * @param tableName the table to use when the tablename is null in the write call
- * @since 1.5.0
- */
- public static void setDefaultTableName(JobConf job, String tableName) {
- OutputConfigurator.setDefaultTableName(CLASS, job, tableName);
- }
-
- /**
- * Gets the default table name from the configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return the default table name
- * @since 1.5.0
- * @see #setDefaultTableName(JobConf, String)
- */
- protected static String getDefaultTableName(JobConf job) {
- return OutputConfigurator.getDefaultTableName(CLASS, job);
- }
-
- /**
- * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new
- * {@link BatchWriterConfig}, with sensible built-in defaults is used. Setting the configuration
- * multiple times overwrites any previous configuration.
- *
- * @param job the Hadoop job instance to be configured
- * @param bwConfig the configuration for the {@link BatchWriter}
- * @since 1.5.0
- */
- public static void setBatchWriterOptions(JobConf job, BatchWriterConfig bwConfig) {
- OutputConfigurator.setBatchWriterOptions(CLASS, job, bwConfig);
- }
-
- /**
- * Gets the {@link BatchWriterConfig} settings.
- *
- * @param job the Hadoop context for the configured job
- * @return the configuration object
- * @since 1.5.0
- * @see #setBatchWriterOptions(JobConf, BatchWriterConfig)
- */
- protected static BatchWriterConfig getBatchWriterOptions(JobConf job) {
- return OutputConfigurator.getBatchWriterOptions(CLASS, job);
- }
-
- /**
- * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric
- * and underscores.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.5.0
- */
- public static void setCreateTables(JobConf job, boolean enableFeature) {
- OutputConfigurator.setCreateTables(CLASS, job, enableFeature);
- }
-
- /**
- * Determines whether tables are permitted to be created as needed.
- *
- * @param job the Hadoop context for the configured job
- * @return true if the feature is disabled, false otherwise
- * @since 1.5.0
- * @see #setCreateTables(JobConf, boolean)
- */
- protected static Boolean canCreateTables(JobConf job) {
- return OutputConfigurator.canCreateTables(CLASS, job);
- }
-
- /**
- * Sets the directive to use simulation mode for this job. In simulation mode, no output is
- * produced. This is useful for testing.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.5.0
- */
- public static void setSimulationMode(JobConf job, boolean enableFeature) {
- OutputConfigurator.setSimulationMode(CLASS, job, enableFeature);
- }
-
- /**
- * Determines whether this feature is enabled.
- *
- * @param job the Hadoop context for the configured job
- * @return true if the feature is enabled, false otherwise
- * @since 1.5.0
- * @see #setSimulationMode(JobConf, boolean)
- */
- protected static Boolean getSimulationMode(JobConf job) {
- return OutputConfigurator.getSimulationMode(CLASS, job);
- }
-
- /**
- * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
- */
- protected static class AccumuloRecordWriter implements RecordWriter<Text,Mutation> {
- private MultiTableBatchWriter mtbw = null;
- private HashMap<Text,BatchWriter> bws = null;
- private Text defaultTableName = null;
-
- private boolean simulate = false;
- private boolean createTables = false;
-
- private long mutCount = 0;
- private long valCount = 0;
-
- private AccumuloClient client;
-
- protected AccumuloRecordWriter(JobConf job)
- throws AccumuloException, AccumuloSecurityException, IOException {
- Level l = getLogLevel(job);
- if (l != null) {
- log.setLevel(getLogLevel(job));
- }
- this.simulate = getSimulationMode(job);
- this.createTables = canCreateTables(job);
-
- if (simulate) {
- log.info("Simulating output only. No writes to tables will occur");
- }
-
- this.bws = new HashMap<>();
-
- String tname = getDefaultTableName(job);
- this.defaultTableName = (tname == null) ? null : new Text(tname);
-
- if (!simulate) {
- this.client = OutputConfigurator.client(CLASS, job);
- mtbw = client.createMultiTableBatchWriter(getBatchWriterOptions(job));
- }
- }
-
- /**
- * Push a mutation into a table. If table is null, the defaultTable will be used. If
- * {@link AccumuloOutputFormat#canCreateTables(JobConf)} is set, the table will be created if it
- * does not exist. The table name must only contain alphanumerics and underscore.
- */
- @Override
- public void write(Text table, Mutation mutation) throws IOException {
- if (table == null || table.getLength() == 0) {
- table = this.defaultTableName;
- }
-
- if (!simulate && table == null) {
- throw new IOException("No table or default table specified. Try simulation mode next time");
- }
-
- ++mutCount;
- valCount += mutation.size();
- printMutation(table, mutation);
-
- if (simulate) {
- return;
- }
-
- if (!bws.containsKey(table)) {
- try {
- addTable(table);
- } catch (final Exception e) {
- log.error("Could not add table '" + table + "'", e);
- throw new IOException(e);
- }
- }
-
- try {
- bws.get(table).addMutation(mutation);
- } catch (MutationsRejectedException e) {
- throw new IOException(e);
- }
- }
-
- public void addTable(Text tableName) throws AccumuloException, AccumuloSecurityException {
- if (simulate) {
- log.info("Simulating adding table: " + tableName);
- return;
- }
-
- log.debug("Adding table: " + tableName);
- BatchWriter bw = null;
- String table = tableName.toString();
-
- if (createTables && !client.tableOperations().exists(table)) {
- try {
- client.tableOperations().create(table);
- } catch (AccumuloSecurityException e) {
- log.error("Accumulo security violation creating " + table, e);
- throw e;
- } catch (TableExistsException e) {
- // Shouldn't happen
- }
- }
-
- try {
- bw = mtbw.getBatchWriter(table);
- } catch (TableNotFoundException e) {
- log.error("Accumulo table " + table + " doesn't exist and cannot be created.", e);
- throw new AccumuloException(e);
- } catch (AccumuloException | AccumuloSecurityException e) {
- throw e;
- }
-
- if (bw != null) {
- bws.put(tableName, bw);
- }
- }
-
- private int printMutation(Text table, Mutation m) {
- if (log.isTraceEnabled()) {
- log.trace(String.format("Table %s row key: %s", table, hexDump(m.getRow())));
- for (ColumnUpdate cu : m.getUpdates()) {
- log.trace(String.format("Table %s column: %s:%s", table, hexDump(cu.getColumnFamily()),
- hexDump(cu.getColumnQualifier())));
- log.trace(String.format("Table %s security: %s", table,
- new ColumnVisibility(cu.getColumnVisibility())));
- log.trace(String.format("Table %s value: %s", table, hexDump(cu.getValue())));
- }
- }
- return m.getUpdates().size();
- }
-
- private String hexDump(byte[] ba) {
- StringBuilder sb = new StringBuilder();
- for (byte b : ba) {
- if ((b > 0x20) && (b < 0x7e)) {
- sb.append((char) b);
- } else {
- sb.append(String.format("x%02x", b));
- }
- }
- return sb.toString();
- }
-
- @Override
- public void close(Reporter reporter) throws IOException {
- log.debug("mutations written: " + mutCount + ", values written: " + valCount);
- if (simulate) {
- return;
- }
-
- try {
- mtbw.close();
- } catch (MutationsRejectedException e) {
- if (!e.getSecurityErrorCodes().isEmpty()) {
- var tables = new HashMap<String,Set<SecurityErrorCode>>();
- e.getSecurityErrorCodes().forEach((tabletId, secSet) -> {
- var tableId = tabletId.getTable().canonical();
- tables.computeIfAbsent(tableId, p -> new HashSet<>()).addAll(secSet);
- });
- log.error("Not authorized to write to tables : " + tables);
- }
-
- if (!e.getConstraintViolationSummaries().isEmpty()) {
- log.error("Constraint violations : " + e.getConstraintViolationSummaries().size());
- }
- throw new IOException(e);
- } finally {
- client.close();
- }
- }
- }
-
- @Override
- public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
- if (!isConnectorInfoSet(job)) {
- throw new IOException("Connector info has not been set.");
- }
- try {
- AccumuloClient c = OutputConfigurator.client(CLASS, job);
- String principal = getPrincipal(job);
- AuthenticationToken token = getAuthenticationToken(job);
- if (!c.securityOperations().authenticateUser(principal, token)) {
- throw new IOException("Unable to authenticate user");
- }
- } catch (AccumuloException | AccumuloSecurityException e) {
- throw new IOException(e);
- }
- }
-
- @Override
- public RecordWriter<Text,Mutation> getRecordWriter(FileSystem ignored, JobConf job, String name,
- Progressable progress) throws IOException {
- try {
- return new AccumuloRecordWriter(job);
- } catch (Exception e) {
- throw new IOException(e);
- }
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
deleted file mode 100644
index e8fa3e8..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
+++ /dev/null
@@ -1,99 +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
- *
- * https://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.mapred;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.RowIterator;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat}
- * provides row names as {@link Text} as keys, and a corresponding {@link PeekingIterator} as a
- * value, which in turn makes the {@link Key}/{@link Value} pairs for that row available to the Map
- * function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloRowInputFormat#setInputTableName(JobConf, String)}
- * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloRowInputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloRowInputFormat
- extends InputFormatBase<Text,PeekingIterator<Entry<Key,Value>>> {
- @Override
- public RecordReader<Text,PeekingIterator<Entry<Key,Value>>> getRecordReader(InputSplit split,
- JobConf job, Reporter reporter) throws IOException {
- log.setLevel(getLogLevel(job));
- RecordReaderBase<Text,PeekingIterator<Entry<Key,Value>>> recordReader =
- new RecordReaderBase<>() {
- RowIterator rowIterator;
-
- @Override
- public void initialize(InputSplit inSplit, JobConf job) throws IOException {
- super.initialize(inSplit, job);
- rowIterator = new RowIterator(scannerIterator);
- }
-
- @Override
- public boolean next(Text key, PeekingIterator<Entry<Key,Value>> value)
- throws IOException {
- if (!rowIterator.hasNext()) {
- return false;
- }
- value.initialize(rowIterator.next());
- numKeysRead = rowIterator.getKVCount();
- key.set((currentKey = value.peek().getKey()).getRow());
- return true;
- }
-
- @Override
- public Text createKey() {
- return new Text();
- }
-
- @Override
- public PeekingIterator<Entry<Key,Value>> createValue() {
- return new PeekingIterator<>();
- }
- };
- recordReader.initialize(split, job);
- return recordReader;
- }
-}
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
deleted file mode 100644
index 597c38b..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ /dev/null
@@ -1,387 +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
- *
- * https://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.mapred;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchScanner;
-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.ScannerBase;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of
- * K,V pairs.
- * <p>
- * Subclasses must implement a {@link #getRecordReader(InputSplit, JobConf, Reporter)} to provide a
- * {@link RecordReader} for K,V.
- * <p>
- * A static base class, RecordReaderBase, is provided to retrieve Accumulo {@link Key}/{@link Value}
- * pairs, but one must implement its {@link RecordReaderBase#next(Object, Object)} to transform them
- * to the desired generic types K,V.
- * <p>
- * See {@link AccumuloInputFormat} for an example implementation.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
-
- /**
- * Sets the name of the input table, over which this job will scan.
- *
- * @param job the Hadoop job instance to be configured
- * @param tableName the table to use when the tablename is null in the write call
- * @since 1.5.0
- */
- public static void setInputTableName(JobConf job, String tableName) {
- InputConfigurator.setInputTableName(CLASS, job, tableName);
- }
-
- /**
- * Gets the table name from the configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return the table name
- * @since 1.5.0
- * @see #setInputTableName(JobConf, String)
- */
- protected static String getInputTableName(JobConf job) {
- return InputConfigurator.getInputTableName(CLASS, job);
- }
-
- /**
- * Sets the input ranges to scan for this job. If not set, the entire table will be scanned.
- *
- * @param job the Hadoop job instance to be configured
- * @param ranges the ranges that will be mapped over
- * @see TableOperations#splitRangeByTablets(String, Range, int)
- * @since 1.5.0
- */
- public static void setRanges(JobConf job, Collection<Range> ranges) {
- InputConfigurator.setRanges(CLASS, job, ranges);
- }
-
- /**
- * Gets the ranges to scan over from a job.
- *
- * @param job the Hadoop context for the configured job
- * @return the ranges
- * @throws IOException if the ranges have been encoded improperly
- * @since 1.5.0
- * @see #setRanges(JobConf, Collection)
- */
- protected static List<Range> getRanges(JobConf job) throws IOException {
- return InputConfigurator.getRanges(CLASS, job);
- }
-
- /**
- * Restricts the columns that will be mapped over for this job.
- *
- * @param job the Hadoop job instance to be configured
- * @param columnFamilyColumnQualifierPairs 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.5.0
- */
- public static void fetchColumns(JobConf job,
- Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
- InputConfigurator.fetchColumns(CLASS, job, columnFamilyColumnQualifierPairs);
- }
-
- /**
- * Gets the columns to be mapped over from this job.
- *
- * @param job the Hadoop context for the configured job
- * @return a set of columns
- * @since 1.5.0
- * @see #fetchColumns(JobConf, Collection)
- */
- protected static Set<Pair<Text,Text>> getFetchedColumns(JobConf job) {
- return InputConfigurator.getFetchedColumns(CLASS, job);
- }
-
- /**
- * Encode an iterator on the input for this job.
- *
- * @param job the Hadoop job instance to be configured
- * @param cfg the configuration of the iterator
- * @since 1.5.0
- */
- public static void addIterator(JobConf job, IteratorSetting cfg) {
- InputConfigurator.addIterator(CLASS, job, cfg);
- }
-
- /**
- * Gets a list of the iterator settings (for iterators to apply to a scanner) from this
- * configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return a list of iterators
- * @since 1.5.0
- * @see #addIterator(JobConf, IteratorSetting)
- */
- protected static List<IteratorSetting> getIterators(JobConf job) {
- return InputConfigurator.getIterators(CLASS, job);
- }
-
- /**
- * 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 job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @see #setRanges(JobConf, Collection)
- * @since 1.5.0
- */
- public static void setAutoAdjustRanges(JobConf job, boolean enableFeature) {
- InputConfigurator.setAutoAdjustRanges(CLASS, job, enableFeature);
- }
-
- /**
- * Determines whether a configuration has auto-adjust ranges enabled. Must be enabled when
- * {@link #setBatchScan(JobConf, boolean)} is true.
- *
- * @param job the Hadoop context for the configured job
- * @return false if the feature is disabled, true otherwise
- * @since 1.5.0
- * @see #setAutoAdjustRanges(JobConf, boolean)
- */
- protected static boolean getAutoAdjustRanges(JobConf job) {
- return InputConfigurator.getAutoAdjustRanges(CLASS, job);
- }
-
- /**
- * Controls the use of the {@link IsolatedScanner} in this job.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.5.0
- */
- public static void setScanIsolation(JobConf job, boolean enableFeature) {
- InputConfigurator.setScanIsolation(CLASS, job, enableFeature);
- }
-
- /**
- * Determines whether a configuration has isolation enabled.
- *
- * @param job the Hadoop context for the configured job
- * @return true if the feature is enabled, false otherwise
- * @since 1.5.0
- * @see #setScanIsolation(JobConf, boolean)
- */
- protected static boolean isIsolated(JobConf job) {
- return InputConfigurator.isIsolated(CLASS, job);
- }
-
- /**
- * 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.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.5.0
- */
- public static void setLocalIterators(JobConf job, boolean enableFeature) {
- InputConfigurator.setLocalIterators(CLASS, job, enableFeature);
- }
-
- /**
- * Determines whether a configuration uses local iterators.
- *
- * @param job the Hadoop context for the configured job
- * @return true if the feature is enabled, false otherwise
- * @since 1.5.0
- * @see #setLocalIterators(JobConf, boolean)
- */
- protected static boolean usesLocalIterators(JobConf job) {
- return InputConfigurator.usesLocalIterators(CLASS, job);
- }
-
- /**
- * 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.
- *
- * <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 job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.5.0
- */
- public static void setOfflineTableScan(JobConf job, boolean enableFeature) {
- InputConfigurator.setOfflineTableScan(CLASS, job, enableFeature);
- }
-
- /**
- * Determines whether a configuration has the offline table scan feature enabled.
- *
- * @param job the Hadoop context for the configured job
- * @return true if the feature is enabled, false otherwise
- * @since 1.5.0
- * @see #setOfflineTableScan(JobConf, boolean)
- */
- protected static boolean isOfflineScan(JobConf job) {
- return InputConfigurator.isOfflineScan(CLASS, job);
- }
-
- /**
- * Controls the use of the {@link BatchScanner} in this job. Using this feature will group Ranges
- * by their source tablet, producing an InputSplit per tablet rather than per Range. This batching
- * helps to reduce overhead when querying a large number of small ranges. (ex: when doing
- * quad-tree decomposition for spatial queries)
- * <p>
- * In order to achieve good locality of InputSplits this option always clips the input Ranges to
- * tablet boundaries. This may result in one input Range contributing to several InputSplits.
- * <p>
- * Note: that the value of {@link #setAutoAdjustRanges(JobConf, boolean)} is ignored and is
- * assumed to be true when BatchScan option is enabled.
- * <p>
- * This configuration is incompatible with:
- * <ul>
- * <li>{@link #setOfflineTableScan(JobConf, boolean)}</li>
- * <li>{@link #setLocalIterators(JobConf, boolean)}</li>
- * <li>{@link #setScanIsolation(JobConf, boolean)}</li>
- * </ul>
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.7.0
- */
- public static void setBatchScan(JobConf job, boolean enableFeature) {
- InputConfigurator.setBatchScan(CLASS, job, enableFeature);
- }
-
- /**
- * Determines whether a configuration has the {@link BatchScanner} feature enabled.
- *
- * @param job the Hadoop context for the configured job
- * @since 1.7.0
- * @see #setBatchScan(JobConf, boolean)
- */
- public static boolean isBatchScan(JobConf job) {
- return InputConfigurator.isBatchScan(CLASS, job);
- }
-
- /**
- * Causes input format to read sample data. If sample data was created using a different
- * configuration or a tables sampler configuration changes while reading data, then the input
- * format will throw an error.
- *
- *
- * @param job the Hadoop job instance to be configured
- * @param samplerConfig The sampler configuration that sample must have been created with inorder
- * for reading sample data to succeed.
- *
- * @since 1.8.0
- * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
- */
- public static void setSamplerConfiguration(JobConf job, SamplerConfiguration samplerConfig) {
- InputConfigurator.setSamplerConfiguration(CLASS, job, samplerConfig);
- }
-
- protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
-
- @Override
- @Deprecated(since = "2.0.0")
- protected List<IteratorSetting> jobIterators(JobConf job, String tableName) {
- return getIterators(job);
- }
-
- /**
- * Apply the configured iterators to the scanner.
- *
- * @param iterators the iterators to set
- * @param scanner the scanner to configure
- * @deprecated since 1.7.0; Use {@link #jobIterators} instead.
- */
- @Deprecated(since = "1.7.0")
- protected void setupIterators(List<IteratorSetting> iterators, Scanner scanner) {
- for (IteratorSetting iterator : iterators) {
- scanner.addScanIterator(iterator);
- }
- }
-
- /**
- * Apply the configured iterators from the configuration to the scanner.
- *
- * @param job the job configuration
- * @param scanner the scanner to configure
- */
- @Deprecated(since = "1.7.0")
- protected void setupIterators(JobConf job, Scanner scanner) {
- setupIterators(getIterators(job), scanner);
- }
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
deleted file mode 100644
index 3c1e749..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
+++ /dev/null
@@ -1,49 +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
- *
- * https://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.mapred;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.data.Range;
-import org.apache.hadoop.mapred.InputSplit;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
- justification = "Intended to share code between mapred and mapreduce")
-public class RangeInputSplit extends org.apache.accumulo.core.client.mapreduce.RangeInputSplit
- implements InputSplit {
-
- public RangeInputSplit() {}
-
- public RangeInputSplit(RangeInputSplit split) throws IOException {
- super(split);
- }
-
- protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
- super(table, tableId, range, locations);
- }
-}
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
deleted file mode 100644
index 4324713..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ /dev/null
@@ -1,783 +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
- *
- * https://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 org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.SecureRandom;
-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.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-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.ScannerBase;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.OfflineScanner;
-import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-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.hadoop.security.token.Token;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * An abstract input format to provide shared methods common to all other input format classes. At
- * the very least, any classes inheriting from this class will need to define their own
- * {@link RecordReader}.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
-
- private static final SecureRandom random = new SecureRandom();
-
- protected static final Class<?> CLASS = AccumuloInputFormat.class;
- protected static final Logger log = Logger.getLogger(CLASS);
-
- /**
- * Sets the name of the classloader context on this scanner
- *
- * @param job the Hadoop job instance to be configured
- * @param context name of the classloader context
- * @since 1.8.0
- */
- public static void setClassLoaderContext(Job job, String context) {
- InputConfigurator.setClassLoaderContext(CLASS, job.getConfiguration(), context);
- }
-
- /**
- * Returns the name of the current classloader context set on this scanner
- *
- * @param job the Hadoop job instance to be configured
- * @return name of the current context
- * @since 1.8.0
- */
- public static String getClassLoaderContext(JobContext job) {
- return InputConfigurator.getClassLoaderContext(CLASS, job.getConfiguration());
- }
-
- /**
- * Sets the connector information needed to communicate with Accumulo in this job.
- *
- * <p>
- * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the
- * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
- * encoded to provide a charset safe conversion to a string, but this conversion is not intended
- * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
- * {@link DelegationToken}s, acquired using
- * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
- * concern.
- *
- * @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
- * @since 1.5.0
- */
- public static void setConnectorInfo(Job job, String principal, AuthenticationToken token)
- throws AccumuloSecurityException {
- if (token instanceof KerberosToken) {
- log.info("Received KerberosToken, attempting to fetch DelegationToken");
- try {
- ClientContext client = InputConfigurator.client(CLASS, job.getConfiguration());
- token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
- } catch (Exception e) {
- log.warn("Failed to automatically obtain DelegationToken, "
- + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
- }
- }
- // DelegationTokens can be passed securely from user to task without serializing insecurely in
- // the configuration
- if (token instanceof DelegationTokenImpl) {
- DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
- // Convert it into a Hadoop Token
- AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
- Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
- delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
- // Add the Hadoop Token to the Job so it gets serialized and passed along.
- job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
- }
-
- 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
- * @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 job 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 job) {
- return InputConfigurator.isConnectorInfoSet(CLASS, job.getConfiguration());
- }
-
- /**
- * Gets the user name from the configuration.
- *
- * @param job 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 job) {
- return InputConfigurator.getPrincipal(CLASS, job.getConfiguration());
- }
-
- /**
- * 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(since = "1.6.0")
- protected static String getTokenClass(JobContext job) {
- return getAuthenticationToken(job).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(since = "1.6.0")
- protected static byte[] getToken(JobContext job) {
- return AuthenticationToken.AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
- }
-
- /**
- * Gets the authenticated token from either the specified token file or directly from the
- * configuration, whichever was used when the job was configured.
- *
- * @param job 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 job) {
- AuthenticationToken token =
- InputConfigurator.getAuthenticationToken(CLASS, job.getConfiguration());
- return InputConfigurator.unwrapAuthenticationToken(job, token);
- }
-
- /**
- * 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
- * @deprecated since 1.6.0
- */
- @Deprecated(since = "1.6.0")
- public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
- setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
- .withInstance(instanceName).withZkHosts(zooKeepers));
- }
-
- /**
- * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
- *
- * @param job the Hadoop job instance to be configured
- *
- * @param clientConfig client configuration containing connection options
- * @since 1.6.0
- */
- public static void setZooKeeperInstance(Job job,
- org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
- InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
- }
-
- /**
- * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
- * configuration.
- *
- * @param job the Hadoop context for the configured job
- * @return an Accumulo instance
- * @since 1.5.0
- */
- protected static org.apache.accumulo.core.client.Instance getInstance(JobContext job) {
- return InputConfigurator.getInstance(CLASS, job.getConfiguration());
- }
-
- /**
- * 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 job the Hadoop context for the configured job
- * @return the log level
- * @since 1.5.0
- * @see #setLogLevel(Job, Level)
- */
- protected static Level getLogLevel(JobContext job) {
- return InputConfigurator.getLogLevel(CLASS, job.getConfiguration());
- }
-
- /**
- * 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 job 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 job) {
- return InputConfigurator.getScanAuthorizations(CLASS, job.getConfiguration());
- }
-
- /**
- * Fetches all {@link InputTableConfig}s that have been set on the given job.
- *
- * @param job the Hadoop job instance to be configured
- * @return the {@link InputTableConfig} objects for the job
- * @since 1.6.0
- */
- protected static Map<String,InputTableConfig> getInputTableConfigs(JobContext job) {
- return InputConfigurator.getInputTableConfigs(CLASS, job.getConfiguration());
- }
-
- /**
- * Fetches a {@link InputTableConfig} 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 InputTableConfig} for the given table
- * @since 1.6.0
- */
- protected static InputTableConfig getInputTableConfig(JobContext job, String tableName) {
- return InputConfigurator.getInputTableConfig(CLASS, job.getConfiguration(), 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}.
- *
- * @param job 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 job) throws IOException {
- InputConfigurator.validatePermissions(CLASS, job.getConfiguration());
- }
-
- /**
- * Construct the {@link org.apache.accumulo.core.client.ClientConfiguration} given the provided
- * context.
- *
- * @param job The Job
- * @return The ClientConfiguration
- * @since 1.7.0
- */
- protected static org.apache.accumulo.core.client.ClientConfiguration
- getClientConfiguration(JobContext job) {
- return InputConfigurator.getClientConfiguration(CLASS, job.getConfiguration());
- }
-
- /**
- * An abstract base class to be used to create {@link org.apache.hadoop.mapreduce.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 AbstractRecordReader<K,V> extends RecordReader<K,V> {
- protected long numKeysRead;
- protected Iterator<Map.Entry<Key,Value>> scannerIterator;
- protected ScannerBase scannerBase;
- protected RangeInputSplit split;
-
- /**
- * Extracts Iterators settings from the context to be used by RecordReader.
- *
- * @param context the Hadoop context for the configured job
- * @param tableName the table name for which the scanner is configured
- * @return List of iterator settings for given table
- * @since 1.7.0
- */
- protected abstract List<IteratorSetting> contextIterators(TaskAttemptContext context,
- String tableName);
-
- /**
- * Configures the iterators on a scanner for the given table name. Will attempt to use
- * configuration from the InputSplit, on failure will try to extract them from
- * TaskAttemptContext.
- *
- * @param context the Hadoop context for the configured job
- * @param tableName the table name for which the scanner is configured
- * @param scanner the scanner for which to configure the iterators
- * @param split InputSplit containing configurations
- * @since 1.7.0
- */
- private void setupIterators(TaskAttemptContext context, ScannerBase scanner, String tableName,
- RangeInputSplit split) {
- List<IteratorSetting> iterators = null;
-
- if (split == null) {
- iterators = contextIterators(context, tableName);
- } else {
- iterators = split.getIterators();
- if (iterators == null) {
- iterators = contextIterators(context, tableName);
- }
- }
-
- for (IteratorSetting iterator : iterators) {
- scanner.addScanIterator(iterator);
- }
- }
-
- /**
- * 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
- * @since 1.6.0
- * @deprecated since 1.7.0; Use {@link #contextIterators} instead.
- */
- @Deprecated(since = "1.7.0")
- protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName,
- RangeInputSplit split) {
- setupIterators(context, (ScannerBase) scanner, tableName, split);
- }
-
- @Override
- public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
-
- split = (RangeInputSplit) inSplit;
- log.debug("Initializing input split: " + split);
-
- Authorizations authorizations = getScanAuthorizations(attempt);
- String classLoaderContext = getClassLoaderContext(attempt);
- String table = split.getTableName();
-
- // in case the table name changed, we can still use the previous name for terms of
- // configuration,
- // but the scanner will use the table id resolved at job setup time
- InputTableConfig tableConfig = getInputTableConfig(attempt, split.getTableName());
-
- ClientContext client = InputConfigurator.client(CLASS, split, attempt.getConfiguration());
-
- log.debug("Created client with user: " + client.whoami());
- log.debug("Creating scanner for table: " + table);
- log.debug("Authorizations are: " + authorizations);
-
- if (split instanceof org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit) {
- org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit batchSplit =
- (org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit) split;
-
- BatchScanner scanner;
- try {
- // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
- // will not span tablets
- int scanThreads = 1;
- scanner = client.createBatchScanner(split.getTableName(), authorizations, scanThreads);
- setupIterators(attempt, scanner, split.getTableName(), split);
- if (classLoaderContext != null) {
- scanner.setClassLoaderContext(classLoaderContext);
- }
- } catch (Exception e) {
- e.printStackTrace();
- throw new IOException(e);
- }
-
- scanner.setRanges(batchSplit.getRanges());
- scannerBase = scanner;
- } else {
- Scanner scanner;
-
- Boolean isOffline = split.isOffline();
- if (isOffline == null) {
- isOffline = tableConfig.isOfflineScan();
- }
-
- Boolean isIsolated = split.isIsolatedScan();
- if (isIsolated == null) {
- isIsolated = tableConfig.shouldUseIsolatedScanners();
- }
-
- Boolean usesLocalIterators = split.usesLocalIterators();
- if (usesLocalIterators == null) {
- usesLocalIterators = tableConfig.shouldUseLocalIterators();
- }
-
- try {
- if (isOffline) {
- scanner = new OfflineScanner(client, TableId.of(split.getTableId()), authorizations);
- } else {
- // Not using public API to create scanner so that we can use table ID
- // Table ID is used in case of renames during M/R job
- scanner = new ScannerImpl(client, TableId.of(split.getTableId()), authorizations);
- }
- if (isIsolated) {
- log.info("Creating isolated scanner");
- @SuppressWarnings("resource")
- var wrapped = new IsolatedScanner(scanner);
- scanner = wrapped;
- }
- if (usesLocalIterators) {
- log.info("Using local iterators");
- scanner = new ClientSideIteratorScanner(scanner);
- }
-
- setupIterators(attempt, scanner, split.getTableName(), split);
- } catch (Exception e) {
- throw new IOException(e);
- }
-
- scanner.setRange(split.getRange());
- scannerBase = scanner;
-
- }
-
- Collection<Pair<Text,Text>> columns = split.getFetchedColumns();
- if (columns == null) {
- columns = tableConfig.getFetchedColumns();
- }
-
- // setup a scanner within the bounds of this split
- for (Pair<Text,Text> c : columns) {
- if (c.getSecond() != null) {
- log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
- scannerBase.fetchColumn(c.getFirst(), c.getSecond());
- } else {
- log.debug("Fetching column family " + c.getFirst());
- scannerBase.fetchColumnFamily(c.getFirst());
- }
- }
-
- SamplerConfiguration samplerConfig = split.getSamplerConfiguration();
- if (samplerConfig == null) {
- samplerConfig = tableConfig.getSamplerConfiguration();
- }
-
- if (samplerConfig != null) {
- scannerBase.setSamplerConfiguration(samplerConfig);
- }
-
- scannerIterator = scannerBase.iterator();
- numKeysRead = 0;
- }
-
- @Override
- public void close() {
- if (scannerBase != null) {
- scannerBase.close();
- }
- }
-
- @Override
- public float getProgress() throws IOException {
- if (numKeysRead > 0 && currentKey == null) {
- return 1.0f;
- }
- return split.getProgress(currentKey);
- }
-
- /**
- * The Key that should be returned to the client
- */
- protected K currentK = null;
-
- /**
- * The Value that should be return to the client
- */
- protected V currentV = null;
-
- /**
- * The Key that is used to determine progress in the current InputSplit. It is not returned to
- * the client and is only used internally
- */
- protected Key currentKey = 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 job, TableId tableId,
- List<Range> ranges)
- throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
- return InputConfigurator.binOffline(tableId, ranges,
- InputConfigurator.client(CLASS, job.getConfiguration()));
- }
-
- /**
- * Gets the splits of the tables that have been set on the job by reading the metadata table for
- * the specified ranges.
- *
- * @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
- */
- @Override
- public List<InputSplit> getSplits(JobContext job) throws IOException {
- Level logLevel = getLogLevel(job);
- log.setLevel(logLevel);
- validateOptions(job);
-
- LinkedList<InputSplit> splits = new LinkedList<>();
- Map<String,InputTableConfig> tableConfigs = getInputTableConfigs(job);
-
- for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
-
- String tableName = tableConfigEntry.getKey();
- InputTableConfig tableConfig = tableConfigEntry.getValue();
-
- ClientContext client;
- try {
- client = InputConfigurator.client(CLASS, job.getConfiguration());
- } catch (AccumuloException | AccumuloSecurityException e) {
- throw new IOException(e);
- }
-
- TableId tableId;
- // resolve table name to id once, and use id from this point forward
- try {
- tableId = client.getTableId(tableName);
- } catch (TableNotFoundException e) {
- throw new IOException(e);
- }
-
- boolean batchScan = InputConfigurator.isBatchScan(CLASS, job.getConfiguration());
- boolean supportBatchScan = !(tableConfig.isOfflineScan()
- || tableConfig.shouldUseIsolatedScanners() || tableConfig.shouldUseLocalIterators());
- if (batchScan && !supportBatchScan) {
- throw new IllegalArgumentException("BatchScanner optimization not available for offline"
- + " scan, isolated, or local iterators");
- }
-
- boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
- if (batchScan && !autoAdjust) {
- throw new IllegalArgumentException(
- "AutoAdjustRanges must be enabled when using BatchScanner optimization");
- }
-
- List<Range> ranges =
- autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
- if (ranges.isEmpty()) {
- ranges = new ArrayList<>(1);
- ranges.add(new Range());
- }
-
- // get the metadata information for these ranges
- Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
- TabletLocator tl;
- try {
- if (tableConfig.isOfflineScan()) {
- binnedRanges = binOfflineTable(job, tableId, ranges);
- while (binnedRanges == null) {
- // Some tablets were still online, try again
- // sleep randomly between 100 and 200 ms
- sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
- binnedRanges = binOfflineTable(job, tableId, ranges);
- }
- } else {
- tl = TabletLocator.getLocator(client, tableId);
- // its possible that the cache could contain complete, but old information about a
- // tables tablets... so clear it
- tl.invalidateCache();
-
- while (!tl.binRanges(client, ranges, binnedRanges).isEmpty()) {
- client.requireNotDeleted(tableId);
- client.requireNotOffline(tableId, tableName);
- binnedRanges.clear();
- log.warn("Unable to locate bins for specified ranges. Retrying.");
- // sleep randomly between 100 and 200 ms
- sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
- tl.invalidateCache();
- }
- }
- } catch (Exception e) {
- throw new IOException(e);
- }
-
- // all of this code will add either range per each locations or split ranges and add
- // range-location split
- // Map from Range to Array of Locations, we only use this if we're don't split
- HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
- if (!autoAdjust) {
- splitsToAdd = new HashMap<>();
- }
-
- HashMap<String,String> hostNameCache = new HashMap<>();
- 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.getCanonicalHostName();
- hostNameCache.put(ip, location);
- }
- for (Map.Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
- Range ke = extentRanges.getKey().toDataRange();
- if (batchScan) {
- // group ranges by tablet to be read by a BatchScanner
- ArrayList<Range> clippedRanges = new ArrayList<>();
- for (Range r : extentRanges.getValue()) {
- clippedRanges.add(ke.clip(r));
- }
- org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit split =
- new org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit(tableName,
- tableId, clippedRanges, new String[] {location});
- org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
- logLevel);
-
- splits.add(split);
- } else {
- // not grouping by tablet
- for (Range r : extentRanges.getValue()) {
- if (autoAdjust) {
- // divide ranges into smaller ranges, based on the tablets
- RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
- ke.clip(r), new String[] {location});
- org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split,
- tableConfig, logLevel);
- split.setOffline(tableConfig.isOfflineScan());
- split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
- split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
- splits.add(split);
- } else {
- // don't divide ranges
- ArrayList<String> locations = splitsToAdd.get(r);
- if (locations == null) {
- locations = new ArrayList<>(1);
- }
- locations.add(location);
- splitsToAdd.put(r, locations);
- }
- }
- }
- }
- }
-
- if (!autoAdjust) {
- for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
- RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
- entry.getKey(), entry.getValue().toArray(new String[0]));
- org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
- logLevel);
- split.setOffline(tableConfig.isOfflineScan());
- split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
- split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-
- splits.add(split);
- }
- }
- }
- return splits;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
deleted file mode 100644
index a50beab..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
+++ /dev/null
@@ -1,172 +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
- *
- * https://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 java.io.IOException;
-
-import org.apache.accumulo.core.client.rfile.RFile;
-import org.apache.accumulo.core.client.rfile.RFileWriter;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to write output in the Accumulo data file format.<br>
- * Care should be taken to write only sorted data (sorted by {@link Key}), as this is an important
- * requirement of Accumulo data files.
- *
- * <p>
- * The output path to be created must be specified via
- * {@link AccumuloFileOutputFormat#setOutputPath(Job, Path)}. This is inherited from
- * {@link FileOutputFormat#setOutputPath(Job, Path)}. Other methods from {@link FileOutputFormat}
- * are not supported and may be ignored or cause failures. Using other Hadoop configuration options
- * that affect the behavior of the underlying files directly in the Job's configuration may work,
- * but are not directly supported at this time.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
-
- private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
- protected static final Logger log = Logger.getLogger(CLASS);
-
- /**
- * Sets the compression type to use for data blocks. Specifying a compression may require
- * additional libraries to be available to your Job.
- *
- * @param job the Hadoop job instance to be configured
- * @param compressionType one of "none", "gz", "bzip2", "lzo", "lz4", "snappy", or "zstd"
- * @since 1.5.0
- */
- public static void setCompressionType(Job job, String compressionType) {
- FileOutputConfigurator.setCompressionType(CLASS, job.getConfiguration(), compressionType);
- }
-
- /**
- * Sets the size for data blocks within each file.<br>
- * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as
- * a group.
- *
- * <p>
- * Making this value smaller may increase seek performance, but at the cost of increasing the size
- * of the indexes (which can also affect seek performance).
- *
- * @param job the Hadoop job instance to be configured
- * @param dataBlockSize the block size, in bytes
- * @since 1.5.0
- */
- public static void setDataBlockSize(Job job, long dataBlockSize) {
- FileOutputConfigurator.setDataBlockSize(CLASS, job.getConfiguration(), dataBlockSize);
- }
-
- /**
- * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by
- * the underlying file system.
- *
- * @param job the Hadoop job instance to be configured
- * @param fileBlockSize the block size, in bytes
- * @since 1.5.0
- */
- public static void setFileBlockSize(Job job, long fileBlockSize) {
- FileOutputConfigurator.setFileBlockSize(CLASS, job.getConfiguration(), fileBlockSize);
- }
-
- /**
- * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy
- * within the file, while larger blocks mean a more shallow index hierarchy within the file. This
- * can affect the performance of queries.
- *
- * @param job the Hadoop job instance to be configured
- * @param indexBlockSize the block size, in bytes
- * @since 1.5.0
- */
- public static void setIndexBlockSize(Job job, long indexBlockSize) {
- FileOutputConfigurator.setIndexBlockSize(CLASS, job.getConfiguration(), indexBlockSize);
- }
-
- /**
- * Sets the file system replication factor for the resulting file, overriding the file system
- * default.
- *
- * @param job the Hadoop job instance to be configured
- * @param replication the number of replicas for produced files
- * @since 1.5.0
- */
- public static void setReplication(Job job, int replication) {
- FileOutputConfigurator.setReplication(CLASS, job.getConfiguration(), replication);
- }
-
- /**
- * Specify a sampler to be used when writing out data. This will result in the output file having
- * sample data.
- *
- * @param job The Hadoop job instance to be configured
- * @param samplerConfig The configuration for creating sample data in the output file.
- * @since 1.8.0
- */
-
- public static void setSampler(Job job, SamplerConfiguration samplerConfig) {
- FileOutputConfigurator.setSampler(CLASS, job.getConfiguration(), samplerConfig);
- }
-
- @Override
- public RecordWriter<Key,Value> getRecordWriter(TaskAttemptContext context) throws IOException {
- // get the path of the temporary output file
- final Configuration conf = context.getConfiguration();
- final AccumuloConfiguration acuConf =
- FileOutputConfigurator.getAccumuloConfiguration(CLASS, context.getConfiguration());
-
- final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
- final Path file = this.getDefaultWorkFile(context, "." + extension);
- final int visCacheSize = FileOutputConfigurator.getVisibilityCacheSize(conf);
-
- return new RecordWriter<>() {
- RFileWriter out = null;
-
- @Override
- public void close(TaskAttemptContext context) throws IOException {
- if (out != null) {
- out.close();
- }
- }
-
- @Override
- public void write(Key key, Value value) throws IOException {
- if (out == null) {
- out = RFile.newWriter().to(file.toString()).withFileSystem(file.getFileSystem(conf))
- .withTableProperties(acuConf).withVisibilityCacheSize(visCacheSize).build();
- out.startDefaultLocalityGroup();
- }
- out.append(key, value);
- }
- };
- }
-}
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
deleted file mode 100644
index 127a94e..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ /dev/null
@@ -1,92 +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
- *
- * https://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 java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapreduce.InputFormat;
-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 org.apache.log4j.Level;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link 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 AccumuloInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloInputFormat#setInputTableName(Job, String)}
- * <li>{@link AccumuloInputFormat#setScanAuthorizations(Job, Authorizations)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
-
- @Override
- public RecordReader<Key,Value> createRecordReader(InputSplit split, TaskAttemptContext context)
- throws IOException, InterruptedException {
- log.setLevel(getLogLevel(context));
-
- // Override the log level from the configuration as if the InputSplit has one it's the more
- // correct one to use.
- if (split instanceof org.apache.accumulo.core.client.mapreduce.RangeInputSplit) {
- org.apache.accumulo.core.client.mapreduce.RangeInputSplit accSplit =
- (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
- Level level = accSplit.getLogLevel();
- if (level != null) {
- log.setLevel(level);
- }
- } else {
- throw new IllegalArgumentException("No RecordReader for " + split.getClass());
- }
-
- return new RecordReaderBase<>() {
- @Override
- public boolean nextKeyValue() throws IOException, InterruptedException {
- if (scannerIterator.hasNext()) {
- ++numKeysRead;
- Entry<Key,Value> entry = scannerIterator.next();
- currentK = currentKey = entry.getKey();
- currentV = entry.getValue();
- if (log.isTraceEnabled()) {
- log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
- }
- return true;
- }
- return false;
- }
- };
- }
-}
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
deleted file mode 100644
index b3020e5..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ /dev/null
@@ -1,100 +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
- *
- * https://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 java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-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;
-
-/**
- * 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, AuthenticationToken)}
- * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloMultiTableInputFormat#setScanAuthorizations(Job, Authorizations)}
- * <li>{@link AccumuloMultiTableInputFormat#setInputTableConfigs(Job, Map)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value> {
-
- /**
- * Sets the {@link InputTableConfig} 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 setInputTableConfigs(Job job, Map<String,InputTableConfig> configs) {
- requireNonNull(configs);
- org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator.setInputTableConfigs(CLASS,
- job.getConfiguration(), configs);
- }
-
- @Override
- public RecordReader<Key,Value> createRecordReader(InputSplit inputSplit,
- TaskAttemptContext context) throws IOException, InterruptedException {
- log.setLevel(getLogLevel(context));
- return new AbstractRecordReader<>() {
- @Override
- public boolean nextKeyValue() throws IOException, InterruptedException {
- if (scannerIterator.hasNext()) {
- ++numKeysRead;
- Map.Entry<Key,Value> entry = scannerIterator.next();
- currentK = currentKey = entry.getKey();
- currentV = entry.getValue();
- if (log.isTraceEnabled()) {
- log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
- }
- return true;
- }
- return false;
- }
-
- @Override
- protected List<IteratorSetting> contextIterators(TaskAttemptContext context,
- String tableName) {
- return getInputTableConfig(context, tableName).getIterators();
- }
- };
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
deleted file mode 100644
index c39c101..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ /dev/null
@@ -1,581 +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
- *
- * https://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 java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.security.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.security.token.Token;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the sink for data. This {@link OutputFormat}
- * accepts keys and values of type {@link Text} (for a table name) and {@link Mutation} from the Map
- * and Reduce functions.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(Job, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
-
- private static final Class<?> CLASS = AccumuloOutputFormat.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> Some tokens, when serialized, divulge sensitive information in the
- * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
- * encoded to provide a charset safe conversion to a string, but this conversion is not intended
- * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
- * {@link DelegationToken}s, acquired using
- * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
- * concern.
- *
- * @param job the Hadoop job instance to be configured
- * @param principal a valid Accumulo user name (user must have Table.CREATE permission if
- * {@link #setCreateTables(Job, boolean)} is set to true)
- * @param token the user's password
- * @since 1.5.0
- */
- public static void setConnectorInfo(Job job, String principal, AuthenticationToken token)
- throws AccumuloSecurityException {
- if (token instanceof KerberosToken) {
- log.info("Received KerberosToken, attempting to fetch DelegationToken");
- try {
- ClientContext client = OutputConfigurator.client(CLASS, job.getConfiguration());
- token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
- } catch (Exception e) {
- log.warn("Failed to automatically obtain DelegationToken, "
- + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
- }
- }
- // DelegationTokens can be passed securely from user to task without serializing insecurely in
- // the configuration
- if (token instanceof DelegationTokenImpl) {
- DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
- // Convert it into a Hadoop Token
- AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
- Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
- delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
- // Add the Hadoop Token to the Job so it gets serialized and passed along.
- job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
- }
-
- OutputConfigurator.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 if
- * {@link #setCreateTables(Job, boolean)} is set to true)
- * @param tokenFile the path to the token file
- * @since 1.6.0
- */
- public static void setConnectorInfo(Job job, String principal, String tokenFile)
- throws AccumuloSecurityException {
- OutputConfigurator.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 OutputConfigurator.isConnectorInfoSet(CLASS, context.getConfiguration());
- }
-
- /**
- * 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 OutputConfigurator.getPrincipal(CLASS, context.getConfiguration());
- }
-
- /**
- * 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(since = "1.6.0")
- 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(since = "1.6.0")
- 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) {
- AuthenticationToken token =
- OutputConfigurator.getAuthenticationToken(CLASS, context.getConfiguration());
- return OutputConfigurator.unwrapAuthenticationToken(context, token);
- }
-
- /**
- * 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
- * @deprecated since 1.6.0; Use
- * {@link #setZooKeeperInstance(Job, org.apache.accumulo.core.client.ClientConfiguration)}
- * instead.
- */
- @Deprecated(since = "1.6.0")
- public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
- setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
- .withInstance(instanceName).withZkHosts(zooKeepers));
- }
-
- /**
- * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
- *
- * @param job the Hadoop job instance to be configured
- *
- * @param clientConfig client configuration for specifying connection timeouts, SSL connection
- * options, etc.
- * @since 1.6.0
- */
- public static void setZooKeeperInstance(Job job,
- org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
- OutputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
- }
-
- /**
- * 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
- */
- protected static org.apache.accumulo.core.client.Instance getInstance(JobContext context) {
- return OutputConfigurator.getInstance(CLASS, context.getConfiguration());
- }
-
- /**
- * 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) {
- OutputConfigurator.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 OutputConfigurator.getLogLevel(CLASS, context.getConfiguration());
- }
-
- /**
- * Sets the default table name to use if one emits a null in place of a table name for a given
- * mutation. Table names can only be alpha-numeric and underscores.
- *
- * @param job the Hadoop job instance to be configured
- * @param tableName the table to use when the tablename is null in the write call
- * @since 1.5.0
- */
- public static void setDefaultTableName(Job job, String tableName) {
- OutputConfigurator.setDefaultTableName(CLASS, job.getConfiguration(), tableName);
- }
-
- /**
- * Gets the default table name from the configuration.
- *
- * @param context the Hadoop context for the configured job
- * @return the default table name
- * @since 1.5.0
- * @see #setDefaultTableName(Job, String)
- */
- protected static String getDefaultTableName(JobContext context) {
- return OutputConfigurator.getDefaultTableName(CLASS, context.getConfiguration());
- }
-
- /**
- * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new
- * {@link BatchWriterConfig}, with sensible built-in defaults is used. Setting the configuration
- * multiple times overwrites any previous configuration.
- *
- * @param job the Hadoop job instance to be configured
- * @param bwConfig the configuration for the {@link BatchWriter}
- * @since 1.5.0
- */
- public static void setBatchWriterOptions(Job job, BatchWriterConfig bwConfig) {
- OutputConfigurator.setBatchWriterOptions(CLASS, job.getConfiguration(), bwConfig);
- }
-
- /**
- * Gets the {@link BatchWriterConfig} settings.
- *
- * @param context the Hadoop context for the configured job
- * @return the configuration object
- * @since 1.5.0
- * @see #setBatchWriterOptions(Job, BatchWriterConfig)
- */
- protected static BatchWriterConfig getBatchWriterOptions(JobContext context) {
- return OutputConfigurator.getBatchWriterOptions(CLASS, context.getConfiguration());
- }
-
- /**
- * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric
- * and underscores.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.5.0
- */
- public static void setCreateTables(Job job, boolean enableFeature) {
- OutputConfigurator.setCreateTables(CLASS, job.getConfiguration(), enableFeature);
- }
-
- /**
- * Determines whether tables are permitted to be created as needed.
- *
- * @param context the Hadoop context for the configured job
- * @return true if the feature is disabled, false otherwise
- * @since 1.5.0
- * @see #setCreateTables(Job, boolean)
- */
- protected static Boolean canCreateTables(JobContext context) {
- return OutputConfigurator.canCreateTables(CLASS, context.getConfiguration());
- }
-
- /**
- * Sets the directive to use simulation mode for this job. In simulation mode, no output is
- * produced. This is useful for testing.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.5.0
- */
- public static void setSimulationMode(Job job, boolean enableFeature) {
- OutputConfigurator.setSimulationMode(CLASS, job.getConfiguration(), enableFeature);
- }
-
- /**
- * Determines whether this feature is enabled.
- *
- * @param context the Hadoop context for the configured job
- * @return true if the feature is enabled, false otherwise
- * @since 1.5.0
- * @see #setSimulationMode(Job, boolean)
- */
- protected static Boolean getSimulationMode(JobContext context) {
- return OutputConfigurator.getSimulationMode(CLASS, context.getConfiguration());
- }
-
- /**
- * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
- */
- protected static class AccumuloRecordWriter extends RecordWriter<Text,Mutation> {
- private MultiTableBatchWriter mtbw = null;
- private HashMap<Text,BatchWriter> bws = null;
- private Text defaultTableName = null;
-
- private boolean simulate = false;
- private boolean createTables = false;
-
- private long mutCount = 0;
- private long valCount = 0;
-
- private AccumuloClient client;
-
- protected AccumuloRecordWriter(TaskAttemptContext context)
- throws AccumuloException, AccumuloSecurityException, IOException {
- Level l = getLogLevel(context);
- if (l != null) {
- log.setLevel(getLogLevel(context));
- }
- this.simulate = getSimulationMode(context);
- this.createTables = canCreateTables(context);
-
- if (simulate) {
- log.info("Simulating output only. No writes to tables will occur");
- }
-
- this.bws = new HashMap<>();
-
- String tname = getDefaultTableName(context);
- this.defaultTableName = (tname == null) ? null : new Text(tname);
-
- if (!simulate) {
- this.client = OutputConfigurator.client(CLASS, context.getConfiguration());
- mtbw = client.createMultiTableBatchWriter(getBatchWriterOptions(context));
- }
- }
-
- /**
- * Push a mutation into a table. If table is null, the defaultTable will be used. If
- * {@link AccumuloOutputFormat#canCreateTables(JobContext)} is set, the table will be created if
- * it does not exist. The table name must only contain alphanumerics and underscore.
- */
- @Override
- public void write(Text table, Mutation mutation) throws IOException {
- if (table == null || table.getLength() == 0) {
- table = this.defaultTableName;
- }
-
- if (!simulate && table == null) {
- throw new IOException("No table or default table specified. Try simulation mode next time");
- }
-
- ++mutCount;
- valCount += mutation.size();
- printMutation(table, mutation);
-
- if (simulate) {
- return;
- }
-
- if (!bws.containsKey(table)) {
- try {
- addTable(table);
- } catch (Exception e) {
- log.error("Could not add table '" + table + "'", e);
- throw new IOException(e);
- }
- }
-
- try {
- bws.get(table).addMutation(mutation);
- } catch (MutationsRejectedException e) {
- throw new IOException(e);
- }
- }
-
- public void addTable(Text tableName) throws AccumuloException, AccumuloSecurityException {
- if (simulate) {
- log.info("Simulating adding table: " + tableName);
- return;
- }
-
- log.debug("Adding table: " + tableName);
- BatchWriter bw = null;
- String table = tableName.toString();
-
- if (createTables && !client.tableOperations().exists(table)) {
- try {
- client.tableOperations().create(table);
- } catch (AccumuloSecurityException e) {
- log.error("Accumulo security violation creating " + table, e);
- throw e;
- } catch (TableExistsException e) {
- // Shouldn't happen
- }
- }
-
- try {
- bw = mtbw.getBatchWriter(table);
- } catch (TableNotFoundException e) {
- log.error("Accumulo table " + table + " doesn't exist and cannot be created.", e);
- throw new AccumuloException(e);
- } catch (AccumuloException | AccumuloSecurityException e) {
- throw e;
- }
-
- if (bw != null) {
- bws.put(tableName, bw);
- }
- }
-
- private int printMutation(Text table, Mutation m) {
- if (log.isTraceEnabled()) {
- log.trace(String.format("Table %s row key: %s", table, hexDump(m.getRow())));
- for (ColumnUpdate cu : m.getUpdates()) {
- log.trace(String.format("Table %s column: %s:%s", table, hexDump(cu.getColumnFamily()),
- hexDump(cu.getColumnQualifier())));
- log.trace(String.format("Table %s security: %s", table,
- new ColumnVisibility(cu.getColumnVisibility())));
- log.trace(String.format("Table %s value: %s", table, hexDump(cu.getValue())));
- }
- }
- return m.getUpdates().size();
- }
-
- private String hexDump(byte[] ba) {
- StringBuilder sb = new StringBuilder();
- for (byte b : ba) {
- if ((b > 0x20) && (b < 0x7e)) {
- sb.append((char) b);
- } else {
- sb.append(String.format("x%02x", b));
- }
- }
- return sb.toString();
- }
-
- @Override
- public void close(TaskAttemptContext attempt) throws IOException, InterruptedException {
- log.debug("mutations written: " + mutCount + ", values written: " + valCount);
- if (simulate) {
- return;
- }
-
- try {
- mtbw.close();
- } catch (MutationsRejectedException e) {
- if (!e.getSecurityErrorCodes().isEmpty()) {
- var tables = new HashMap<String,Set<SecurityErrorCode>>();
- e.getSecurityErrorCodes().forEach((table, code) -> tables
- .computeIfAbsent(table.getTable().canonical(), k -> new HashSet<>()).addAll(code));
- log.error("Not authorized to write to tables : " + tables);
- }
-
- if (!e.getConstraintViolationSummaries().isEmpty()) {
- log.error("Constraint violations : " + e.getConstraintViolationSummaries().size());
- }
- throw new IOException(e);
- } finally {
- client.close();
- }
- }
- }
-
- @Override
- public void checkOutputSpecs(JobContext job) throws IOException {
- if (!isConnectorInfoSet(job)) {
- throw new IOException("Connector info has not been set.");
- }
- String principal = getPrincipal(job);
- AuthenticationToken token = getAuthenticationToken(job);
- try {
- AccumuloClient c = OutputConfigurator.client(CLASS, job.getConfiguration());
- if (!c.securityOperations().authenticateUser(principal, token)) {
- throw new IOException("Unable to authenticate user");
- }
- } catch (AccumuloException | AccumuloSecurityException e) {
- throw new IOException(e);
- }
- }
-
- @Override
- public OutputCommitter getOutputCommitter(TaskAttemptContext context) {
- return new NullOutputFormat<Text,Mutation>().getOutputCommitter(context);
- }
-
- @Override
- public RecordWriter<Text,Mutation> getRecordWriter(TaskAttemptContext attempt)
- throws IOException {
- try {
- return new AccumuloRecordWriter(attempt);
- } catch (Exception e) {
- throw new IOException(e);
- }
- }
-
-}
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
deleted file mode 100644
index 0a97744..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
+++ /dev/null
@@ -1,88 +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
- *
- * https://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 java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.RowIterator;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat}
- * provides row names as {@link Text} as keys, and a corresponding {@link PeekingIterator} as a
- * value, which in turn makes the {@link Key}/{@link Value} pairs for that row available to the Map
- * function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloRowInputFormat#setInputTableName(Job, String)}
- * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(Job, Authorizations)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloRowInputFormat
- extends InputFormatBase<Text,PeekingIterator<Entry<Key,Value>>> {
- @Override
- public RecordReader<Text,PeekingIterator<Entry<Key,Value>>> createRecordReader(InputSplit split,
- TaskAttemptContext context) throws IOException, InterruptedException {
- log.setLevel(getLogLevel(context));
- return new RecordReaderBase<>() {
- RowIterator rowIterator;
-
- @Override
- public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
- super.initialize(inSplit, attempt);
- rowIterator = new RowIterator(scannerIterator);
- currentK = new Text();
- currentV = null;
- }
-
- @Override
- public boolean nextKeyValue() throws IOException, InterruptedException {
- if (!rowIterator.hasNext()) {
- return false;
- }
- currentV = new PeekingIterator<>(rowIterator.next());
- numKeysRead = rowIterator.getKVCount();
- currentKey = currentV.peek().getKey();
- currentK = new Text(currentKey.getRow());
- return true;
- }
- };
- }
-}
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
deleted file mode 100644
index f4e4023..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ /dev/null
@@ -1,386 +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
- *
- * https://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 java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchScanner;
-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.ScannerBase;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-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;
-
-/**
- * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of
- * K,V pairs.
- * <p>
- * Subclasses must implement a {@link #createRecordReader(InputSplit, TaskAttemptContext)} to
- * provide a {@link RecordReader} for K,V.
- * <p>
- * A static base class, RecordReaderBase, is provided to retrieve Accumulo {@link Key}/{@link Value}
- * pairs, but one must implement its {@link RecordReaderBase#nextKeyValue()} to transform them to
- * the desired generic types K,V.
- * <p>
- * See {@link AccumuloInputFormat} for an example implementation.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
-
- /**
- * Gets the table name from the configuration.
- *
- * @param context the Hadoop context for the configured job
- * @return the table name
- * @since 1.5.0
- * @see #setInputTableName(Job, String)
- */
- protected static String getInputTableName(JobContext context) {
- return InputConfigurator.getInputTableName(CLASS, context.getConfiguration());
- }
-
- /**
- * Sets the name of the input table, over which this job will scan.
- *
- * @param job the Hadoop job instance to be configured
- * @param tableName the table to use when the tablename is null in the write call
- * @since 1.5.0
- */
- public static void setInputTableName(Job job, String tableName) {
- InputConfigurator.setInputTableName(CLASS, job.getConfiguration(), tableName);
- }
-
- /**
- * Sets the input ranges to scan for the single input table associated with this job.
- *
- * @param job the Hadoop job instance to be configured
- * @param ranges the ranges that will be mapped over
- * @see TableOperations#splitRangeByTablets(String, Range, int)
- * @since 1.5.0
- */
- public static void setRanges(Job job, Collection<Range> ranges) {
- InputConfigurator.setRanges(CLASS, job.getConfiguration(), ranges);
- }
-
- /**
- * Gets the ranges to scan over from a job.
- *
- * @param context the Hadoop context for the configured job
- * @return the ranges
- * @since 1.5.0
- * @see #setRanges(Job, Collection)
- */
- protected static List<Range> getRanges(JobContext context) throws IOException {
- return InputConfigurator.getRanges(CLASS, context.getConfiguration());
- }
-
- /**
- * Restricts the columns that will be mapped over for this job for the default input table.
- *
- * @param job the Hadoop job instance to be configured
- * @param columnFamilyColumnQualifierPairs 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.5.0
- */
- public static void fetchColumns(Job job,
- Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
- InputConfigurator.fetchColumns(CLASS, job.getConfiguration(), columnFamilyColumnQualifierPairs);
- }
-
- /**
- * Gets the columns to be mapped over from this job.
- *
- * @param context the Hadoop context for the configured job
- * @return a set of columns
- * @since 1.5.0
- * @see #fetchColumns(Job, Collection)
- */
- protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext context) {
- return InputConfigurator.getFetchedColumns(CLASS, context.getConfiguration());
- }
-
- /**
- * Encode an iterator on the single input table for this job.
- *
- * @param job the Hadoop job instance to be configured
- * @param cfg the configuration of the iterator
- * @since 1.5.0
- */
- public static void addIterator(Job job, IteratorSetting cfg) {
- InputConfigurator.addIterator(CLASS, job.getConfiguration(), cfg);
- }
-
- /**
- * Gets a list of the iterator settings (for iterators to apply to a scanner) from this
- * configuration.
- *
- * @param context the Hadoop context for the configured job
- * @return a list of iterators
- * @since 1.5.0
- * @see #addIterator(Job, IteratorSetting)
- */
- protected static List<IteratorSetting> getIterators(JobContext context) {
- return InputConfigurator.getIterators(CLASS, context.getConfiguration());
- }
-
- /**
- * 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 job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @see #setRanges(Job, Collection)
- * @since 1.5.0
- */
- public static void setAutoAdjustRanges(Job job, boolean enableFeature) {
- InputConfigurator.setAutoAdjustRanges(CLASS, job.getConfiguration(), enableFeature);
- }
-
- /**
- * Determines whether a configuration has auto-adjust ranges enabled. Must be enabled when
- * {@link #setBatchScan(Job, boolean)} is true.
- *
- * @param context the Hadoop context for the configured job
- * @return false if the feature is disabled, true otherwise
- * @since 1.5.0
- * @see #setAutoAdjustRanges(Job, boolean)
- */
- protected static boolean getAutoAdjustRanges(JobContext context) {
- return InputConfigurator.getAutoAdjustRanges(CLASS, context.getConfiguration());
- }
-
- /**
- * Controls the use of the {@link IsolatedScanner} in this job.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.5.0
- */
- public static void setScanIsolation(Job job, boolean enableFeature) {
- InputConfigurator.setScanIsolation(CLASS, job.getConfiguration(), enableFeature);
- }
-
- /**
- * Determines whether a configuration has isolation enabled.
- *
- * @param context the Hadoop context for the configured job
- * @return true if the feature is enabled, false otherwise
- * @since 1.5.0
- * @see #setScanIsolation(Job, boolean)
- */
- protected static boolean isIsolated(JobContext context) {
- return InputConfigurator.isIsolated(CLASS, context.getConfiguration());
- }
-
- /**
- * 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.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.5.0
- */
- public static void setLocalIterators(Job job, boolean enableFeature) {
- InputConfigurator.setLocalIterators(CLASS, job.getConfiguration(), enableFeature);
- }
-
- /**
- * Determines whether a configuration uses local iterators.
- *
- * @param context the Hadoop context for the configured job
- * @return true if the feature is enabled, false otherwise
- * @since 1.5.0
- * @see #setLocalIterators(Job, boolean)
- */
- protected static boolean usesLocalIterators(JobContext context) {
- return InputConfigurator.usesLocalIterators(CLASS, context.getConfiguration());
- }
-
- /**
- * 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.
- *
- * <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 job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.5.0
- */
- public static void setOfflineTableScan(Job job, boolean enableFeature) {
- InputConfigurator.setOfflineTableScan(CLASS, job.getConfiguration(), enableFeature);
- }
-
- /**
- * Determines whether a configuration has the offline table scan feature enabled.
- *
- * @param context the Hadoop context for the configured job
- * @return true if the feature is enabled, false otherwise
- * @since 1.5.0
- * @see #setOfflineTableScan(Job, boolean)
- */
- protected static boolean isOfflineScan(JobContext context) {
- return InputConfigurator.isOfflineScan(CLASS, context.getConfiguration());
- }
-
- /**
- * Controls the use of the {@link BatchScanner} in this job. Using this feature will group Ranges
- * by their source tablet, producing an InputSplit per tablet rather than per Range. This batching
- * helps to reduce overhead when querying a large number of small ranges. (ex: when doing
- * quad-tree decomposition for spatial queries)
- * <p>
- * In order to achieve good locality of InputSplits this option always clips the input Ranges to
- * tablet boundaries. This may result in one input Range contributing to several InputSplits.
- * <p>
- * Note: that the value of {@link #setAutoAdjustRanges(Job, boolean)} is ignored and is assumed to
- * be true when BatchScan option is enabled.
- * <p>
- * This configuration is incompatible with:
- * <ul>
- * <li>{@link #setOfflineTableScan(org.apache.hadoop.mapreduce.Job, boolean)}</li>
- * <li>{@link #setLocalIterators(org.apache.hadoop.mapreduce.Job, boolean)}</li>
- * <li>{@link #setScanIsolation(org.apache.hadoop.mapreduce.Job, boolean)}</li>
- * </ul>
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @param job the Hadoop job instance to be configured
- * @param enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.7.0
- */
- public static void setBatchScan(Job job, boolean enableFeature) {
- InputConfigurator.setBatchScan(CLASS, job.getConfiguration(), enableFeature);
- }
-
- /**
- * Determines whether a configuration has the {@link BatchScanner} feature enabled.
- *
- * @param context the Hadoop context for the configured job
- * @since 1.7.0
- * @see #setBatchScan(Job, boolean)
- */
- public static boolean isBatchScan(JobContext context) {
- return InputConfigurator.isBatchScan(CLASS, context.getConfiguration());
- }
-
- /**
- * Causes input format to read sample data. If sample data was created using a different
- * configuration or a tables sampler configuration changes while reading data, then the input
- * format will throw an error.
- *
- *
- * @param job the Hadoop job instance to be configured
- * @param samplerConfig The sampler configuration that sample must have been created with inorder
- * for reading sample data to succeed.
- *
- * @since 1.8.0
- * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
- */
- public static void setSamplerConfiguration(Job job, SamplerConfiguration samplerConfig) {
- InputConfigurator.setSamplerConfiguration(CLASS, job.getConfiguration(), samplerConfig);
- }
-
- protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
-
- @Override
- @Deprecated(since = "2.0.0")
- protected List<IteratorSetting> contextIterators(TaskAttemptContext context, String tableName) {
- return getIterators(context);
- }
-
- /**
- * 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
- * @deprecated since 1.7.0; Use {@link #contextIterators} instead.
- */
- @Deprecated(since = "1.7.0")
- protected void setupIterators(TaskAttemptContext context, Scanner scanner) {
- // tableName is given as null as it will be ignored in eventual call to #contextIterators
- setupIterators(context, scanner, null, null);
- }
-
- /**
- * Initialize a scanner over the given input split using this task attempt configuration.
- *
- * @deprecated since 1.7.0; Use {@link #contextIterators} instead.
- */
- @Deprecated(since = "1.7.0")
- protected void setupIterators(TaskAttemptContext context, Scanner scanner,
- org.apache.accumulo.core.client.mapreduce.RangeInputSplit split) {
- setupIterators(context, scanner, null, split);
- }
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
deleted file mode 100644
index c48df18..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
+++ /dev/null
@@ -1,394 +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
- *
- * https://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 java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Objects;
-
-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.ScannerBase;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-/**
- * 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.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class InputTableConfig implements Writable {
-
- private List<IteratorSetting> iterators;
- private List<Range> ranges;
- private Collection<Pair<Text,Text>> columns;
-
- private boolean autoAdjustRanges = true;
- private boolean useLocalIterators = false;
- private boolean useIsolatedScanners = false;
- private boolean offlineScan = false;
- private SamplerConfiguration samplerConfig = null;
-
- public InputTableConfig() {}
-
- /**
- * 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
- */
- public InputTableConfig(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
- *
- * @param ranges the ranges that will be mapped over
- * @since 1.6.0
- */
- public InputTableConfig 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<>();
- }
-
- /**
- * 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 InputTableConfig fetchColumns(Collection<Pair<Text,Text>> columns) {
- this.columns = columns;
- return this;
- }
-
- /**
- * Returns the columns to be fetched for this configuration
- */
- public Collection<Pair<Text,Text>> getFetchedColumns() {
- return columns != null ? columns : new HashSet<>();
- }
-
- /**
- * Set iterators on to be used in the query.
- *
- * @param iterators the configurations for the iterators
- * @since 1.6.0
- */
- public InputTableConfig 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<>();
- }
-
- /**
- * 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 InputTableConfig 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 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 InputTableConfig 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)
- */
- public boolean shouldUseLocalIterators() {
- return useLocalIterators;
- }
-
- /**
- * 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.properties 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 InputTableConfig 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 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 InputTableConfig 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;
- }
-
- /**
- * Set the sampler configuration to use when reading from the data.
- *
- * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
- * @see InputFormatBase#setSamplerConfiguration(org.apache.hadoop.mapreduce.Job,
- * SamplerConfiguration)
- *
- * @since 1.8.0
- */
- public void setSamplerConfiguration(SamplerConfiguration samplerConfiguration) {
- this.samplerConfig = samplerConfiguration;
- }
-
- /**
- *
- * @since 1.8.0
- */
- public SamplerConfiguration getSamplerConfiguration() {
- return samplerConfig;
- }
-
- @Override
- public void write(DataOutput dataOutput) throws IOException {
- 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);
- dataOutput.writeBoolean(offlineScan);
- if (samplerConfig == null) {
- dataOutput.writeBoolean(false);
- } else {
- dataOutput.writeBoolean(true);
- new SamplerConfigurationImpl(samplerConfig).write(dataOutput);
- }
- }
-
- @Override
- public void readFields(DataInput dataInput) throws IOException {
- // load iterators
- long iterSize = dataInput.readInt();
- if (iterSize > 0) {
- iterators = new ArrayList<>();
- }
- for (int i = 0; i < iterSize; i++) {
- iterators.add(new IteratorSetting(dataInput));
- }
- // load ranges
- long rangeSize = dataInput.readInt();
- if (rangeSize > 0) {
- ranges = new ArrayList<>();
- }
- 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<>();
- }
- 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<>(colFam, null));
- } else if (numPairs == 2) {
- Text colQual = new Text();
- colQual.readFields(dataInput);
- columns.add(new Pair<>(colFam, colQual));
- }
- }
- autoAdjustRanges = dataInput.readBoolean();
- useLocalIterators = dataInput.readBoolean();
- useIsolatedScanners = dataInput.readBoolean();
- offlineScan = dataInput.readBoolean();
-
- if (dataInput.readBoolean()) {
- samplerConfig = new SamplerConfigurationImpl(dataInput).toSamplerConfiguration();
- }
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- InputTableConfig that = (InputTableConfig) o;
- return autoAdjustRanges == that.autoAdjustRanges && useLocalIterators == that.useLocalIterators
- && useIsolatedScanners == that.useIsolatedScanners && offlineScan == that.offlineScan
- && Objects.equals(iterators, that.iterators) && Objects.equals(ranges, that.ranges)
- && Objects.equals(columns, that.columns)
- && Objects.equals(samplerConfig, that.samplerConfig);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(iterators, ranges, columns, autoAdjustRanges, useLocalIterators,
- useIsolatedScanners, offlineScan, samplerConfig);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
deleted file mode 100644
index dcdf837..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ /dev/null
@@ -1,524 +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
- *
- * https://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 java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Base64;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.log4j.Level;
-
-/**
- * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class RangeInputSplit extends InputSplit implements Writable {
- private Range range;
- private String[] locations;
- private String tableId, tableName, instanceName, zooKeepers, principal;
- private org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource tokenSource;
- private String tokenFile;
- private AuthenticationToken token;
- private Boolean offline, isolatedScan, localIterators;
- private Authorizations auths;
- private Set<Pair<Text,Text>> fetchedColumns;
- private List<IteratorSetting> iterators;
- private SamplerConfiguration samplerConfig;
- private Level level;
-
- public RangeInputSplit() {
- range = new Range();
- locations = new String[0];
- tableName = "";
- tableId = "";
- }
-
- public RangeInputSplit(RangeInputSplit split) throws IOException {
- this.range = split.getRange();
- this.setLocations(split.getLocations());
- this.setTableName(split.getTableName());
- this.setTableId(split.getTableId());
- }
-
- protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
- this.range = range;
- setLocations(locations);
- this.tableName = table;
- this.tableId = tableId;
- }
-
- public Range getRange() {
- return range;
- }
-
- public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
- return org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.getProgress(start, end,
- position);
- }
-
- public float getProgress(Key currentKey) {
- if (currentKey == null) {
- return 0f;
- }
- if (range.contains(currentKey)) {
- if (range.getStartKey() != null && range.getEndKey() != null) {
- if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
- // just look at the row progress
- return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(),
- currentKey.getRowData());
- } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
- // just look at the column family progress
- return getProgress(range.getStartKey().getColumnFamilyData(),
- range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
- } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL)
- != 0) {
- // 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 {
- return org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.getRangeLength(range);
- }
-
- @Override
- public String[] getLocations() throws IOException {
- return Arrays.copyOf(locations, locations.length);
- }
-
- @Override
- public void readFields(DataInput in) throws IOException {
- range.readFields(in);
- tableName = in.readUTF();
- tableId = in.readUTF();
- int numLocs = in.readInt();
- locations = new String[numLocs];
- for (int i = 0; i < numLocs; ++i) {
- locations[i] = in.readUTF();
- }
-
- if (in.readBoolean()) {
- isolatedScan = in.readBoolean();
- }
-
- if (in.readBoolean()) {
- offline = in.readBoolean();
- }
-
- if (in.readBoolean()) {
- localIterators = in.readBoolean();
- }
-
- if (in.readBoolean()) {
- int numColumns = in.readInt();
- List<String> columns = new ArrayList<>(numColumns);
- for (int i = 0; i < numColumns; i++) {
- columns.add(in.readUTF());
- }
-
- fetchedColumns = org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator
- .deserializeFetchedColumns(columns);
- }
-
- if (in.readBoolean()) {
- String strAuths = in.readUTF();
- auths = new Authorizations(strAuths.getBytes(UTF_8));
- }
-
- if (in.readBoolean()) {
- principal = in.readUTF();
- }
-
- if (in.readBoolean()) {
- int ordinal = in.readInt();
- this.tokenSource =
- org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource
- .values()[ordinal];
-
- switch (this.tokenSource) {
- case INLINE:
- String tokenClass = in.readUTF();
- byte[] base64TokenBytes = in.readUTF().getBytes(UTF_8);
- byte[] tokenBytes = Base64.getDecoder().decode(base64TokenBytes);
-
- this.token = AuthenticationTokenSerializer.deserialize(tokenClass, tokenBytes);
- break;
-
- case FILE:
- this.tokenFile = in.readUTF();
-
- break;
- default:
- throw new IOException("Cannot parse unknown TokenSource ordinal");
- }
- }
-
- if (in.readBoolean()) {
- instanceName = in.readUTF();
- }
-
- if (in.readBoolean()) {
- zooKeepers = in.readUTF();
- }
-
- if (in.readBoolean()) {
- int numIterators = in.readInt();
- iterators = new ArrayList<>(numIterators);
- for (int i = 0; i < numIterators; i++) {
- iterators.add(new IteratorSetting(in));
- }
- }
-
- if (in.readBoolean()) {
- level = Level.toLevel(in.readInt());
- }
-
- if (in.readBoolean()) {
- samplerConfig = new SamplerConfigurationImpl(in).toSamplerConfiguration();
- }
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
- range.write(out);
- out.writeUTF(tableName);
- out.writeUTF(tableId);
- out.writeInt(locations.length);
- for (String location : locations) {
- out.writeUTF(location);
- }
-
- out.writeBoolean(isolatedScan != null);
- if (isolatedScan != null) {
- out.writeBoolean(isolatedScan);
- }
-
- out.writeBoolean(offline != null);
- if (offline != null) {
- out.writeBoolean(offline);
- }
-
- out.writeBoolean(localIterators != null);
- if (localIterators != null) {
- out.writeBoolean(localIterators);
- }
-
- out.writeBoolean(fetchedColumns != null);
- if (fetchedColumns != null) {
- String[] cols = org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator
- .serializeColumns(fetchedColumns);
- out.writeInt(cols.length);
- for (String col : cols) {
- out.writeUTF(col);
- }
- }
-
- out.writeBoolean(auths != null);
- if (auths != null) {
- out.writeUTF(auths.serialize());
- }
-
- out.writeBoolean(principal != null);
- if (principal != null) {
- out.writeUTF(principal);
- }
-
- out.writeBoolean(tokenSource != null);
- if (tokenSource != null) {
- out.writeInt(tokenSource.ordinal());
-
- if (token != null && tokenFile != null) {
- throw new IOException(
- "Cannot use both inline AuthenticationToken and file-based AuthenticationToken");
- } else if (token != null) {
- out.writeUTF(token.getClass().getName());
- out.writeUTF(
- Base64.getEncoder().encodeToString(AuthenticationTokenSerializer.serialize(token)));
- } else {
- out.writeUTF(tokenFile);
- }
- }
-
- out.writeBoolean(instanceName != null);
- if (instanceName != null) {
- out.writeUTF(instanceName);
- }
-
- out.writeBoolean(zooKeepers != null);
- if (zooKeepers != null) {
- out.writeUTF(zooKeepers);
- }
-
- out.writeBoolean(iterators != null);
- if (iterators != null) {
- out.writeInt(iterators.size());
- for (IteratorSetting iterator : iterators) {
- iterator.write(out);
- }
- }
-
- out.writeBoolean(level != null);
- if (level != null) {
- out.writeInt(level.toInt());
- }
-
- out.writeBoolean(samplerConfig != null);
- if (samplerConfig != null) {
- new SamplerConfigurationImpl(samplerConfig).write(out);
- }
- }
-
- /**
- * Use {@link #getTableName}
- *
- * @deprecated since 1.6.1, use getTableName() instead.
- */
- @Deprecated(since = "1.6.1")
- public String getTable() {
- return getTableName();
- }
-
- public String getTableName() {
- return tableName;
- }
-
- /**
- * Use {@link #setTableName}
- *
- * @deprecated since 1.6.1, use setTableName() instead.
- */
- @Deprecated(since = "1.6.1")
- public void setTable(String table) {
- setTableName(table);
- }
-
- public void setTableName(String table) {
- this.tableName = table;
- }
-
- public void setTableId(String tableId) {
- this.tableId = tableId;
- }
-
- public String getTableId() {
- return tableId;
- }
-
- /**
- * @see #getInstance(org.apache.accumulo.core.client.ClientConfiguration)
- * @deprecated since 1.7.0, use getInstance(ClientConfiguration) instead.
- */
- @Deprecated(since = "1.7.0")
- public org.apache.accumulo.core.client.Instance getInstance() {
- return getInstance(org.apache.accumulo.core.client.ClientConfiguration.loadDefault());
- }
-
- public org.apache.accumulo.core.client.Instance
- getInstance(org.apache.accumulo.core.client.ClientConfiguration base) {
- if (null == instanceName) {
- return null;
- }
-
- if (null == zooKeepers) {
- return null;
- }
-
- return new org.apache.accumulo.core.client.ZooKeeperInstance(
- base.withInstance(getInstanceName()).withZkHosts(getZooKeepers()));
- }
-
- public String getInstanceName() {
- return instanceName;
- }
-
- public void setInstanceName(String instanceName) {
- this.instanceName = instanceName;
- }
-
- public String getZooKeepers() {
- return zooKeepers;
- }
-
- public void setZooKeepers(String zooKeepers) {
- this.zooKeepers = zooKeepers;
- }
-
- public String getPrincipal() {
- return principal;
- }
-
- public void setPrincipal(String principal) {
- this.principal = principal;
- }
-
- public AuthenticationToken getToken() {
- return token;
- }
-
- public void setToken(AuthenticationToken token) {
- this.tokenSource =
- org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource.INLINE;
- this.token = token;
- }
-
- public void setToken(String tokenFile) {
- this.tokenSource =
- org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource.FILE;
- this.tokenFile = tokenFile;
- }
-
- public Boolean isOffline() {
- return offline;
- }
-
- public void setOffline(Boolean offline) {
- this.offline = offline;
- }
-
- public void setLocations(String[] locations) {
- this.locations = Arrays.copyOf(locations, locations.length);
- }
-
- public Boolean isIsolatedScan() {
- return isolatedScan;
- }
-
- public void setIsolatedScan(Boolean isolatedScan) {
- this.isolatedScan = isolatedScan;
- }
-
- public Authorizations getAuths() {
- return auths;
- }
-
- public void setAuths(Authorizations auths) {
- this.auths = auths;
- }
-
- public void setRange(Range range) {
- this.range = range;
- }
-
- public Boolean usesLocalIterators() {
- return localIterators;
- }
-
- public void setUsesLocalIterators(Boolean localIterators) {
- this.localIterators = localIterators;
- }
-
- public Set<Pair<Text,Text>> getFetchedColumns() {
- return fetchedColumns;
- }
-
- public void setFetchedColumns(Collection<Pair<Text,Text>> fetchedColumns) {
- this.fetchedColumns = new HashSet<>();
- for (Pair<Text,Text> columns : fetchedColumns) {
- this.fetchedColumns.add(columns);
- }
- }
-
- public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
- this.fetchedColumns = fetchedColumns;
- }
-
- public List<IteratorSetting> getIterators() {
- return iterators;
- }
-
- public void setIterators(List<IteratorSetting> iterators) {
- this.iterators = iterators;
- }
-
- public Level getLogLevel() {
- return level;
- }
-
- public void setLogLevel(Level level) {
- this.level = level;
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder(256);
- sb.append("Range: ").append(range);
- sb.append(" Locations: ").append(Arrays.asList(locations));
- sb.append(" Table: ").append(tableName);
- sb.append(" TableID: ").append(tableId);
- sb.append(" InstanceName: ").append(instanceName);
- sb.append(" zooKeepers: ").append(zooKeepers);
- sb.append(" principal: ").append(principal);
- sb.append(" tokenSource: ").append(tokenSource);
- sb.append(" authenticationToken: ").append(token);
- sb.append(" authenticationTokenFile: ").append(tokenFile);
- sb.append(" Authorizations: ").append(auths);
- sb.append(" offlineScan: ").append(offline);
- sb.append(" isolatedScan: ").append(isolatedScan);
- sb.append(" localIterators: ").append(localIterators);
- sb.append(" fetchColumns: ").append(fetchedColumns);
- sb.append(" iterators: ").append(iterators);
- sb.append(" logLevel: ").append(level);
- sb.append(" samplerConfig: ").append(samplerConfig);
- return sb.toString();
- }
-
- public void setSamplerConfiguration(SamplerConfiguration samplerConfiguration) {
- this.samplerConfig = samplerConfiguration;
- }
-
- public SamplerConfiguration getSamplerConfiguration() {
- return samplerConfig;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java
deleted file mode 100644
index fea37ae..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java
+++ /dev/null
@@ -1,67 +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
- *
- * https://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.lib.partition;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Partitioner;
-
-/**
- * Hadoop partitioner that uses ranges based on row keys, and optionally sub-bins based on hashing.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce.partition instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class KeyRangePartitioner extends Partitioner<Key,Writable> implements Configurable {
- private RangePartitioner rp = new RangePartitioner();
-
- @Override
- public int getPartition(Key key, Writable value, int numPartitions) {
- return rp.getPartition(key.getRow(), value, numPartitions);
- }
-
- @Override
- public Configuration getConf() {
- return rp.getConf();
- }
-
- @Override
- public void setConf(Configuration conf) {
- rp.setConf(conf);
- }
-
- /**
- * Sets the hdfs file name to use, containing a newline separated list of Base64 encoded split
- * points that represent ranges for partitioning
- */
- public static void setSplitFile(Job job, String file) {
- RangePartitioner.setSplitFile(job, file);
- }
-
- /**
- * Sets the number of random sub-bins per range
- */
- public static void setNumSubBins(Job job, int num) {
- RangePartitioner.setNumSubBins(job, num);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
deleted file mode 100644
index 91ccf3c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
+++ /dev/null
@@ -1,133 +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
- *
- * https://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.lib.partition;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Arrays;
-import java.util.Base64;
-import java.util.Scanner;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.DistributedCacheHelper;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Partitioner;
-
-/**
- * Hadoop partitioner that uses ranges, and optionally sub-bins based on hashing.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce.partition instead from the
- * accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class RangePartitioner extends Partitioner<Text,Writable> implements Configurable {
- private static final String PREFIX = RangePartitioner.class.getName();
- private static final String CUTFILE_KEY = PREFIX + ".cutFile";
- private static final String NUM_SUBBINS = PREFIX + ".subBins";
-
- private Configuration conf;
-
- @Override
- public int getPartition(Text key, Writable value, int numPartitions) {
- try {
- return findPartition(key, getCutPoints(), getNumSubBins());
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- int findPartition(Text key, Text[] array, int numSubBins) {
- // find the bin for the range, and guarantee it is positive
- int index = Arrays.binarySearch(array, key);
- index = index < 0 ? (index + 1) * -1 : index;
-
- // both conditions work with numSubBins == 1, but this check is to avoid
- // hashing, when we don't need to, for speed
- if (numSubBins < 2) {
- return index;
- }
- return (key.toString().hashCode() & Integer.MAX_VALUE) % numSubBins + index * numSubBins;
- }
-
- private int _numSubBins = 0;
-
- private synchronized int getNumSubBins() {
- if (_numSubBins < 1) {
- // get number of sub-bins and guarantee it is positive
- _numSubBins = Math.max(1, getConf().getInt(NUM_SUBBINS, 1));
- }
- return _numSubBins;
- }
-
- private Text[] cutPointArray = null;
-
- private synchronized Text[] getCutPoints() throws IOException {
- if (cutPointArray == null) {
- String cutFileName = conf.get(CUTFILE_KEY);
- TreeSet<Text> cutPoints = new TreeSet<>();
- try (
- InputStream inputStream =
- DistributedCacheHelper.openCachedFile(cutFileName, CUTFILE_KEY, conf);
- Scanner in = new Scanner(inputStream, UTF_8)) {
- while (in.hasNextLine()) {
- cutPoints.add(new Text(Base64.getDecoder().decode(in.nextLine())));
- }
- }
-
- cutPointArray = cutPoints.toArray(new Text[cutPoints.size()]);
-
- if (cutPointArray == null) {
- throw new IOException("Cutpoint array not properly created from file" + cutFileName);
- }
- }
- return cutPointArray;
- }
-
- @Override
- public Configuration getConf() {
- return conf;
- }
-
- @Override
- public void setConf(Configuration conf) {
- this.conf = conf;
- }
-
- /**
- * Sets the hdfs file name to use, containing a newline separated list of Base64 encoded split
- * points that represent ranges for partitioning
- */
- public static void setSplitFile(Job job, String file) {
- DistributedCacheHelper.addCacheFile(job, file, CUTFILE_KEY);
- job.getConfiguration().set(CUTFILE_KEY, file);
- }
-
- /**
- * Sets the number of random sub-bins per range
- */
- public static void setNumSubBins(Job job, int num) {
- job.getConfiguration().setInt(NUM_SUBBINS, num);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java
deleted file mode 100644
index 34b3052..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java
+++ /dev/null
@@ -1,40 +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
- *
- * https://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.replication;
-
-/**
- * The peer already exists
- */
-@Deprecated(since = "2.1.0")
-public class PeerExistsException extends Exception {
- private static final long serialVersionUID = 1L;
-
- public PeerExistsException(String peer) {
- this(peer, (String) null);
- }
-
- public PeerExistsException(String peer, String message) {
- super("Peer '" + peer + "' already exists"
- + (message == null || message.isEmpty() ? "" : message));
- }
-
- public PeerExistsException(String message, Throwable cause) {
- super(message, cause);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
deleted file mode 100644
index 4da5ec9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
+++ /dev/null
@@ -1,43 +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
- *
- * https://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.replication;
-
-/**
- * The peer already exists
- */
-@Deprecated(since = "2.1.0")
-public class PeerNotFoundException extends Exception {
- private static final long serialVersionUID = 1L;
-
- public PeerNotFoundException(String peer) {
- this(peer, (String) null);
- }
-
- public PeerNotFoundException(String peer, String message) {
- super("Peer '" + peer + "' not found " + (message == null || message.isEmpty() ? "" : message));
- }
-
- public PeerNotFoundException(String message, Throwable cause) {
- super(message, cause);
- }
-
- public PeerNotFoundException(String peer, String message, Throwable cause) {
- super("Peer '" + peer + "' not found " + message, cause);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
index 3b6d10a..64956dc 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
@@ -23,6 +23,7 @@
import java.util.Collection;
import java.util.Map;
import java.util.Map.Entry;
+import java.util.Objects;
import java.util.function.Predicate;
import org.apache.accumulo.core.client.Scanner;
@@ -36,7 +37,9 @@
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.RowRangeUtil;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
/**
@@ -76,6 +79,38 @@
* @return this
*/
ScannerFSOptions from(String... files);
+
+ /**
+ * Specify FencedPath files to read from. When multiple are specified the {@link Scanner}
+ * constructed will present a merged view.
+ *
+ * @param files one or more FencedPaths to read.
+ * @return this
+ *
+ * @since 3.1.0
+ */
+ ScannerFSOptions from(FencedPath... files);
+
+ /**
+ * @since 3.1.0
+ */
+ class FencedPath {
+ private final Path path;
+ private final Range fence;
+
+ public FencedPath(Path path, Range fence) {
+ this.path = Objects.requireNonNull(path);
+ this.fence = RowRangeUtil.requireKeyExtentDataRange(fence);
+ }
+
+ public Path getPath() {
+ return path;
+ }
+
+ public Range getFence() {
+ return fence;
+ }
+ }
}
/**
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
index 767ebc3..b62fc84 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
@@ -19,6 +19,7 @@
package org.apache.accumulo.core.client.rfile;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@@ -140,6 +141,11 @@
public long requestCount() {
return 0L;
}
+
+ @Override
+ public long evictionCount() {
+ return 0L;
+ }
};
}
@@ -211,10 +217,9 @@
blockCacheManager.start(BlockCacheConfiguration.forTabletServer(cc));
this.indexCache = blockCacheManager.getBlockCache(CacheType.INDEX);
this.dataCache = blockCacheManager.getBlockCache(CacheType.DATA);
- } catch (RuntimeException e) {
- throw e;
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (ReflectiveOperationException e) {
+ throw new IllegalArgumentException(
+ "Configuration does not contain loadable class for block cache manager factory", e);
}
}
if (indexCache == null) {
@@ -351,7 +356,7 @@
CachableBuilder cb =
new CachableBuilder().input(inputStream, "source-" + i).length(sources[i].getLength())
.conf(opts.in.getConf()).cacheProvider(cacheProvider).cryptoService(cryptoService);
- readers.add(new RFile.Reader(cb));
+ readers.add(RFile.getReader(cb, sources[i].getRange()));
}
if (getSamplerConfiguration() != null) {
@@ -389,14 +394,14 @@
iterator = IteratorConfigUtil.loadIterators(iterator, iteratorBuilder);
}
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
iterator.seek(getRange() == null ? EMPTY_RANGE : getRange(), families, !families.isEmpty());
return new IteratorAdapter(iterator);
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
}
@@ -407,14 +412,10 @@
source.getInputStream().close();
}
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
- try {
- if (this.blockCacheManager != null) {
- this.blockCacheManager.stop();
- }
- } catch (Exception e1) {
- throw new RuntimeException(e1);
+ if (this.blockCacheManager != null) {
+ this.blockCacheManager.stop();
}
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
index f4ab7e9..e63c795 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
@@ -38,32 +38,37 @@
class RFileScannerBuilder implements RFile.InputArguments, RFile.ScannerFSOptions {
static class InputArgs extends FSConfArgs {
- private Path[] paths;
+ private FencedPath[] rFiles;
private RFileSource[] sources;
InputArgs(String... files) {
- this.paths = new Path[files.length];
+ this.rFiles = new FencedPath[files.length];
for (int i = 0; i < files.length; i++) {
- this.paths[i] = new Path(files[i]);
+ this.rFiles[i] = new FencedPath(new Path(files[i]), new Range());
}
}
+ InputArgs(FencedPath... files) {
+ this.rFiles = files;
+ }
+
InputArgs(RFileSource... sources) {
this.sources = sources;
}
RFileSource[] getSources() throws IOException {
if (sources == null) {
- sources = new RFileSource[paths.length];
- for (int i = 0; i < paths.length; i++) {
- sources[i] = new RFileSource(getFileSystem().open(paths[i]),
- getFileSystem().getFileStatus(paths[i]).getLen());
+ sources = new RFileSource[rFiles.length];
+ for (int i = 0; i < rFiles.length; i++) {
+ final Path path = rFiles[i].getPath();
+ sources[i] = new RFileSource(getFileSystem().open(path),
+ getFileSystem().getFileStatus(path).getLen(), rFiles[i].getFence());
}
} else {
for (int i = 0; i < sources.length; i++) {
if (!(sources[i].getInputStream() instanceof FSDataInputStream)) {
sources[i] = new RFileSource(new FSDataInputStream(sources[i].getInputStream()),
- sources[i].getLength());
+ sources[i].getLength(), rFiles[i].getFence());
}
}
}
@@ -128,6 +133,13 @@
}
@Override
+ public ScannerFSOptions from(FencedPath... files) {
+ Objects.requireNonNull(files);
+ opts.in = new InputArgs(files);
+ return this;
+ }
+
+ @Override
public ScannerOptions withTableProperties(Iterable<Entry<String,String>> tableConfig) {
Objects.requireNonNull(tableConfig);
this.opts.tableConfig = new HashMap<>();
@@ -150,4 +162,5 @@
this.opts.bounds = range;
return this;
}
+
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java
index cb5d0bb..a77e2a4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java
@@ -19,6 +19,9 @@
package org.apache.accumulo.core.client.rfile;
import java.io.InputStream;
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.Range;
/**
* RFile metadata is stored at the end of the file. Inorder to read an RFile, its length must be
@@ -29,10 +32,19 @@
public class RFileSource {
private final InputStream in;
private final long len;
+ private final Range range;
public RFileSource(InputStream in, long len) {
- this.in = in;
+ this(in, len, new Range());
+ }
+
+ /**
+ * @since 3.1.0
+ */
+ public RFileSource(InputStream in, long len, Range range) {
+ this.in = Objects.requireNonNull(in);
this.len = len;
+ this.range = Objects.requireNonNull(range);
}
public InputStream getInputStream() {
@@ -42,4 +54,13 @@
public long getLength() {
return len;
}
+
+ /**
+ * @return The range of the RFileSource
+ *
+ * @since 3.1.0
+ */
+ public Range getRange() {
+ return range;
+ }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
index be1850c..b1d7957 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
@@ -38,6 +38,7 @@
import org.apache.accumulo.core.conf.DefaultConfiguration;
import org.apache.accumulo.core.crypto.CryptoFactoryLoader;
import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
import org.apache.accumulo.core.metadata.ValidationUtil;
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
@@ -119,7 +120,8 @@
visCacheSize);
} else {
return new RFileWriter(fileops.newWriterBuilder()
- .forFile(out.path.toString(), out.getFileSystem(), out.getConf(), cs)
+ .forFile(UnreferencedTabletFile.of(out.getFileSystem(), out.path), out.getFileSystem(),
+ out.getConf(), cs)
.withTableConfiguration(acuconf).withStartDisabled().build(), visCacheSize);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java b/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java
index 072f6ca..59895e3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java
@@ -23,6 +23,7 @@
import java.io.DataOutput;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.Map;
import java.util.Set;
@@ -85,17 +86,6 @@
}
/**
- * Subclasses with options should override this method and return true if the option is valid for
- * the subclass or if {@code super.isValidOption(opt)} returns true.
- *
- * @deprecated since 2.1.0, replaced by {@link #validateOptions(Map)}
- */
- @Deprecated(since = "2.1.0")
- protected boolean isValidOption(String option) {
- return REQUIRED_SAMPLER_OPTIONS.contains(option);
- }
-
- /**
* Subclasses with options should override this method and call {@code super.init(config)}.
*/
@SuppressFBWarnings(value = "UNSAFE_HASH_EQUALS",
@@ -142,7 +132,7 @@
try {
hash(new DataoutputHasher(hasher), k);
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
return hasher.hash().asInt() % modulus == 0;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
index 6e53023..0652a3b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
@@ -23,6 +23,7 @@
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
@@ -115,7 +116,7 @@
token.write(out);
return baos.toByteArray();
} catch (IOException e) {
- throw new RuntimeException("Bug found in serialization code", e);
+ throw new UncheckedIOException("Bug found in serialization code", e);
}
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
index 50f44c7..a28cc06 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
@@ -25,6 +25,7 @@
import java.io.DataOutput;
import java.io.File;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.Collections;
import java.util.Set;
@@ -107,8 +108,10 @@
clone.principal = principal;
clone.keytab = keytab == null ? keytab : keytab.getCanonicalFile();
return clone;
- } catch (CloneNotSupportedException | IOException e) {
- throw new RuntimeException(e);
+ } catch (CloneNotSupportedException e) {
+ throw new IllegalStateException(e);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
index 8390f0f..30e570c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
@@ -101,8 +101,8 @@
AtomicBoolean calledFirstReadInt = new AtomicBoolean(false);
DataInput wrapped = (DataInput) Proxy.newProxyInstance(DataInput.class.getClassLoader(),
arg0.getClass().getInterfaces(), (obj, method, args) -> {
- // wrap the original DataInput in order to return the integer that was read
- // and then not used, because it didn't match -2
+ // wrap the original DataInput in order to simulate replacing the integer that was
+ // previously read and then not used back into the input, after it didn't match -2
if (!calledFirstReadInt.get() && method.getName().equals("readInt")) {
calledFirstReadInt.set(true);
return version;
@@ -160,7 +160,7 @@
clone.password = Arrays.copyOf(password, password.length);
return clone;
} catch (CloneNotSupportedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java
index 5660f66..77a9244 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java
@@ -24,14 +24,13 @@
import java.util.Set;
import java.util.function.Consumer;
+import org.apache.accumulo.access.AccessExpression;
import org.apache.accumulo.core.client.admin.TableOperations;
import org.apache.accumulo.core.client.summary.CountingSummarizer;
import org.apache.accumulo.core.data.ArrayByteSequence;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.ColumnVisibility.Node;
/**
* Counts unique authorizations in column visibility labels. Leverages super class to defend against
@@ -82,7 +81,10 @@
if (vis.length() > 0) {
Set<ByteSequence> auths = cache.get(vis);
if (auths == null) {
- auths = findAuths(vis);
+ auths = new HashSet<>();
+ for (String auth : AccessExpression.of(vis.toArray()).getAuthorizations().asSet()) {
+ auths.add(new ArrayByteSequence(auth));
+ }
cache.put(new ArrayByteSequence(vis), auths);
}
@@ -91,33 +93,5 @@
}
}
}
-
- private Set<ByteSequence> findAuths(ByteSequence vis) {
- HashSet<ByteSequence> auths = new HashSet<>();
- byte[] expression = vis.toArray();
- Node root = new ColumnVisibility(expression).getParseTree();
-
- findAuths(root, expression, auths);
-
- return auths;
- }
-
- private void findAuths(Node node, byte[] expression, HashSet<ByteSequence> auths) {
- switch (node.getType()) {
- case AND:
- case OR:
- for (Node child : node.getChildren()) {
- findAuths(child, expression, auths);
- }
- break;
- case TERM:
- auths.add(node.getTerm(expression));
- break;
- case EMPTY:
- break;
- default:
- throw new IllegalArgumentException("Unknown node type " + node.getType());
- }
- }
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
index b7889c3..ca36f3d 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
@@ -30,7 +30,8 @@
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.dataImpl.TabletIdImpl;
import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
-import org.apache.accumulo.core.util.HostAndPort;
+
+import com.google.common.net.HostAndPort;
/**
* @since 1.6.0
@@ -102,6 +103,11 @@
}
@Override
+ public long getPausedCount() {
+ return tac.getTimesPaused();
+ }
+
+ @Override
public List<IteratorSetting> getIterators() {
ArrayList<IteratorSetting> ret = new ArrayList<>();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
index ba9160f..d125fd2 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
@@ -57,7 +57,7 @@
private Authorizations authorizations;
ActiveScanImpl(ClientContext context,
- org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan)
+ org.apache.accumulo.core.tabletscan.thrift.ActiveScan activeScan)
throws TableNotFoundException {
this.scanId = activeScan.scanId;
this.client = activeScan.client;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
index 3929c3e..2ccc440 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
@@ -19,8 +19,6 @@
package org.apache.accumulo.core.clientImpl;
import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
import java.util.Map;
import java.util.Properties;
import java.util.function.Predicate;
@@ -38,121 +36,74 @@
public class ClientConfConverter {
private static final Logger log = LoggerFactory.getLogger(ClientConfConverter.class);
- private static Map<String,String> confProps = new HashMap<>();
- private static Map<String,String> propsConf = new HashMap<>();
-
- @SuppressWarnings("deprecation")
- private static void init() {
- propsConf.put(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST
- .getKey());
- propsConf.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT
- .getKey());
- propsConf.put(ClientProperty.SSL_ENABLED.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_ENABLED
- .getKey());
- propsConf.put(ClientProperty.SSL_KEYSTORE_PATH.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_PATH
- .getKey());
- propsConf.put(ClientProperty.SSL_KEYSTORE_TYPE.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_TYPE
- .getKey());
- propsConf.put(ClientProperty.SSL_KEYSTORE_PASSWORD.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_PASSWORD
- .getKey());
- propsConf.put(ClientProperty.SSL_TRUSTSTORE_PATH.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_PATH
- .getKey());
- propsConf.put(ClientProperty.SSL_TRUSTSTORE_TYPE.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_TYPE
- .getKey());
- propsConf.put(ClientProperty.SSL_TRUSTSTORE_PASSWORD.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD
- .getKey());
- propsConf.put(ClientProperty.SSL_USE_JSSE.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_USE_JSSE.getKey());
- propsConf.put(ClientProperty.SASL_ENABLED.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SASL_ENABLED
- .getKey());
- propsConf.put(ClientProperty.SASL_QOP.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SASL_QOP.getKey());
- propsConf.put(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(),
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
- .getKey());
-
- for (Map.Entry<String,String> entry : propsConf.entrySet()) {
- confProps.put(entry.getValue(), entry.getKey());
- }
- }
+ private static final Map<String,String> accumuloConfToClientProps = new HashMap<>();
+ private static final Map<String,String> clientPropsToAccumuloConf = new HashMap<>();
static {
- init();
- }
+ // mapping of ClientProperty equivalents in AccumuloConfiguration
+ Map<ClientProperty,Property> conversions = new HashMap<>();
+ conversions.put(ClientProperty.INSTANCE_ZOOKEEPERS, Property.INSTANCE_ZK_HOST);
+ conversions.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, Property.INSTANCE_ZK_TIMEOUT);
- @SuppressWarnings("deprecation")
- public static org.apache.accumulo.core.client.ClientConfiguration
- toClientConf(Properties properties) {
- org.apache.accumulo.core.client.ClientConfiguration config =
- org.apache.accumulo.core.client.ClientConfiguration.create();
- for (Object keyObj : properties.keySet()) {
- String propKey = (String) keyObj;
- String val = properties.getProperty(propKey);
- String confKey = propsConf.get(propKey);
- if (confKey == null) {
- config.setProperty(propKey, val);
- } else {
- config.setProperty(confKey, val);
- }
- if (propKey.equals(ClientProperty.SSL_KEYSTORE_PATH.getKey())) {
- config.setProperty(
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
- "true");
- }
- }
- return config;
- }
+ conversions.put(ClientProperty.SASL_ENABLED, Property.INSTANCE_RPC_SASL_ENABLED);
+ conversions.put(ClientProperty.SASL_QOP, Property.RPC_SASL_QOP);
- @SuppressWarnings("deprecation")
- public static Properties
- toProperties(org.apache.accumulo.core.client.ClientConfiguration clientConf) {
- Properties props = new Properties();
- Iterator<String> clientConfIter = clientConf.getKeys();
- while (clientConfIter.hasNext()) {
- String confKey = clientConfIter.next();
- String val = clientConf.getString(confKey);
- String propKey = confProps.get(confKey);
- if (propKey == null) {
- if (!confKey.equals(
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH
- .getKey())) {
- props.setProperty(confKey, val);
- }
- } else {
- props.setProperty(propKey, val);
- }
+ conversions.put(ClientProperty.SSL_ENABLED, Property.INSTANCE_RPC_SSL_ENABLED);
+ conversions.put(ClientProperty.SSL_KEYSTORE_PASSWORD, Property.RPC_SSL_KEYSTORE_PASSWORD);
+ conversions.put(ClientProperty.SSL_KEYSTORE_PATH, Property.RPC_SSL_KEYSTORE_PATH);
+ conversions.put(ClientProperty.SSL_KEYSTORE_TYPE, Property.RPC_SSL_KEYSTORE_TYPE);
+ conversions.put(ClientProperty.SSL_TRUSTSTORE_PASSWORD, Property.RPC_SSL_TRUSTSTORE_PASSWORD);
+ conversions.put(ClientProperty.SSL_TRUSTSTORE_PATH, Property.RPC_SSL_TRUSTSTORE_PATH);
+ conversions.put(ClientProperty.SSL_TRUSTSTORE_TYPE, Property.RPC_SSL_TRUSTSTORE_TYPE);
+ conversions.put(ClientProperty.SSL_USE_JSSE, Property.RPC_USE_JSSE);
+
+ for (Map.Entry<ClientProperty,Property> entry : conversions.entrySet()) {
+ accumuloConfToClientProps.put(entry.getValue().getKey(), entry.getKey().getKey());
+ clientPropsToAccumuloConf.put(entry.getKey().getKey(), entry.getValue().getKey());
}
- return props;
}
public static Properties toProperties(AccumuloConfiguration config) {
- return toProperties(toClientConf(config));
+ final var propsExtractedFromConfig = new Properties();
+
+ // Extract kerberos primary from the config
+ final String serverPrincipal = config.get(Property.GENERAL_KERBEROS_PRINCIPAL);
+ if (serverPrincipal != null && !serverPrincipal.isEmpty()) {
+ var krbName = new KerberosName(serverPrincipal);
+ propsExtractedFromConfig.setProperty(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(),
+ krbName.getServiceName());
+ }
+
+ // Extract the remaining properties from the config
+ config.stream().filter(e -> accumuloConfToClientProps.keySet().contains(e.getKey()))
+ .forEach(e -> propsExtractedFromConfig.setProperty(e.getKey(), e.getValue()));
+
+ // For all the extracted properties, convert them to their ClientProperty names
+ final var convertedProps = new Properties();
+ propsExtractedFromConfig.forEach((k, v) -> {
+ String confKey = String.valueOf(k);
+ String val = String.valueOf(v);
+ String propKey = accumuloConfToClientProps.get(confKey);
+ convertedProps.setProperty(propKey == null ? confKey : propKey, val);
+ });
+ return convertedProps;
}
public static AccumuloConfiguration toAccumuloConf(Properties properties) {
- return toAccumuloConf(toClientConf(properties));
- }
-
- /**
- * A utility method for converting client configuration to a standard configuration object for use
- * internally.
- *
- * @param config the original config
- * @return the client configuration presented in the form of an {@link AccumuloConfiguration}
- */
- @SuppressWarnings("deprecation")
- public static AccumuloConfiguration
- toAccumuloConf(final org.apache.accumulo.core.client.ClientConfiguration config) {
+ final var convertedProps = new Properties();
+ for (String propKey : properties.stringPropertyNames()) {
+ String val = properties.getProperty(propKey);
+ String confKey = clientPropsToAccumuloConf.get(propKey);
+ if (propKey.equals(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey())) {
+ confKey = Property.GENERAL_KERBEROS_PRINCIPAL.getKey();
+ // Avoid providing a realm since we don't know what it is...
+ val += "/_HOST@" + SaslConnectionParams.getDefaultRealm();
+ }
+ convertedProps.setProperty(confKey == null ? propKey : confKey, val);
+ if (propKey.equals(ClientProperty.SSL_KEYSTORE_PATH.getKey())) {
+ convertedProps.setProperty(Property.INSTANCE_RPC_SSL_CLIENT_AUTH.getKey(), "true");
+ }
+ }
final AccumuloConfiguration defaults = DefaultConfiguration.getInstance();
@@ -160,7 +111,7 @@
@Override
public boolean isPropertySet(Property prop) {
- return config.containsKey(prop.getKey());
+ return convertedProps.containsKey(prop.getKey());
}
@Override
@@ -181,51 +132,15 @@
}
}
}
-
- if (config.containsKey(key)) {
- return config.getString(key);
- } else {
- // Reconstitute the server kerberos property from the client config
- if (property == Property.GENERAL_KERBEROS_PRINCIPAL) {
- if (config.containsKey(
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
- .getKey())) {
- // Avoid providing a realm since we don't know what it is...
- return config.getString(
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
- .getKey())
- + "/_HOST@" + SaslConnectionParams.getDefaultRealm();
- }
- }
- return defaults.get(property);
- }
+ return convertedProps.getProperty(key, defaults.get(property));
}
@Override
public void getProperties(Map<String,String> props, Predicate<String> filter) {
defaults.getProperties(props, filter);
-
- Iterator<String> keyIter = config.getKeys();
- while (keyIter.hasNext()) {
- String key = keyIter.next();
+ for (String key : convertedProps.stringPropertyNames()) {
if (filter.test(key)) {
- props.put(key, config.getString(key));
- }
- }
-
- // Two client props that don't exist on the server config. Client doesn't need to know about
- // the Kerberos instance from the principle, but servers do
- // Automatically reconstruct the server property when converting a client config.
- if (props.containsKey(
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
- .getKey())) {
- final String serverPrimary = props.remove(
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
- .getKey());
- if (filter.test(Property.GENERAL_KERBEROS_PRINCIPAL.getKey())) {
- // Use the _HOST expansion. It should be unnecessary in "client land".
- props.put(Property.GENERAL_KERBEROS_PRINCIPAL.getKey(),
- serverPrimary + "/_HOST@" + SaslConnectionParams.getDefaultRealm());
+ props.put(key, convertedProps.getProperty(key));
}
}
@@ -247,8 +162,8 @@
}
private org.apache.hadoop.conf.Configuration getHadoopConfiguration() {
- String credProviderPaths =
- config.getString(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
+ String credProviderPaths = convertedProps
+ .getProperty(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
if (credProviderPaths != null && !credProviderPaths.isEmpty()) {
org.apache.hadoop.conf.Configuration hConf = new org.apache.hadoop.conf.Configuration();
HadoopCredentialProvider.setPath(hConf, credProviderPaths);
@@ -259,40 +174,7 @@
return null;
}
+
};
}
-
- @SuppressWarnings("deprecation")
- public static org.apache.accumulo.core.client.ClientConfiguration
- toClientConf(AccumuloConfiguration conf) {
- org.apache.accumulo.core.client.ClientConfiguration clientConf =
- org.apache.accumulo.core.client.ClientConfiguration.create();
-
- // Servers will only have the full principal in their configuration -- parse the
- // primary and realm from it.
- final String serverPrincipal = conf.get(Property.GENERAL_KERBEROS_PRINCIPAL);
-
- final KerberosName krbName;
- if (serverPrincipal != null && !serverPrincipal.isEmpty()) {
- krbName = new KerberosName(serverPrincipal);
- clientConf.setProperty(
- org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY,
- krbName.getServiceName());
- }
-
- HashSet<String> clientKeys = new HashSet<>();
- for (org.apache.accumulo.core.client.ClientConfiguration.ClientProperty prop : org.apache.accumulo.core.client.ClientConfiguration.ClientProperty
- .values()) {
- clientKeys.add(prop.getKey());
- }
-
- String key;
- for (Map.Entry<String,String> entry : conf) {
- key = entry.getKey();
- if (clientKeys.contains(key)) {
- clientConf.setProperty(key, entry.getValue());
- }
- }
- return clientConf;
- }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 75b6394..eb77916 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -22,7 +22,6 @@
import static com.google.common.base.Suppliers.memoize;
import static com.google.common.base.Suppliers.memoizeWithExpiration;
import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
@@ -37,6 +36,7 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.Optional;
import java.util.Properties;
import java.util.UUID;
import java.util.concurrent.Callable;
@@ -76,12 +76,14 @@
import org.apache.accumulo.core.data.KeyValue;
import org.apache.accumulo.core.data.NamespaceId;
import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory;
import org.apache.accumulo.core.fate.zookeeper.ZooReader;
import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
import org.apache.accumulo.core.manager.state.tables.TableState;
import org.apache.accumulo.core.metadata.RootTable;
import org.apache.accumulo.core.metadata.schema.Ample;
@@ -148,8 +150,6 @@
private final TableOperationsImpl tableops;
private final NamespaceOperations namespaceops;
private InstanceOperations instanceops = null;
- @SuppressWarnings("deprecation")
- private org.apache.accumulo.core.client.admin.ReplicationOperations replicationops = null;
private final SingletonReservation singletonReservation;
private final ThreadPools clientThreadPools;
private ThreadPoolExecutor cleanupThreadPool;
@@ -406,11 +406,10 @@
try {
final var zLockPath = ServiceLock.path(root + "/" + addr);
ZcStat stat = new ZcStat();
- byte[] lockData = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
- if (lockData != null) {
- String[] fields = new String(lockData, UTF_8).split(",", 2);
- UUID uuid = UUID.fromString(fields[0]);
- String group = fields[1];
+ Optional<ServiceLockData> sld = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
+ if (sld.isPresent()) {
+ UUID uuid = sld.orElseThrow().getServerUUID(ThriftService.TABLET_SCAN);
+ String group = sld.orElseThrow().getGroup(ThriftService.TABLET_SCAN);
liveScanServers.put(addr, new Pair<>(uuid, group));
}
} catch (IllegalArgumentException e) {
@@ -510,35 +509,34 @@
*/
public List<String> getManagerLocations() {
ensureOpen();
- return getManagerLocations(zooCache, getInstanceID().canonical());
- }
-
- // available only for sharing code with old ZooKeeperInstance
- public static List<String> getManagerLocations(ZooCache zooCache, String instanceId) {
var zLockManagerPath =
- ServiceLock.path(Constants.ZROOT + "/" + instanceId + Constants.ZMANAGER_LOCK);
+ ServiceLock.path(Constants.ZROOT + "/" + getInstanceID() + Constants.ZMANAGER_LOCK);
OpTimer timer = null;
if (log.isTraceEnabled()) {
- log.trace("tid={} Looking up manager location in zookeeper.", Thread.currentThread().getId());
+ log.trace("tid={} Looking up manager location in zookeeper at {}.",
+ Thread.currentThread().getId(), zLockManagerPath);
timer = new OpTimer().start();
}
- byte[] loc = zooCache.getLockData(zLockManagerPath);
+ Optional<ServiceLockData> sld = zooCache.getLockData(zLockManagerPath);
+ String location = null;
+ if (sld.isPresent()) {
+ location = sld.orElseThrow().getAddressString(ThriftService.MANAGER);
+ }
if (timer != null) {
timer.stop();
log.trace("tid={} Found manager at {} in {}", Thread.currentThread().getId(),
- (loc == null ? "null" : new String(loc, UTF_8)),
- String.format("%.3f secs", timer.scale(SECONDS)));
+ (location == null ? "null" : location), String.format("%.3f secs", timer.scale(SECONDS)));
}
- if (loc == null) {
+ if (location == null) {
return Collections.emptyList();
}
- return Collections.singletonList(new String(loc, UTF_8));
+ return Collections.singletonList(location);
}
/**
@@ -549,37 +547,27 @@
public InstanceId getInstanceID() {
ensureOpen();
if (instanceId == null) {
+ // lookup by name
final String instanceName = info.getInstanceName();
- instanceId = getInstanceID(zooCache, instanceName);
- verifyInstanceId(zooCache, instanceId.canonical(), instanceName);
+ String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
+ byte[] data = zooCache.get(instanceNamePath);
+ if (data == null) {
+ throw new RuntimeException(
+ "Instance name " + instanceName + " does not exist in zookeeper. "
+ + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
+ }
+ String instanceIdString = new String(data, UTF_8);
+ // verify that the instanceId found via the instanceName actually exists as an instance
+ if (zooCache.get(Constants.ZROOT + "/" + instanceIdString) == null) {
+ throw new RuntimeException("Instance id " + instanceIdString
+ + (instanceName == null ? "" : " pointed to by the name " + instanceName)
+ + " does not exist in zookeeper");
+ }
+ instanceId = InstanceId.of(instanceIdString);
}
return instanceId;
}
- // available only for sharing code with old ZooKeeperInstance
- public static InstanceId getInstanceID(ZooCache zooCache, String instanceName) {
- requireNonNull(zooCache, "zooCache cannot be null");
- requireNonNull(instanceName, "instanceName cannot be null");
- String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
- byte[] data = zooCache.get(instanceNamePath);
- if (data == null) {
- throw new RuntimeException("Instance name " + instanceName + " does not exist in zookeeper. "
- + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
- }
- return InstanceId.of(new String(data, UTF_8));
- }
-
- // available only for sharing code with old ZooKeeperInstance
- public static void verifyInstanceId(ZooCache zooCache, String instanceId, String instanceName) {
- requireNonNull(zooCache, "zooCache cannot be null");
- requireNonNull(instanceId, "instanceId cannot be null");
- if (zooCache.get(Constants.ZROOT + "/" + instanceId) == null) {
- throw new RuntimeException("Instance id " + instanceId
- + (instanceName == null ? "" : " pointed to by the name " + instanceName)
- + " does not exist in zookeeper");
- }
- }
-
public String getZooKeeperRoot() {
ensureOpen();
return ZooUtil.getRoot(getInstanceID());
@@ -849,18 +837,6 @@
}
@Override
- @Deprecated
- public synchronized org.apache.accumulo.core.client.admin.ReplicationOperations
- replicationOperations() {
- ensureOpen();
- if (replicationops == null) {
- replicationops = new ReplicationOperationsImpl(this);
- }
-
- return replicationops;
- }
-
- @Override
public Properties properties() {
ensureOpen();
Properties result = new Properties();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java
index ad7827c..699fa49 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java
@@ -18,8 +18,6 @@
*/
package org.apache.accumulo.core.clientImpl;
-import java.io.IOException;
-
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.TableNotFoundException;
@@ -62,15 +60,14 @@
}
@Override
- public <T> T instantiate(String className, Class<T> base)
- throws ReflectiveOperationException, IOException {
+ public <T> T instantiate(String className, Class<T> base) throws ReflectiveOperationException {
return ClientServiceEnvironmentImpl.class.getClassLoader().loadClass(className).asSubclass(base)
.getDeclaredConstructor().newInstance();
}
@Override
public <T> T instantiate(TableId tableId, String className, Class<T> base)
- throws ReflectiveOperationException, IOException {
+ throws ReflectiveOperationException {
return instantiate(className, base);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/CompactionStrategyConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/CompactionStrategyConfigUtil.java
deleted file mode 100644
index f09adc2..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/CompactionStrategyConfigUtil.java
+++ /dev/null
@@ -1,62 +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
- *
- * https://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.clientImpl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.admin.CompactionConfig;
-import org.apache.accumulo.core.client.admin.CompactionStrategyConfig;
-
-@SuppressWarnings("removal")
-public class CompactionStrategyConfigUtil {
-
- public static final CompactionStrategyConfig DEFAULT_STRATEGY = new CompactionStrategyConfig("") {
- @Override
- public CompactionStrategyConfig setOptions(Map<String,String> opts) {
- throw new UnsupportedOperationException();
- }
- };
-
- private static final int MAGIC = 0xcc5e6024;
-
- public static void encode(DataOutput dout, CompactionConfig cc) {
- var cs = cc.getCompactionStrategy();
- UserCompactionUtils.encode(dout, MAGIC, 1, cs.getClassName(), cs.getOptions());
- }
-
- public static void decode(CompactionConfig cc, DataInput din) {
- var pcd = UserCompactionUtils.decode(din, MAGIC, 1);
- var csc = new CompactionStrategyConfig(pcd.className).setOptions(pcd.opts);
-
- if (!isDefault(csc)) {
- cc.setCompactionStrategy(csc);
- }
- }
-
- public static boolean isDefault(CompactionStrategyConfig compactionStrategy) {
- return compactionStrategy.equals(DEFAULT_STRATEGY);
-
- }
-
- public static boolean isDefault(CompactionConfig compactionConfig) {
- return isDefault(compactionConfig.getCompactionStrategy());
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
index cb76751..654135a 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
@@ -18,10 +18,10 @@
*/
package org.apache.accumulo.core.clientImpl;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@@ -42,6 +42,8 @@
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -50,6 +52,7 @@
import org.apache.accumulo.core.client.Durability;
import org.apache.accumulo.core.client.TimedOutException;
import org.apache.accumulo.core.clientImpl.TabletLocator.TabletServerMutations;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Condition;
@@ -63,21 +66,15 @@
import org.apache.accumulo.core.dataImpl.thrift.TConditionalSession;
import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
import org.apache.accumulo.core.dataImpl.thrift.TMutation;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.LockID;
+import org.apache.accumulo.core.lock.ServiceLock;
import org.apache.accumulo.core.rpc.ThriftUtil;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.VisibilityEvaluator;
-import org.apache.accumulo.core.security.VisibilityParseException;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.BadArgumentException;
import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.threads.ThreadPools;
import org.apache.accumulo.core.util.threads.Threads;
import org.apache.commons.collections4.map.LRUMap;
@@ -91,6 +88,7 @@
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.net.HostAndPort;
public class ConditionalWriterImpl implements ConditionalWriter {
@@ -98,14 +96,14 @@
private static final int MAX_SLEEP = 30000;
- private Authorizations auths;
- private VisibilityEvaluator ve;
- private Map<Text,Boolean> cache = Collections.synchronizedMap(new LRUMap<>(1000));
+ private final Authorizations auths;
+ private final AccessEvaluator accessEvaluator;
+ private final Map<Text,Boolean> cache = Collections.synchronizedMap(new LRUMap<>(1000));
private final ClientContext context;
- private TabletLocator locator;
+ private final TabletLocator locator;
private final TableId tableId;
private final String tableName;
- private long timeout;
+ private final long timeout;
private final Durability durability;
private final String classLoaderContext;
private final ConditionalWriterConfig config;
@@ -154,7 +152,7 @@
count--;
return result;
} catch (InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
@@ -233,7 +231,7 @@
@Override
public void run() {
- TabletClientService.Iface client = null;
+ TabletIngestClientService.Iface client = null;
for (SessionID sid : sessions) {
if (!sid.isActive()) {
@@ -377,7 +375,7 @@
this.config = config;
this.context = context;
this.auths = config.getAuthorizations();
- this.ve = new VisibilityEvaluator(config.getAuthorizations());
+ this.accessEvaluator = AccessEvaluator.of(config.getAuthorizations().toAccessAuthorizations());
this.threadPool = context.threadPools().createScheduledExecutorService(
config.getMaxWriteThreads(), this.getClass().getSimpleName());
this.locator = new SyncingTabletLocator(context, tableId);
@@ -486,7 +484,7 @@
private HashMap<HostAndPort,SessionID> cachedSessionIDs = new HashMap<>();
- private SessionID reserveSessionID(HostAndPort location, TabletClientService.Iface client,
+ private SessionID reserveSessionID(HostAndPort location, TabletIngestClientService.Iface client,
TInfo tinfo) throws ThriftSecurityException, TException {
// avoid cost of repeatedly making RPC to create sessions, reuse sessions
synchronized (cachedSessionIDs) {
@@ -555,18 +553,19 @@
return activeSessions;
}
- private TabletClientService.Iface getClient(HostAndPort location) throws TTransportException {
- TabletClientService.Iface client;
+ private TabletIngestClientService.Iface getClient(HostAndPort location)
+ throws TTransportException {
+ TabletIngestClientService.Iface client;
if (timeout < context.getClientTimeoutInMillis()) {
- client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context, timeout);
+ client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context, timeout);
} else {
- client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context);
+ client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context);
}
return client;
}
private void sendToServer(HostAndPort location, TabletServerMutations<QCMutation> mutations) {
- TabletClientService.Iface client = null;
+ TabletIngestClientService.Iface client = null;
TInfo tinfo = TraceUtil.traceInfo();
@@ -723,7 +722,7 @@
}
private void invalidateSession(long sessionId, HostAndPort location) throws TException {
- TabletClientService.Iface client = null;
+ TabletIngestClientService.Iface client = null;
TInfo tinfo = TraceUtil.traceInfo();
@@ -810,21 +809,24 @@
}
private boolean isVisible(ByteSequence cv) {
- Text testVis = new Text(cv.toArray());
- if (testVis.getLength() == 0) {
+
+ if (cv.length() == 0) {
return true;
}
+ byte[] arrayVis = cv.toArray();
+ Text testVis = new Text(arrayVis);
+
Boolean b = cache.get(testVis);
if (b != null) {
return b;
}
try {
- boolean bb = ve.evaluate(new ColumnVisibility(testVis));
+ boolean bb = accessEvaluator.canAccess(arrayVis);
cache.put(new Text(testVis), bb);
return bb;
- } catch (VisibilityParseException | BadArgumentException e) {
+ } catch (InvalidAccessExpressionException e) {
return false;
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
deleted file mode 100644
index e55375a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
+++ /dev/null
@@ -1,212 +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
- *
- * https://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.clientImpl;
-
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-
-import java.util.List;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchDeleter;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ConditionalWriter;
-import org.apache.accumulo.core.client.ConditionalWriterConfig;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.client.admin.NamespaceOperations;
-import org.apache.accumulo.core.client.admin.ReplicationOperations;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
-import org.apache.accumulo.core.trace.TraceUtil;
-
-/**
- * This class now delegates to {@link ClientContext}, except for the methods which were not copied
- * over to that.
- */
-@Deprecated(since = "2.0.0")
-public class ConnectorImpl extends org.apache.accumulo.core.client.Connector {
-
- private static final String SYSTEM_TOKEN_NAME =
- "org.apache.accumulo.server.security.SystemCredentials$SystemToken";
- private final ClientContext context;
-
- public ConnectorImpl(ClientContext context) throws AccumuloSecurityException, AccumuloException {
- this.context = context;
- SingletonManager.setMode(Mode.CONNECTOR);
- if (context.getCredentials().getToken().isDestroyed()) {
- throw new AccumuloSecurityException(context.getCredentials().getPrincipal(),
- SecurityErrorCode.TOKEN_EXPIRED);
- }
- // Skip fail fast for system services; string literal for class name, to avoid dependency on
- // server jar
- final String tokenClassName = context.getCredentials().getToken().getClass().getName();
- if (!SYSTEM_TOKEN_NAME.equals(tokenClassName)) {
- if (!ThriftClientTypes.CLIENT.execute(context,
- client -> client.authenticate(TraceUtil.traceInfo(), context.rpcCreds()))) {
- throw new AccumuloSecurityException("Authentication failed, access denied",
- SecurityErrorCode.BAD_CREDENTIALS);
- }
- }
- }
-
- public ClientContext getAccumuloClient() {
- return context;
- }
-
- @Override
- public org.apache.accumulo.core.client.Instance getInstance() {
- return new org.apache.accumulo.core.client.Instance() {
- @Override
- public String getRootTabletLocation() {
- return context.getRootTabletLocation();
- }
-
- @Override
- public List<String> getMasterLocations() {
- return context.getManagerLocations();
- }
-
- @Override
- public String getInstanceID() {
- return context.getInstanceID().canonical();
- }
-
- @Override
- public String getInstanceName() {
- return context.getInstanceName();
- }
-
- @Override
- public String getZooKeepers() {
- return context.getZooKeepers();
- }
-
- @Override
- public int getZooKeepersSessionTimeOut() {
- return context.getZooKeepersSessionTimeOut();
- }
-
- @Override
- public org.apache.accumulo.core.client.Connector getConnector(String principal,
- AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
- return org.apache.accumulo.core.client.Connector.from(context);
- }
- };
- }
-
- @Override
- public BatchScanner createBatchScanner(String tableName, Authorizations authorizations,
- int numQueryThreads) throws TableNotFoundException {
- return context.createBatchScanner(tableName, authorizations, numQueryThreads);
- }
-
- @Override
- public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
- int numQueryThreads, long maxMemory, long maxLatency, int maxWriteThreads)
- throws TableNotFoundException {
- return context.createBatchDeleter(tableName, authorizations, numQueryThreads,
- new BatchWriterConfig().setMaxMemory(maxMemory).setMaxLatency(maxLatency, MILLISECONDS)
- .setMaxWriteThreads(maxWriteThreads));
- }
-
- @Override
- public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
- int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException {
- return context.createBatchDeleter(tableName, authorizations, numQueryThreads, config);
- }
-
- @Override
- public BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency,
- int maxWriteThreads) throws TableNotFoundException {
- return context.createBatchWriter(tableName, new BatchWriterConfig().setMaxMemory(maxMemory)
- .setMaxLatency(maxLatency, MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
- }
-
- @Override
- public BatchWriter createBatchWriter(String tableName, BatchWriterConfig config)
- throws TableNotFoundException {
- return context.createBatchWriter(tableName, config);
- }
-
- @Override
- public MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency,
- int maxWriteThreads) {
- return context.createMultiTableBatchWriter(new BatchWriterConfig().setMaxMemory(maxMemory)
- .setMaxLatency(maxLatency, MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
- }
-
- @Override
- public MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config) {
- return context.createMultiTableBatchWriter(config);
- }
-
- @Override
- public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config)
- throws TableNotFoundException {
- return context.createConditionalWriter(tableName, config);
- }
-
- @Override
- public Scanner createScanner(String tableName, Authorizations authorizations)
- throws TableNotFoundException {
- return context.createScanner(tableName, authorizations);
- }
-
- @Override
- public String whoami() {
- return context.whoami();
- }
-
- @Override
- public TableOperations tableOperations() {
- return context.tableOperations();
- }
-
- @Override
- public NamespaceOperations namespaceOperations() {
- return context.namespaceOperations();
- }
-
- @Override
- public SecurityOperations securityOperations() {
- return context.securityOperations();
- }
-
- @Override
- public InstanceOperations instanceOperations() {
- return context.instanceOperations();
- }
-
- @Override
- public ReplicationOperations replicationOperations() {
- return context.replicationOperations();
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java
index e32047b..eb7dad6 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java
@@ -86,14 +86,14 @@
*
* @param instanceID Accumulo instance ID
* @return Thrift credentials
- * @throws RuntimeException if the authentication token has been destroyed (expired)
+ * @throws IllegalStateException if the authentication token has been destroyed (expired)
*/
public TCredentials toThrift(InstanceId instanceID) {
TCredentials tCreds = new TCredentials(getPrincipal(), getToken().getClass().getName(),
ByteBuffer.wrap(AuthenticationTokenSerializer.serialize(getToken())),
instanceID.canonical());
if (getToken().isDestroyed()) {
- throw new RuntimeException("Token has been destroyed",
+ throw new IllegalStateException("Token has been destroyed",
new AccumuloSecurityException(getPrincipal(), SecurityErrorCode.TOKEN_EXPIRED));
}
return tCreds;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java
index f59f9f3..dd7ce09 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java
@@ -19,7 +19,7 @@
package org.apache.accumulo.core.clientImpl;
import org.apache.accumulo.core.client.Durability;
-import org.apache.accumulo.core.tabletserver.thrift.TDurability;
+import org.apache.accumulo.core.tabletingest.thrift.TDurability;
public class DurabilityImpl {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
index 084d59e..215f7c6 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
@@ -20,15 +20,13 @@
import static com.google.common.base.Preconditions.checkArgument;
import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static java.util.concurrent.TimeUnit.SECONDS;
import static java.util.stream.Collectors.toList;
import static org.apache.accumulo.core.rpc.ThriftUtil.createClient;
import static org.apache.accumulo.core.rpc.ThriftUtil.createTransport;
import static org.apache.accumulo.core.rpc.ThriftUtil.getClient;
import static org.apache.accumulo.core.rpc.ThriftUtil.returnClient;
+import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.ConcurrentModificationException;
@@ -56,11 +54,10 @@
import org.apache.accumulo.core.data.InstanceId;
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client;
import org.apache.accumulo.core.trace.TraceUtil;
import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.LocalityGroupUtil;
import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
import org.apache.accumulo.core.util.Retry;
@@ -69,6 +66,8 @@
import org.apache.thrift.transport.TTransport;
import org.slf4j.LoggerFactory;
+import com.google.common.net.HostAndPort;
+
/**
* Provides a class for administering the accumulo instance
*/
@@ -137,9 +136,9 @@
var log = LoggerFactory.getLogger(InstanceOperationsImpl.class);
- Retry retry =
- Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS)
- .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry();
+ Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+ .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5)
+ .logInterval(Duration.ofMinutes(3)).createRetry();
while (true) {
try {
@@ -206,6 +205,13 @@
}
@Override
+ public Map<String,String> getSystemProperties()
+ throws AccumuloException, AccumuloSecurityException {
+ return ThriftClientTypes.CLIENT.execute(context,
+ client -> client.getSystemProperties(TraceUtil.traceInfo(), context.rpcCreds()));
+ }
+
+ @Override
public List<String> getManagerLocations() {
return context.getManagerLocations();
}
@@ -297,7 +303,7 @@
public List<ActiveCompaction> getActiveCompactions()
throws AccumuloException, AccumuloSecurityException {
- Map<String,List<HostAndPort>> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
+ Map<String,Set<HostAndPort>> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
List<String> tservers = getTabletServers();
int numThreads = Math.max(4, Math.min((tservers.size() + compactors.size()) / 10, 256));
@@ -344,8 +350,7 @@
@Override
public void ping(String tserver) throws AccumuloException {
- try (
- TTransport transport = createTransport(AddressUtil.parseAddress(tserver, false), context)) {
+ try (TTransport transport = createTransport(AddressUtil.parseAddress(tserver), context)) {
Client client = createClient(ThriftClientTypes.TABLET_SERVER, transport);
client.getTabletServerStatus(TraceUtil.traceInfo(), context.rpcCreds());
} catch (TException e) {
@@ -360,7 +365,7 @@
client -> client.waitForBalance(TraceUtil.traceInfo()));
} catch (AccumuloSecurityException ex) {
// should never happen
- throw new RuntimeException("Unexpected exception thrown", ex);
+ throw new IllegalStateException("Unexpected exception thrown", ex);
}
}
@@ -382,12 +387,6 @@
}
@Override
- @Deprecated(since = "2.1.0")
- public String getInstanceID() {
- return getInstanceId().canonical();
- }
-
- @Override
public InstanceId getInstanceId() {
return context.getInstanceID();
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
index 0b1d496..3c92fa5 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
@@ -20,13 +20,12 @@
import static com.google.common.base.Preconditions.checkArgument;
import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.apache.accumulo.core.util.Validators.EXISTING_NAMESPACE_NAME;
import static org.apache.accumulo.core.util.Validators.NEW_NAMESPACE_NAME;
import java.nio.ByteBuffer;
+import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
import java.util.ConcurrentModificationException;
@@ -245,9 +244,9 @@
EXISTING_NAMESPACE_NAME.validate(namespace);
checkArgument(mapMutator != null, "mapMutator is null");
- Retry retry =
- Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS)
- .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry();
+ Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+ .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5)
+ .logInterval(Duration.ofMinutes(3)).createRetry();
while (true) {
try {
@@ -338,7 +337,8 @@
public Map<String,String> namespaceIdMap() {
return Namespaces.getNameToIdMap(context).entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().canonical(), (v1, v2) -> {
- throw new RuntimeException(String.format("Duplicate key for values %s and %s", v1, v2));
+ throw new IllegalStateException(
+ String.format("Duplicate key for values %s and %s", v1, v2));
}, TreeMap::new));
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
index a03cc81..9200591 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
@@ -18,13 +18,14 @@
*/
package org.apache.accumulo.core.clientImpl;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES;
import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -59,14 +60,12 @@
import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
import org.apache.accumulo.core.manager.state.tables.TableState;
import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
import org.apache.accumulo.core.metadata.schema.TabletMetadata;
import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
import org.apache.accumulo.core.security.Authorizations;
import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.spi.common.ServiceEnvironment;
import org.apache.accumulo.core.util.LocalityGroupUtil;
import org.apache.accumulo.core.volume.VolumeConfiguration;
import org.apache.hadoop.conf.Configuration;
@@ -94,12 +93,6 @@
this.sampleConf = samplerConf;
}
- @Deprecated(since = "2.0.0")
- @Override
- public AccumuloConfiguration getConfig() {
- return conf;
- }
-
@Override
public IteratorScope getIteratorScope() {
return IteratorScope.scan;
@@ -118,12 +111,6 @@
private final ArrayList<SortedKeyValueIterator<Key,Value>> topLevelIterators =
new ArrayList<>();
- @Deprecated(since = "2.0.0")
- @Override
- public void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {
- topLevelIterators.add(iter);
- }
-
@Override
public Authorizations getAuthorizations() {
return authorizations;
@@ -157,12 +144,6 @@
sampleConf);
}
- @Deprecated(since = "2.1.0")
- @Override
- public ServiceEnvironment getServiceEnv() {
- return new ClientServiceEnvironmentImpl(context);
- }
-
@Override
public PluginEnvironment getPluginEnv() {
return new ClientServiceEnvironmentImpl(context);
@@ -205,11 +186,10 @@
nextTablet();
}
- } catch (Exception e) {
- if (e instanceof RuntimeException) {
- throw (RuntimeException) e;
- }
- throw new RuntimeException(e);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
+ throw new IllegalStateException(e);
}
}
@@ -232,8 +212,10 @@
}
return ret;
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
+ throw new IllegalStateException(e);
}
}
@@ -329,11 +311,11 @@
if (scannerSamplerConfigImpl != null && !scannerSamplerConfigImpl.equals(samplerConfImpl)) {
throw new SampleNotPresentException();
}
- for (TabletFile file : absFiles) {
+ for (StoredTabletFile file : absFiles) {
var cs = CryptoFactoryLoader.getServiceForClientWithTable(systemConf, tableConf, tableId);
- FileSystem fs = VolumeConfiguration.fileSystemForPath(file.getPathStr(), conf);
+ FileSystem fs = VolumeConfiguration.fileSystemForPath(file.getNormalizedPathStr(), conf);
FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
- .forFile(file.getPathStr(), fs, conf, cs).withTableConfiguration(tableCC).build();
+ .forFile(file, fs, conf, cs).withTableConfiguration(tableCC).build();
if (scannerSamplerConfigImpl != null) {
reader = reader.getSample(scannerSamplerConfigImpl);
if (reader == null) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
deleted file mode 100644
index 49aa1f5..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
+++ /dev/null
@@ -1,150 +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
- *
- * https://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.clientImpl;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.Collections;
-import java.util.HashSet;
-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.BatchScanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.ReplicationOperations;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Deprecated
-public class ReplicationOperationsImpl implements ReplicationOperations {
- private static final Logger log = LoggerFactory.getLogger(ReplicationOperationsImpl.class);
-
- private final ClientContext context;
-
- public ReplicationOperationsImpl(ClientContext context) {
- this.context = requireNonNull(context);
- }
-
- @Override
- public void addPeer(final String name, final String replicaType)
- throws AccumuloException, AccumuloSecurityException {
- context.instanceOperations().setProperty(
- Property.REPLICATION_PEERS.getKey() + requireNonNull(name), requireNonNull(replicaType));
- }
-
- @Override
- public void removePeer(final String name) throws AccumuloException, AccumuloSecurityException {
- context.instanceOperations()
- .removeProperty(Property.REPLICATION_PEERS.getKey() + requireNonNull(name));
- }
-
- @Override
- public void drain(String tableName)
- throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
- drain(tableName, referencedFiles(requireNonNull(tableName)));
- }
-
- @Override
- public void drain(final String tableName, final Set<String> wals)
- throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
- requireNonNull(tableName);
-
- final TInfo tinfo = TraceUtil.traceInfo();
- final TCredentials rpcCreds = context.rpcCreds();
-
- // Ask the manager if the table is fully replicated given these WALs, but don't poll inside the
- // manager
- boolean drained = false;
- while (!drained) {
- drained = getManagerDrain(tinfo, rpcCreds, tableName, wals);
-
- if (!drained) {
- try {
- Thread.sleep(1000);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new RuntimeException("Thread interrupted", e);
- }
- }
- }
- }
-
- protected boolean getManagerDrain(final TInfo tinfo, final TCredentials rpcCreds,
- final String tableName, final Set<String> wals)
- throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
- return ThriftClientTypes.MANAGER.execute(context,
- client -> client.drainReplicationTable(tinfo, rpcCreds, tableName, wals));
- }
-
- @Override
- public Set<String> referencedFiles(String tableName) throws TableNotFoundException {
- log.debug("Collecting referenced files for replication of table {}", tableName);
- TableId tableId = context.getTableId(tableName);
- log.debug("Found id of {} for name {}", tableId, tableName);
-
- // Get the WALs currently referenced by the table
- BatchScanner metaBs = context.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
- metaBs.setRanges(Collections.singleton(TabletsSection.getRange(tableId)));
- metaBs.fetchColumnFamily(LogColumnFamily.NAME);
- Set<String> wals = new HashSet<>();
- try {
- for (Entry<Key,Value> entry : metaBs) {
- LogEntry logEntry = LogEntry.fromMetaWalEntry(entry);
- wals.add(new Path(logEntry.filename).toString());
- }
- } finally {
- metaBs.close();
- }
-
- // And the WALs that need to be replicated for this table
- metaBs = context.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
- metaBs.setRanges(Collections.singleton(ReplicationSection.getRange()));
- metaBs.fetchColumnFamily(ReplicationSection.COLF);
- try {
- Text buffer = new Text();
- for (Entry<Key,Value> entry : metaBs) {
- if (tableId.equals(ReplicationSection.getTableId(entry.getKey()))) {
- ReplicationSection.getFile(entry.getKey(), buffer);
- wals.add(buffer.toString());
- }
- }
- } finally {
- metaBs.close();
- }
- return wals;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
index 28aa5b9..a24a236 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
@@ -18,10 +18,10 @@
*/
package org.apache.accumulo.core.clientImpl;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
import java.util.Collection;
import java.util.Collections;
@@ -56,11 +56,11 @@
Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures) {
TabletLocation rootTabletLocation = getRootTabletLocation(context);
if (rootTabletLocation != null) {
- TabletServerMutations<T> tsm = new TabletServerMutations<>(rootTabletLocation.tablet_session);
+ var tsm = new TabletServerMutations<T>(rootTabletLocation.getTserverSession());
for (T mutation : mutations) {
tsm.addMutation(RootTable.EXTENT, mutation);
}
- binnedMutations.put(rootTabletLocation.tablet_location, tsm);
+ binnedMutations.put(rootTabletLocation.getTserverLocation(), tsm);
} else {
failures.addAll(mutations);
}
@@ -73,7 +73,7 @@
TabletLocation rootTabletLocation = getRootTabletLocation(context);
if (rootTabletLocation != null) {
for (Range range : ranges) {
- TabletLocatorImpl.addRange(binnedRanges, rootTabletLocation.tablet_location,
+ TabletLocatorImpl.addRange(binnedRanges, rootTabletLocation.getTserverLocation(),
RootTable.EXTENT, range);
}
return Collections.emptyList();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
index 182b912..1f065bb 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
@@ -27,11 +27,15 @@
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.SampleNotPresentException;
import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
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.clientImpl.ThriftScanner.ScanState;
+import org.apache.accumulo.core.clientImpl.ThriftScanner.ScanTimedOutException;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.KeyValue;
import org.apache.accumulo.core.data.Range;
@@ -143,7 +147,8 @@
readAheadOperation = context.submitScannerReadAheadTask(this::readBatch);
}
- private List<KeyValue> readBatch() throws Exception {
+ private List<KeyValue> readBatch() throws ScanTimedOutException, AccumuloException,
+ AccumuloSecurityException, TableNotFoundException {
List<KeyValue> batch;
@@ -176,11 +181,10 @@
}
} catch (ExecutionException ee) {
wrapExecutionException(ee);
- throw new RuntimeException(ee);
- } catch (RuntimeException e) {
- throw e;
- } catch (Exception e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(ee);
+ } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException
+ | ScanTimedOutException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
if (!nextBatch.isEmpty()) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
index 8f6daed..df40a21 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
@@ -21,7 +21,7 @@
import java.util.Collection;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.Callable;
+import java.util.function.Supplier;
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -31,8 +31,6 @@
import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Syncs itself with the static collection of TabletLocators, so that when the server clears it, it
@@ -40,19 +38,13 @@
* using SyncingTabletLocator.
*/
public class SyncingTabletLocator extends TabletLocator {
- private static final Logger log = LoggerFactory.getLogger(SyncingTabletLocator.class);
private volatile TabletLocator locator;
- private final Callable<TabletLocator> getLocatorFunction;
+ private final Supplier<TabletLocator> getLocatorFunction;
- public SyncingTabletLocator(Callable<TabletLocator> getLocatorFunction) {
+ public SyncingTabletLocator(Supplier<TabletLocator> getLocatorFunction) {
this.getLocatorFunction = getLocatorFunction;
- try {
- this.locator = getLocatorFunction.call();
- } catch (Exception e) {
- log.error("Problem obtaining TabletLocator", e);
- throw new RuntimeException(e);
- }
+ this.locator = getLocatorFunction.get();
}
public SyncingTabletLocator(final ClientContext context, final TableId tableId) {
@@ -64,12 +56,7 @@
if (!loc.isValid()) {
synchronized (this) {
if (locator == loc) {
- try {
- loc = locator = getLocatorFunction.call();
- } catch (Exception e) {
- log.error("Problem obtaining TabletLocator", e);
- throw new RuntimeException(e);
- }
+ loc = locator = getLocatorFunction.get();
}
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index 129cb6a..6b94fd8 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -19,15 +19,15 @@
package org.apache.accumulo.core.clientImpl;
import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.concurrent.TimeUnit.SECONDS;
import static java.util.stream.Collectors.toSet;
import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
import static org.apache.accumulo.core.util.Validators.NEW_TABLE_NAME;
@@ -36,7 +36,7 @@
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.ByteBuffer;
-import java.security.SecureRandom;
+import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -119,9 +119,8 @@
import org.apache.accumulo.core.manager.thrift.FateOperation;
import org.apache.accumulo.core.manager.thrift.FateService;
import org.apache.accumulo.core.manager.thrift.ManagerClientService;
-import org.apache.accumulo.core.metadata.MetadataServicer;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.schema.TabletDeletedException;
import org.apache.accumulo.core.metadata.schema.TabletMetadata;
import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
@@ -132,10 +131,9 @@
import org.apache.accumulo.core.security.Authorizations;
import org.apache.accumulo.core.summary.SummarizerConfigurationUtil;
import org.apache.accumulo.core.summary.SummaryCollection;
+import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService;
import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.LocalityGroupUtil;
import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
import org.apache.accumulo.core.util.MapCounter;
@@ -156,11 +154,10 @@
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
public class TableOperationsImpl extends TableOperationsHelper {
- private static final SecureRandom random = new SecureRandom();
-
public static final String PROPERTY_EXCLUDE_PREFIX = "!";
public static final String COMPACTION_CANCELED_MSG = "Compaction canceled";
public static final String TABLE_DELETED_MSG = "Table is being deleted";
@@ -198,7 +195,8 @@
public boolean exists(String tableName) {
EXISTING_TABLE_NAME.validate(tableName);
- if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME)) {
+ if (tableName.equals(AccumuloTable.METADATA.tableName())
+ || tableName.equals(AccumuloTable.ROOT.tableName())) {
return true;
}
@@ -531,7 +529,7 @@
}
}
} catch (InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
} finally {
executor.shutdown();
}
@@ -562,26 +560,26 @@
continue;
}
- HostAndPort address = HostAndPort.fromString(tl.tablet_location);
+ HostAndPort address = HostAndPort.fromString(tl.getTserverLocation());
try {
- TabletClientService.Client client =
- ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
+ TabletManagementClientService.Client client =
+ ThriftUtil.getClient(ThriftClientTypes.TABLET_MGMT, address, context);
try {
OpTimer timer = null;
if (log.isTraceEnabled()) {
log.trace("tid={} Splitting tablet {} on {} at {}", Thread.currentThread().getId(),
- tl.tablet_extent, address, split);
+ tl.getExtent(), address, split);
timer = new OpTimer().start();
}
- client.splitTablet(TraceUtil.traceInfo(), context.rpcCreds(),
- tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
+ client.splitTablet(TraceUtil.traceInfo(), context.rpcCreds(), tl.getExtent().toThrift(),
+ TextUtil.getByteBuffer(split));
// just split it, might as well invalidate it in the cache
- tabLocator.invalidateCache(tl.tablet_extent);
+ tabLocator.invalidateCache(tl.getExtent());
if (timer != null) {
timer.stop();
@@ -606,10 +604,10 @@
+ " Seen {} failures.", split, env.tableName, locationFailures);
}
- tabLocator.invalidateCache(tl.tablet_extent);
+ tabLocator.invalidateCache(tl.getExtent());
continue;
} catch (TException e) {
- tabLocator.invalidateCache(context, tl.tablet_location);
+ tabLocator.invalidateCache(context, tl.getTserverLocation());
continue;
}
@@ -665,37 +663,22 @@
private List<Text> _listSplits(String tableName)
throws TableNotFoundException, AccumuloSecurityException {
+
TableId tableId = context.getTableId(tableName);
- TreeMap<KeyExtent,String> tabletLocations = new TreeMap<>();
+
while (true) {
- try {
- tabletLocations.clear();
- // the following method throws AccumuloException for some conditions that should be retried
- MetadataServicer.forTableId(context, tableId).getTabletLocations(tabletLocations);
- break;
- } catch (AccumuloSecurityException ase) {
- throw ase;
- } catch (Exception e) {
+ try (TabletsMetadata tabletsMetadata = context.getAmple().readTablets().forTable(tableId)
+ .fetch(PREV_ROW).checkConsistency().build()) {
+ return tabletsMetadata.stream().map(tm -> tm.getExtent().endRow()).filter(Objects::nonNull)
+ .collect(Collectors.toList());
+ } catch (TabletDeletedException tde) {
+ // see if the table was deleted
context.requireTableExists(tableId, tableName);
-
- if (e instanceof RuntimeException && e.getCause() instanceof AccumuloSecurityException) {
- throw (AccumuloSecurityException) e.getCause();
- }
-
- log.info("{} ... retrying ...", e, e);
+ log.debug("A merge happened while trying to list splits for {} {}, retrying ", tableName,
+ tableId, tde);
sleepUninterruptibly(3, SECONDS);
}
}
-
- ArrayList<Text> endRows = new ArrayList<>(tabletLocations.size());
- for (KeyExtent ke : tabletLocations.keySet()) {
- if (ke.endRow() != null) {
- endRows.add(ke.endRow());
- }
- }
-
- return endRows;
-
}
/**
@@ -860,8 +843,6 @@
}
}
- ensureStrategyCanLoad(tableName, config);
-
if (!UserCompactionUtils.isDefault(config.getConfigurer())) {
if (!testClassLoad(tableName, config.getConfigurer().getClassName(),
CompactionConfigurer.class.getName())) {
@@ -902,19 +883,6 @@
}
}
- @SuppressWarnings("removal")
- private void ensureStrategyCanLoad(String tableName, CompactionConfig config)
- throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
- // Make sure the specified compaction strategy exists on a tabletserver
- if (!CompactionStrategyConfigUtil.isDefault(config.getCompactionStrategy())) {
- if (!testClassLoad(tableName, config.getCompactionStrategy().getClassName(),
- "org.apache.accumulo.tserver.compaction.CompactionStrategy")) {
- throw new AccumuloException("TabletServer could not load CompactionStrategy class "
- + config.getCompactionStrategy().getClassName());
- }
- }
- }
-
@Override
public void cancelCompaction(String tableName)
throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
@@ -1053,9 +1021,9 @@
EXISTING_TABLE_NAME.validate(tableName);
checkArgument(mapMutator != null, "mapMutator is null");
- Retry retry =
- Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS)
- .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry();
+ Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+ .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5)
+ .logInterval(Duration.ofMinutes(3)).createRetry();
while (true) {
try {
@@ -1261,7 +1229,7 @@
log.warn("Unable to locate bins for specified range. Retrying.");
// sleep randomly between 100 and 200ms
- sleepUninterruptibly(100 + random.nextInt(100), MILLISECONDS);
+ sleepUninterruptibly(100 + RANDOM.get().nextInt(100), MILLISECONDS);
binnedRanges.clear();
tl.invalidateCache();
}
@@ -1327,34 +1295,6 @@
return ret;
}
- @Override
- @Deprecated(since = "2.0.0")
- public void importDirectory(String tableName, String dir, String failureDir, boolean setTime)
- throws IOException, AccumuloSecurityException, TableNotFoundException, AccumuloException {
- EXISTING_TABLE_NAME.validate(tableName);
- checkArgument(dir != null, "dir is null");
- checkArgument(failureDir != null, "failureDir is null");
-
- // check for table existence
- context.getTableId(tableName);
- Path dirPath = checkPath(dir, "Bulk", "");
- Path failPath = checkPath(failureDir, "Bulk", "failure");
-
- List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)),
- ByteBuffer.wrap(dirPath.toString().getBytes(UTF_8)),
- ByteBuffer.wrap(failPath.toString().getBytes(UTF_8)),
- ByteBuffer.wrap((setTime + "").getBytes(UTF_8)));
- Map<String,String> opts = new HashMap<>();
-
- try {
- doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_BULK_IMPORT,
- args, opts);
- } catch (TableExistsException e) {
- // should not happen
- throw new AssertionError(e);
- }
- }
-
private void waitForTableStateTransition(TableId tableId, TableState expectedState)
throws AccumuloException, TableNotFoundException {
Text startRow = null;
@@ -1381,9 +1321,6 @@
range = new Range(startRow, lastRow);
}
- TabletsMetadata tablets = TabletsMetadata.builder(context).scanMetadataTable()
- .overRange(range).fetch(LOCATION, PREV_ROW).build();
-
KeyExtent lastExtent = null;
int total = 0;
@@ -1392,34 +1329,38 @@
Text continueRow = null;
MapCounter<String> serverCounts = new MapCounter<>();
- for (TabletMetadata tablet : tablets) {
- total++;
- Location loc = tablet.getLocation();
+ try (TabletsMetadata tablets = TabletsMetadata.builder(context).scanMetadataTable()
+ .overRange(range).fetch(LOCATION, PREV_ROW).build()) {
- if ((expectedState == TableState.ONLINE
- && (loc == null || loc.getType() == LocationType.FUTURE))
- || (expectedState == TableState.OFFLINE && loc != null)) {
- if (continueRow == null) {
- continueRow = tablet.getExtent().toMetaRow();
+ for (TabletMetadata tablet : tablets) {
+ total++;
+ Location loc = tablet.getLocation();
+
+ if ((expectedState == TableState.ONLINE
+ && (loc == null || loc.getType() == LocationType.FUTURE))
+ || (expectedState == TableState.OFFLINE && loc != null)) {
+ if (continueRow == null) {
+ continueRow = tablet.getExtent().toMetaRow();
+ }
+ waitFor++;
+ lastRow = tablet.getExtent().toMetaRow();
+
+ if (loc != null) {
+ serverCounts.increment(loc.getHostPortSession(), 1);
+ }
}
- waitFor++;
- lastRow = tablet.getExtent().toMetaRow();
- if (loc != null) {
- serverCounts.increment(loc.getHostPortSession(), 1);
+ if (!tablet.getExtent().tableId().equals(tableId)) {
+ throw new AccumuloException(
+ "Saw unexpected table Id " + tableId + " " + tablet.getExtent());
}
- }
- if (!tablet.getExtent().tableId().equals(tableId)) {
- throw new AccumuloException(
- "Saw unexpected table Id " + tableId + " " + tablet.getExtent());
- }
+ if (lastExtent != null && !tablet.getExtent().isPreviousExtent(lastExtent)) {
+ holes++;
+ }
- if (lastExtent != null && !tablet.getExtent().isPreviousExtent(lastExtent)) {
- holes++;
+ lastExtent = tablet.getExtent();
}
-
- lastExtent = tablet.getExtent();
}
if (continueRow != null) {
@@ -1539,7 +1480,8 @@
public Map<String,String> tableIdMap() {
return context.getTableNameToIdMap().entrySet().stream()
.collect(Collectors.toMap(Entry::getKey, e -> e.getValue().canonical(), (v1, v2) -> {
- throw new RuntimeException(String.format("Duplicate key for values %s and %s", v1, v2));
+ throw new IllegalStateException(
+ String.format("Duplicate key for values %s and %s", v1, v2));
}, TreeMap::new));
}
@@ -1563,7 +1505,7 @@
// this operation may us a lot of memory... its likely that connections to tabletservers
// hosting metadata tablets will be cached, so do not use cached
// connections
- pair = ThriftClientTypes.CLIENT.getTabletServerConnection(context, false);
+ pair = ThriftClientTypes.CLIENT.getThriftServerConnection(context, false);
diskUsages = pair.getSecond().getDiskUsage(tableNames, context.rpcCreds());
} catch (ThriftTableOperationException e) {
switch (e.getType()) {
@@ -1886,7 +1828,7 @@
List<Range> prev =
groupedByTablets.put(tabletId, Collections.unmodifiableList(entry2.getValue()));
if (prev != null) {
- throw new RuntimeException(
+ throw new IllegalStateException(
"Unexpected : tablet at multiple locations : " + location + " " + tabletId);
}
}
@@ -1945,9 +1887,9 @@
locator.invalidateCache();
- Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
- .incrementBy(100, MILLISECONDS).maxWait(2, SECONDS).backOffFactor(1.5)
- .logInterval(3, MINUTES).createRetry();
+ Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+ .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofSeconds(2)).backOffFactor(1.5)
+ .logInterval(Duration.ofMinutes(3)).createRetry();
while (!locator.binRanges(context, rangeList, binnedRanges).isEmpty()) {
context.requireTableExists(tableId, tableName);
@@ -1958,7 +1900,7 @@
String.format("locating tablets in table %s(%s) for %d ranges", tableName, tableId,
rangeList.size()));
} catch (InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
locator.invalidateCache();
}
@@ -2119,8 +2061,11 @@
@Override
public TimeType getTimeType(final String tableName) throws TableNotFoundException {
TableId tableId = context.getTableId(tableName);
- Optional<TabletMetadata> tabletMetadata = context.getAmple().readTablets().forTable(tableId)
- .fetch(TabletMetadata.ColumnType.TIME).checkConsistency().build().stream().findFirst();
+ Optional<TabletMetadata> tabletMetadata;
+ try (TabletsMetadata tabletsMetadata = context.getAmple().readTablets().forTable(tableId)
+ .fetch(TabletMetadata.ColumnType.TIME).checkConsistency().build()) {
+ tabletMetadata = tabletsMetadata.stream().findFirst();
+ }
TabletMetadata timeData =
tabletMetadata.orElseThrow(() -> new IllegalStateException("Failed to retrieve TimeType"));
return timeData.getTime().getType();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
index b8e4c53..9c528c3 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
@@ -25,6 +25,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -34,9 +35,8 @@
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.AccumuloTable;
import org.apache.accumulo.core.metadata.MetadataLocationObtainer;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
import org.apache.accumulo.core.singletons.SingletonManager;
import org.apache.accumulo.core.singletons.SingletonService;
import org.apache.accumulo.core.util.Interner;
@@ -140,14 +140,15 @@
if (tl == null) {
MetadataLocationObtainer mlo = new MetadataLocationObtainer();
- if (RootTable.ID.equals(tableId)) {
+ if (AccumuloTable.ROOT.tableId().equals(tableId)) {
tl = new RootTabletLocator(new ZookeeperLockChecker(context));
- } else if (MetadataTable.ID.equals(tableId)) {
- tl = new TabletLocatorImpl(MetadataTable.ID, getLocator(context, RootTable.ID), mlo,
+ } else if (AccumuloTable.METADATA.tableId().equals(tableId)) {
+ tl = new TabletLocatorImpl(AccumuloTable.METADATA.tableId(),
+ getLocator(context, AccumuloTable.ROOT.tableId()), mlo,
new ZookeeperLockChecker(context));
} else {
- tl = new TabletLocatorImpl(tableId, getLocator(context, MetadataTable.ID), mlo,
- new ZookeeperLockChecker(context));
+ tl = new TabletLocatorImpl(tableId, getLocator(context, AccumuloTable.METADATA.tableId()),
+ mlo, new ZookeeperLockChecker(context));
}
locators.put(key, tl);
}
@@ -194,54 +195,53 @@
}
}
- public static class TabletLocation implements Comparable<TabletLocation> {
+ public static class TabletLocation {
private static final Interner<String> interner = new Interner<>();
- public final KeyExtent tablet_extent;
- public final String tablet_location;
- public final String tablet_session;
+ private final KeyExtent tablet_extent;
+ private final String tserverLocation;
+ private final String tserverSession;
public TabletLocation(KeyExtent tablet_extent, String tablet_location, String session) {
checkArgument(tablet_extent != null, "tablet_extent is null");
checkArgument(tablet_location != null, "tablet_location is null");
checkArgument(session != null, "session is null");
this.tablet_extent = tablet_extent;
- this.tablet_location = interner.intern(tablet_location);
- this.tablet_session = interner.intern(session);
+ this.tserverLocation = interner.intern(tablet_location);
+ this.tserverSession = interner.intern(session);
}
@Override
public boolean equals(Object o) {
if (o instanceof TabletLocation) {
TabletLocation otl = (TabletLocation) o;
- return tablet_extent.equals(otl.tablet_extent)
- && tablet_location.equals(otl.tablet_location)
- && tablet_session.equals(otl.tablet_session);
+ return getExtent().equals(otl.getExtent())
+ && getTserverLocation().equals(otl.getTserverLocation())
+ && getTserverSession().equals(otl.getTserverSession());
}
return false;
}
@Override
public int hashCode() {
- throw new UnsupportedOperationException(
- "hashcode is not implemented for class " + this.getClass());
+ return Objects.hash(getExtent(), tserverLocation, tserverSession);
}
@Override
public String toString() {
- return "(" + tablet_extent + "," + tablet_location + "," + tablet_session + ")";
+ return "(" + getExtent() + "," + getTserverLocation() + "," + getTserverSession() + ")";
}
- @Override
- public int compareTo(TabletLocation o) {
- int result = tablet_extent.compareTo(o.tablet_extent);
- if (result == 0) {
- result = tablet_location.compareTo(o.tablet_location);
- if (result == 0) {
- result = tablet_session.compareTo(o.tablet_session);
- }
- }
- return result;
+ public KeyExtent getExtent() {
+ return tablet_extent;
+ }
+
+ public String getTserverLocation() {
+ return tserverLocation;
+ }
+
+ public String getTserverSession() {
+ return tserverSession;
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
index 02844db..dafada9 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
@@ -18,9 +18,9 @@
*/
package org.apache.accumulo.core.clientImpl;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
import java.util.ArrayList;
import java.util.Collection;
@@ -125,7 +125,7 @@
return null;
}
- Pair<String,String> lock = new Pair<>(tl.tablet_location, tl.tablet_session);
+ Pair<String,String> lock = new Pair<>(tl.getTserverLocation(), tl.getTserverSession());
if (okLocks.contains(lock)) {
return tl;
@@ -135,14 +135,14 @@
return null;
}
- if (lockChecker.isLockHeld(tl.tablet_location, tl.tablet_session)) {
+ if (lockChecker.isLockHeld(tl.getTserverLocation(), tl.getTserverSession())) {
okLocks.add(lock);
return tl;
}
if (log.isTraceEnabled()) {
- log.trace("Tablet server {} {} no longer holds its lock", tl.tablet_location,
- tl.tablet_session);
+ log.trace("Tablet server {} {} no longer holds its lock", tl.getTserverLocation(),
+ tl.getTserverSession());
}
invalidLocks.add(lock);
@@ -244,22 +244,22 @@
private <T extends Mutation> boolean addMutation(
Map<String,TabletServerMutations<T>> binnedMutations, T mutation, TabletLocation tl,
LockCheckerSession lcSession) {
- TabletServerMutations<T> tsm = binnedMutations.get(tl.tablet_location);
+ TabletServerMutations<T> tsm = binnedMutations.get(tl.getTserverLocation());
if (tsm == null) {
// do lock check once per tserver here to make binning faster
boolean lockHeld = lcSession.checkLock(tl) != null;
if (lockHeld) {
- tsm = new TabletServerMutations<>(tl.tablet_session);
- binnedMutations.put(tl.tablet_location, tsm);
+ tsm = new TabletServerMutations<>(tl.getTserverSession());
+ binnedMutations.put(tl.getTserverLocation(), tsm);
} else {
return false;
}
}
// its possible the same tserver could be listed with different sessions
- if (tsm.getSession().equals(tl.tablet_session)) {
- tsm.addMutation(tl.tablet_extent, mutation);
+ if (tsm.getSession().equals(tl.getTserverSession())) {
+ tsm.addMutation(tl.getExtent(), mutation);
return true;
}
@@ -269,10 +269,10 @@
static boolean isContiguous(List<TabletLocation> tabletLocations) {
Iterator<TabletLocation> iter = tabletLocations.iterator();
- KeyExtent prevExtent = iter.next().tablet_extent;
+ KeyExtent prevExtent = iter.next().getExtent();
while (iter.hasNext()) {
- KeyExtent currExtent = iter.next().tablet_extent;
+ KeyExtent currExtent = iter.next().getExtent();
if (!currExtent.isPreviousExtent(prevExtent)) {
return false;
@@ -323,14 +323,14 @@
tabletLocations.add(tl);
- while (tl.tablet_extent.endRow() != null
- && !range.afterEndKey(new Key(tl.tablet_extent.endRow()).followingKey(PartialKey.ROW))) {
+ while (tl.getExtent().endRow() != null
+ && !range.afterEndKey(new Key(tl.getExtent().endRow()).followingKey(PartialKey.ROW))) {
if (useCache) {
- Text row = new Text(tl.tablet_extent.endRow());
+ Text row = new Text(tl.getExtent().endRow());
row.append(new byte[] {0}, 0, 1);
tl = lcSession.checkLock(locateTabletInCache(row));
} else {
- tl = _locateTablet(context, tl.tablet_extent.endRow(), true, false, false, lcSession);
+ tl = _locateTablet(context, tl.getExtent().endRow(), true, false, false, lcSession);
}
if (tl == null) {
@@ -349,7 +349,8 @@
// then after that merges and splits happen.
if (isContiguous(tabletLocations)) {
for (TabletLocation tl2 : tabletLocations) {
- TabletLocatorImpl.addRange(binnedRanges, tl2.tablet_location, tl2.tablet_extent, range);
+ TabletLocatorImpl.addRange(binnedRanges, tl2.getTserverLocation(), tl2.getExtent(),
+ range);
}
} else {
failures.add(range);
@@ -454,8 +455,8 @@
wLock.lock();
try {
for (TabletLocation cacheEntry : metaCache.values()) {
- if (cacheEntry.tablet_location.equals(server)) {
- badExtents.add(cacheEntry.tablet_extent);
+ if (cacheEntry.getTserverLocation().equals(server)) {
+ badExtents.add(cacheEntry.getExtent());
invalidatedCount++;
}
}
@@ -516,7 +517,7 @@
if (timer != null) {
timer.stop();
log.trace("tid={} Located tablet {} at {} in {}", Thread.currentThread().getId(),
- (tl == null ? "null" : tl.tablet_extent), (tl == null ? "null" : tl.tablet_location),
+ (tl == null ? "null" : tl.getExtent()), (tl == null ? "null" : tl.getTserverLocation()),
String.format("%.3f secs", timer.scale(SECONDS)));
}
@@ -538,7 +539,7 @@
while (locations != null && locations.getLocations().isEmpty()
&& locations.getLocationless().isEmpty()) {
// try the next tablet, the current tablet does not have any tablets that overlap the row
- Text er = ptl.tablet_extent.endRow();
+ Text er = ptl.getExtent().endRow();
if (er != null && er.compareTo(lastTabletRow) < 0) {
// System.out.println("er "+er+" ltr "+lastTabletRow);
ptl = parent.locateTablet(context, er, true, retry);
@@ -563,20 +564,20 @@
Text lastEndRow = null;
for (TabletLocation tabletLocation : locations.getLocations()) {
- KeyExtent ke = tabletLocation.tablet_extent;
+ KeyExtent ke = tabletLocation.getExtent();
TabletLocation locToCache;
// create new location if current prevEndRow == endRow
if ((lastEndRow != null) && (ke.prevEndRow() != null)
&& ke.prevEndRow().equals(lastEndRow)) {
locToCache = new TabletLocation(new KeyExtent(ke.tableId(), ke.endRow(), lastEndRow),
- tabletLocation.tablet_location, tabletLocation.tablet_session);
+ tabletLocation.getTserverLocation(), tabletLocation.getTserverSession());
} else {
locToCache = tabletLocation;
}
// save endRow for next iteration
- lastEndRow = locToCache.tablet_extent.endRow();
+ lastEndRow = locToCache.getExtent().endRow();
updateCache(locToCache, lcSession);
}
@@ -585,20 +586,20 @@
}
private void updateCache(TabletLocation tabletLocation, LockCheckerSession lcSession) {
- if (!tabletLocation.tablet_extent.tableId().equals(tableId)) {
+ if (!tabletLocation.getExtent().tableId().equals(tableId)) {
// sanity check
throw new IllegalStateException(
- "Unexpected extent returned " + tableId + " " + tabletLocation.tablet_extent);
+ "Unexpected extent returned " + tableId + " " + tabletLocation.getExtent());
}
- if (tabletLocation.tablet_location == null) {
+ if (tabletLocation.getTserverLocation() == null) {
// sanity check
throw new IllegalStateException(
- "Cannot add null locations to cache " + tableId + " " + tabletLocation.tablet_extent);
+ "Cannot add null locations to cache " + tableId + " " + tabletLocation.getExtent());
}
// clear out any overlapping extents in cache
- removeOverlapping(metaCache, tabletLocation.tablet_extent);
+ removeOverlapping(metaCache, tabletLocation.getExtent());
// do not add to cache unless lock is held
if (lcSession.checkLock(tabletLocation) == null) {
@@ -606,14 +607,14 @@
}
// add it to cache
- Text er = tabletLocation.tablet_extent.endRow();
+ Text er = tabletLocation.getExtent().endRow();
if (er == null) {
er = MAX_TEXT;
}
metaCache.put(er, tabletLocation);
if (!badExtents.isEmpty()) {
- removeOverlapping(badExtents, tabletLocation.tablet_extent);
+ removeOverlapping(badExtents, tabletLocation.getExtent());
}
}
@@ -631,7 +632,7 @@
while (iter.hasNext()) {
Entry<Text,TabletLocation> entry = iter.next();
- KeyExtent ke = entry.getValue().tablet_extent;
+ KeyExtent ke = entry.getValue().getExtent();
if (stopRemoving(nke, ke)) {
break;
@@ -663,7 +664,7 @@
Entry<Text,TabletLocation> entry = metaCache.ceilingEntry(row);
if (entry != null) {
- KeyExtent ke = entry.getValue().tablet_extent;
+ KeyExtent ke = entry.getValue().getExtent();
if (ke.prevEndRow() == null || ke.prevEndRow().compareTo(row) < 0) {
return entry.getValue();
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
index 322fd0f..b39240f 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
@@ -18,8 +18,6 @@
*/
package org.apache.accumulo.core.clientImpl;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.concurrent.TimeUnit.SECONDS;
import java.io.IOException;
@@ -73,13 +71,12 @@
import org.apache.accumulo.core.spi.scan.ScanServerAttempt;
import org.apache.accumulo.core.spi.scan.ScanServerSelections;
import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletscan.thrift.ScanServerBusyException;
+import org.apache.accumulo.core.tabletscan.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
-import org.apache.accumulo.core.tabletserver.thrift.ScanServerBusyException;
-import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
import org.apache.accumulo.core.trace.TraceUtil;
import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.OpTimer;
import org.apache.accumulo.core.util.Retry;
import org.apache.thrift.TApplicationException;
@@ -88,6 +85,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.net.HostAndPort;
+
public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value>> {
private static final Logger log = LoggerFactory.getLogger(TabletServerBatchReaderIterator.class);
@@ -160,17 +159,15 @@
log.warn("Failed to add Batch Scan result", e);
}
fatalException = e;
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
};
try {
lookup(ranges, rr);
- } catch (RuntimeException re) {
- throw re;
- } catch (Exception e) {
- throw new RuntimeException("Failed to create iterator", e);
+ } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
+ throw new IllegalStateException("Failed to create iterator", e);
}
}
@@ -196,7 +193,7 @@
if (fatalException instanceof RuntimeException) {
throw (RuntimeException) fatalException;
} else {
- throw new RuntimeException(fatalException);
+ throw new IllegalStateException(fatalException);
}
}
@@ -207,13 +204,14 @@
+ " so that it can be closed when this Iterator is exhausted. Not"
+ " retaining a reference to the BatchScanner guarantees that you are"
+ " leaking threads in your client JVM.", shortMsg);
- throw new RuntimeException(shortMsg + " Ensure proper handling of the BatchScanner.");
+ throw new IllegalStateException(
+ shortMsg + " Ensure proper handling of the BatchScanner.");
}
batchIterator = batch.iterator();
return batch != LAST_BATCH;
} catch (InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
}
@@ -253,9 +251,9 @@
int lastFailureSize = Integer.MAX_VALUE;
- Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
- .incrementBy(100, MILLISECONDS).maxWait(10, SECONDS).backOffFactor(1.07)
- .logInterval(1, MINUTES).createFactory().createRetry();
+ Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+ .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMinutes(10)).backOffFactor(1.07)
+ .logInterval(Duration.ofMinutes(1)).createFactory().createRetry();
while (true) {
@@ -286,7 +284,7 @@
try {
retry.waitForNextAttempt(log, "binRanges retry failures");
} catch (InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
index 980ba04..d54d144 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
@@ -19,8 +19,6 @@
package org.apache.accumulo.core.clientImpl;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static java.util.concurrent.TimeUnit.SECONDS;
import static java.util.function.Function.identity;
import static java.util.stream.Collectors.toList;
import static java.util.stream.Collectors.toMap;
@@ -29,6 +27,7 @@
import java.lang.management.CompilationMXBean;
import java.lang.management.GarbageCollectorMXBean;
import java.lang.management.ManagementFactory;
+import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@@ -61,6 +60,7 @@
import org.apache.accumulo.core.client.TimedOutException;
import org.apache.accumulo.core.clientImpl.TabletLocator.TabletServerMutations;
import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
import org.apache.accumulo.core.constraints.Violations;
import org.apache.accumulo.core.data.ConstraintViolationSummary;
@@ -71,14 +71,12 @@
import org.apache.accumulo.core.dataImpl.TabletIdImpl;
import org.apache.accumulo.core.dataImpl.thrift.TMutation;
import org.apache.accumulo.core.dataImpl.thrift.UpdateErrors;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock;
import org.apache.accumulo.core.rpc.ThriftUtil;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.Retry;
import org.apache.accumulo.core.util.threads.ThreadPools;
import org.apache.accumulo.core.util.threads.Threads;
@@ -90,6 +88,7 @@
import org.slf4j.LoggerFactory;
import com.google.common.base.Joiner;
+import com.google.common.net.HostAndPort;
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.context.Scope;
@@ -506,7 +505,7 @@
wait();
}
} catch (InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
@@ -949,13 +948,13 @@
// happen after the batch writer closes. See #3721
try {
final HostAndPort parsedServer = HostAndPort.fromString(location);
- final TabletClientService.Iface client;
+ final TabletIngestClientService.Iface client;
if (timeoutTracker.getTimeOut() < context.getClientTimeoutInMillis()) {
- client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context,
+ client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context,
timeoutTracker.getTimeOut());
} else {
- client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
+ client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context);
}
try {
@@ -1093,9 +1092,9 @@
private void cancelSession() throws InterruptedException, ThriftSecurityException {
- Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
- .incrementBy(100, MILLISECONDS).maxWait(60, SECONDS).backOffFactor(1.5)
- .logInterval(3, MINUTES).createRetry();
+ Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+ .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMinutes(1)).backOffFactor(1.5)
+ .logInterval(Duration.ofMinutes(3)).createRetry();
final HostAndPort parsedServer = HostAndPort.fromString(location);
@@ -1108,7 +1107,7 @@
// exceptions.
while (!somethingFailed.get()) {
- TabletClientService.Client client = null;
+ TabletIngestClientService.Client client = null;
// Check if a lock is held by any tserver at the host and port. It does not need to be the
// exact tserver instance that existed when the session was created because if a new
@@ -1123,10 +1122,10 @@
try {
if (timeout < context.getClientTimeoutInMillis()) {
- client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context,
+ client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context,
timeout);
} else {
- client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
+ client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context);
}
if (useCloseUpdate) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
index c76277f..1402732 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
@@ -19,9 +19,9 @@
package org.apache.accumulo.core.clientImpl;
import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import java.io.IOException;
-import java.security.SecureRandom;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
@@ -34,7 +34,6 @@
import java.util.Set;
import java.util.SortedMap;
import java.util.SortedSet;
-import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import java.util.stream.Collectors;
@@ -46,6 +45,7 @@
import org.apache.accumulo.core.client.TimedOutException;
import org.apache.accumulo.core.client.sample.SamplerConfiguration;
import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.data.Column;
@@ -69,15 +69,13 @@
import org.apache.accumulo.core.spi.scan.ScanServerAttempt;
import org.apache.accumulo.core.spi.scan.ScanServerSelections;
import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletscan.thrift.ScanServerBusyException;
+import org.apache.accumulo.core.tabletscan.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletscan.thrift.TooManyFilesException;
import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
-import org.apache.accumulo.core.tabletserver.thrift.ScanServerBusyException;
-import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.OpTimer;
import org.apache.accumulo.core.util.Retry;
import org.apache.hadoop.io.Text;
@@ -86,6 +84,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.net.HostAndPort;
+
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.context.Scope;
@@ -105,7 +105,6 @@
// metadata
public static final Map<TabletType,Set<String>> serversWaitedForWrites =
new EnumMap<>(TabletType.class);
- private static final SecureRandom random = new SecureRandom();
static {
for (TabletType ttype : TabletType.values()) {
@@ -270,9 +269,9 @@
static <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, Duration maxWaitTime,
String description, Duration timeoutLeft, ClientContext context, TableId tableId,
Logger log) {
- Retry retry = Retry.builder().infiniteRetries().retryAfter(100, TimeUnit.MILLISECONDS)
- .incrementBy(100, TimeUnit.MILLISECONDS).maxWait(1, SECONDS).backOffFactor(1.5)
- .logInterval(3, TimeUnit.MINUTES).createRetry();
+ Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+ .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofSeconds(1)).backOffFactor(1.5)
+ .logInterval(Duration.ofMinutes(3)).createRetry();
long startTime = System.nanoTime();
Optional<T> optional = condition.get();
@@ -319,7 +318,7 @@
Thread.sleep(millis);
}
// wait 2 * last time, with +-10% random jitter
- return (long) (Math.min(millis * 2, maxSleep) * (.9 + random.nextDouble() / 5));
+ return (long) (Math.min(millis * 2, maxSleep) * (.9 + RANDOM.get().nextDouble() / 5));
}
public static List<KeyValue> scan(ClientContext context, ScanState scanState, long timeOut)
@@ -374,18 +373,18 @@
} else {
// when a tablet splits we do want to continue scanning the low child
// of the split if we are already passed it
- Range dataRange = loc.tablet_extent.toDataRange();
+ Range dataRange = loc.getExtent().toDataRange();
if (scanState.range.getStartKey() != null
&& dataRange.afterEndKey(scanState.range.getStartKey())) {
// go to the next tablet
- scanState.startRow = loc.tablet_extent.endRow();
+ scanState.startRow = loc.getExtent().endRow();
scanState.skipStartRow = true;
loc = null;
} else if (scanState.range.getEndKey() != null
&& dataRange.beforeStartKey(scanState.range.getEndKey())) {
// should not happen
- throw new RuntimeException("Unexpected tablet, extent : " + loc.tablet_extent
+ throw new IllegalStateException("Unexpected tablet, extent : " + loc.getExtent()
+ " range : " + scanState.range + " startRow : " + scanState.startRow);
}
}
@@ -411,7 +410,7 @@
}
Span child2 = TraceUtil.startSpan(ThriftScanner.class, "scan::location",
- Map.of("tserver", loc.tablet_location));
+ Map.of("tserver", loc.getTserverLocation()));
try (Scope scanLocation = child2.makeCurrent()) {
results = scan(loc, scanState, context, timeOut, startTime);
} catch (AccumuloSecurityException e) {
@@ -422,7 +421,7 @@
throw e;
} catch (TApplicationException tae) {
TraceUtil.setException(child2, tae, true);
- throw new AccumuloServerException(scanState.getErrorLocation().tablet_location, tae);
+ throw new AccumuloServerException(scanState.getErrorLocation().getTserverLocation(), tae);
} catch (TSampleNotPresentException tsnpe) {
String message = "Table " + context.getPrintableTableInfoFromId(scanState.tableId)
+ " does not have sampling configured or built";
@@ -437,7 +436,7 @@
}
lastError = error;
- TabletLocator.getLocator(context, scanState.tableId).invalidateCache(loc.tablet_extent);
+ TabletLocator.getLocator(context, scanState.tableId).invalidateCache(loc.getExtent());
loc = null;
// no need to try the current scan id somewhere else
@@ -512,7 +511,7 @@
sleepMillis = pause(sleepMillis, maxSleepTime, scanState.runOnScanServer);
} catch (TException e) {
TabletLocator.getLocator(context, scanState.tableId).invalidateCache(context,
- loc.tablet_location);
+ loc.getTserverLocation());
error = "Scan failed, thrift error " + e.getClass().getName() + " " + e.getMessage()
+ " " + scanState.getErrorLocation();
if (!error.equals(lastError)) {
@@ -564,17 +563,17 @@
TabletLocation newLoc;
- var tabletId = new TabletIdImpl(loc.tablet_extent);
+ var tabletId = new TabletIdImpl(loc.getExtent());
if (scanState.scanID != null && scanState.prevLoc != null
- && scanState.prevLoc.tablet_session.equals("scan_server")
- && scanState.prevLoc.tablet_extent.equals(loc.tablet_extent)) {
+ && scanState.prevLoc.getTserverSession().equals("scan_server")
+ && scanState.prevLoc.getExtent().equals(loc.getExtent())) {
// this is the case of continuing a scan on a scan server for the same tablet, so lets not
// call the scan server selector and just go back to the previous scan server
newLoc = scanState.prevLoc;
log.trace(
"For tablet {} continuing scan on scan server {} without consulting scan server selector, using busyTimeout {}",
- loc.tablet_extent, newLoc.tablet_location, scanState.busyTimeout);
+ loc.getExtent(), newLoc.getTserverLocation(), scanState.busyTimeout);
} else {
// obtain a snapshot once and only expose this snapshot to the plugin for consistency
var attempts = scanState.scanAttempts.snapshot();
@@ -606,7 +605,7 @@
public <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, Duration maxWaitTime,
String description) {
return ThriftScanner.waitUntil(condition, maxWaitTime, description, timeoutLeft,
- context, loc.tablet_extent.tableId(), log);
+ context, loc.getExtent().tableId(), log);
}
};
@@ -616,19 +615,18 @@
String scanServer = actions.getScanServer(tabletId);
if (scanServer != null) {
- newLoc = new TabletLocation(loc.tablet_extent, scanServer, "scan_server");
+ newLoc = new TabletLocation(loc.getExtent(), scanServer, "scan_server");
delay = actions.getDelay();
scanState.busyTimeout = actions.getBusyTimeout();
log.trace(
"For tablet {} using hints {} scan server selector chose scan_server:{} delay:{} busyTimeout:{}",
- loc.tablet_extent, scanState.executionHints, scanServer, delay,
- scanState.busyTimeout);
+ loc.getExtent(), scanState.executionHints, scanServer, delay, scanState.busyTimeout);
} else {
newLoc = loc;
delay = actions.getDelay();
scanState.busyTimeout = Duration.ZERO;
log.trace("For tablet {} using hints {} scan server selector chose tablet_server",
- loc.tablet_extent, scanState.executionHints);
+ loc.getExtent(), scanState.executionHints);
}
if (!delay.isZero()) {
@@ -641,7 +639,7 @@
}
}
- var reporter = scanState.scanAttempts.createReporter(newLoc.tablet_location, tabletId);
+ var reporter = scanState.scanAttempts.createReporter(newLoc.getTserverLocation(), tabletId);
try {
return scanRpc(newLoc, scanState, context, scanState.busyTimeout.toMillis());
@@ -666,7 +664,7 @@
final TInfo tinfo = TraceUtil.traceInfo();
- final HostAndPort parsedLocation = HostAndPort.fromString(loc.tablet_location);
+ final HostAndPort parsedLocation = HostAndPort.fromString(loc.getTserverLocation());
TabletScanClientService.Client client =
ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedLocation, context);
@@ -681,23 +679,24 @@
scanState.prevLoc = loc;
if (scanState.scanID == null) {
- Thread.currentThread().setName("Starting scan tserver=" + loc.tablet_location + " tableId="
- + loc.tablet_extent.tableId());
+ Thread.currentThread().setName("Starting scan tserver=" + loc.getTserverLocation()
+ + " tableId=" + loc.getExtent().tableId());
if (log.isTraceEnabled()) {
- String msg = "Starting scan tserver=" + loc.tablet_location + " tablet="
- + loc.tablet_extent + " range=" + scanState.range + " ssil="
+ String msg = "Starting scan tserver=" + loc.getTserverLocation() + " tablet="
+ + loc.getExtent() + " range=" + scanState.range + " ssil="
+ scanState.serverSideIteratorList + " ssio=" + scanState.serverSideIteratorOptions
+ " context=" + scanState.classLoaderContext;
log.trace("tid={} {}", Thread.currentThread().getId(), msg);
timer = new OpTimer().start();
}
- TabletType ttype = TabletType.type(loc.tablet_extent);
- boolean waitForWrites = !serversWaitedForWrites.get(ttype).contains(loc.tablet_location);
+ TabletType ttype = TabletType.type(loc.getExtent());
+ boolean waitForWrites =
+ !serversWaitedForWrites.get(ttype).contains(loc.getTserverLocation());
InitialScan is = client.startScan(tinfo, scanState.context.rpcCreds(),
- loc.tablet_extent.toThrift(), scanState.range.toThrift(),
+ loc.getExtent().toThrift(), scanState.range.toThrift(),
scanState.columns.stream().map(Column::toThrift).collect(Collectors.toList()),
scanState.size, scanState.serverSideIteratorList, scanState.serverSideIteratorOptions,
scanState.authorizations.getAuthorizationsBB(), waitForWrites, scanState.isolated,
@@ -705,7 +704,7 @@
SamplerConfigurationImpl.toThrift(scanState.samplerConfig), scanState.batchTimeOut,
scanState.classLoaderContext, scanState.executionHints, busyTimeout);
if (waitForWrites) {
- serversWaitedForWrites.get(ttype).add(loc.tablet_location);
+ serversWaitedForWrites.get(ttype).add(loc.getTserverLocation());
}
sr = is.result;
@@ -719,7 +718,7 @@
} else {
// log.debug("Calling continue scan : "+scanState.range+" loc = "+loc);
String msg =
- "Continuing scan tserver=" + loc.tablet_location + " scanid=" + scanState.scanID;
+ "Continuing scan tserver=" + loc.getTserverLocation() + " scanid=" + scanState.scanID;
Thread.currentThread().setName(msg);
if (log.isTraceEnabled()) {
@@ -744,7 +743,7 @@
} else {
// log.debug("No more : tab end row = "+loc.tablet_extent.getEndRow()+" range =
// "+scanState.range);
- if (loc.tablet_extent.endRow() == null) {
+ if (loc.getExtent().endRow() == null) {
scanState.finished = true;
if (timer != null) {
@@ -755,8 +754,8 @@
}
} else if (scanState.range.getEndKey() == null || !scanState.range
- .afterEndKey(new Key(loc.tablet_extent.endRow()).followingKey(PartialKey.ROW))) {
- scanState.startRow = loc.tablet_extent.endRow();
+ .afterEndKey(new Key(loc.getExtent().endRow()).followingKey(PartialKey.ROW))) {
+ scanState.startRow = loc.getExtent().endRow();
scanState.skipStartRow = true;
if (timer != null) {
@@ -803,7 +802,7 @@
TInfo tinfo = TraceUtil.traceInfo();
log.debug("Closing active scan {} {}", scanState.prevLoc, scanState.scanID);
- HostAndPort parsedLocation = HostAndPort.fromString(scanState.prevLoc.tablet_location);
+ HostAndPort parsedLocation = HostAndPort.fromString(scanState.prevLoc.getTserverLocation());
TabletScanClientService.Client client = null;
try {
client =
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java
index f4c7047..0f84154 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java
@@ -25,9 +25,9 @@
import org.apache.accumulo.core.rpc.SaslConnectionParams;
import org.apache.accumulo.core.rpc.SslConnectionParams;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.util.HostAndPort;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.net.HostAndPort;
@VisibleForTesting
public class ThriftTransportKey {
@@ -58,7 +58,7 @@
this.saslParams = saslParams;
if (saslParams != null && sslParams != null) {
// TSasl and TSSL transport factories don't play nicely together
- throw new RuntimeException("Cannot use both SSL and SASL thrift transports");
+ throw new IllegalArgumentException("Cannot use both SSL and SASL thrift transports");
}
this.hash = Objects.hash(type, server, timeout, sslParams, saslParams);
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java
index 6f62125..1863cf6 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java
@@ -20,8 +20,8 @@
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
-import java.security.SecureRandom;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collections;
@@ -43,7 +43,6 @@
import org.apache.accumulo.core.rpc.ThriftUtil;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.Pair;
import org.apache.accumulo.core.util.threads.Threads;
import org.apache.thrift.TConfiguration;
@@ -53,13 +52,13 @@
import org.slf4j.LoggerFactory;
import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
public class ThriftTransportPool {
private static final Logger log = LoggerFactory.getLogger(ThriftTransportPool.class);
- private static final SecureRandom random = new SecureRandom();
private static final long ERROR_THRESHOLD = 20L;
private static final long STUCK_THRESHOLD = MINUTES.toMillis(2);
@@ -134,7 +133,7 @@
if (serversSet.isEmpty()) {
return null;
}
- Collections.shuffle(serversSet, random);
+ Collections.shuffle(serversSet, RANDOM.get());
for (ThriftTransportKey ttk : serversSet) {
CachedConnection connection = connectionPool.reserveAny(ttk);
if (connection != null) {
@@ -244,7 +243,7 @@
try {
checkThread.join();
} catch (InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java
index 2c71bc5..e64ae66 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java
@@ -217,8 +217,6 @@
is.write(dout);
}
- CompactionStrategyConfigUtil.encode(dout, cc);
-
encodeConfigurer(dout, cc.getConfigurer());
encodeSelector(dout, cc.getSelector());
encode(dout, cc.getExecutionHints());
@@ -260,8 +258,6 @@
cc.setIterators(iterators);
- CompactionStrategyConfigUtil.decode(cc, din);
-
var configurer = decodeConfigurer(din);
if (!isDefault(configurer)) {
cc.setConfigurer(configurer);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
deleted file mode 100644
index 9f124ed..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
+++ /dev/null
@@ -1,55 +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
- *
- * https://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.clientImpl;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.metadata.schema.Ample;
-
-public class Writer {
-
- private ClientContext context;
- private TableId tableId;
-
- public Writer(ClientContext context, TableId tableId) {
- checkArgument(context != null, "context is null");
- checkArgument(tableId != null, "tableId is null");
- this.context = context;
- this.tableId = tableId;
- }
-
- public void update(Mutation m) throws AccumuloException, TableNotFoundException {
- checkArgument(m != null, "m is null");
-
- if (m.size() == 0) {
- throw new IllegalArgumentException("Can not add empty mutations");
- }
-
- String table = Ample.DataLevel.of(tableId).metaTable();
-
- try (var writer = context.createBatchWriter(table)) {
- writer.addMutation(m);
- }
-
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
index 23f036b..39c7cec 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
@@ -20,8 +20,8 @@
import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.clientImpl.TabletLocatorImpl.TabletServerLockChecker;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
+import org.apache.accumulo.core.lock.ServiceLock;
public class ZookeeperLockChecker implements TabletServerLockChecker {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
index f13420d..8689a56 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
@@ -19,8 +19,6 @@
package org.apache.accumulo.core.clientImpl.bulk;
import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.stream.Collectors.groupingBy;
import static org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.pathToCacheId;
import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
@@ -28,6 +26,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -74,6 +73,7 @@
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.file.FileOperations;
import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
import org.apache.accumulo.core.spi.crypto.CryptoService;
import org.apache.accumulo.core.util.Retry;
import org.apache.accumulo.core.volume.VolumeConfiguration;
@@ -85,9 +85,9 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
import com.google.common.base.Preconditions;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
import com.google.common.collect.Sets;
public class BulkImport implements ImportDestinationArguments, ImportMappingOptions {
@@ -141,9 +141,9 @@
if (propValue != null) {
maxTablets = Integer.parseInt(propValue);
}
- Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
- .incrementBy(100, MILLISECONDS).maxWait(2, MINUTES).backOffFactor(1.5)
- .logInterval(3, MINUTES).createRetry();
+ Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+ .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMinutes(2)).backOffFactor(1.5)
+ .logInterval(Duration.ofMinutes(3)).createRetry();
// retry if a merge occurs
boolean shouldRetry = true;
@@ -179,7 +179,7 @@
try {
retry.waitForNextAttempt(log, String.format("bulk import to %s(%s)", tableName, tableId));
} catch (InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
log.info(ae.getMessage() + ". Retrying bulk import to " + tableName);
}
@@ -261,9 +261,9 @@
long l;
}
- public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path mapFile,
- long fileSize, Collection<KeyExtent> extents, FileSystem ns, Cache<String,Long> fileLenCache,
- CryptoService cs) throws IOException {
+ public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf,
+ UnreferencedTabletFile dataFile, long fileSize, Collection<KeyExtent> extents, FileSystem ns,
+ Cache<String,Long> fileLenCache, CryptoService cs) throws IOException {
if (extents.size() == 1) {
return Collections.singletonMap(extents.iterator().next(), fileSize);
@@ -277,17 +277,14 @@
Text row = new Text();
- FileSKVIterator index = FileOperations.getInstance().newIndexReaderBuilder()
- .forFile(mapFile.toString(), ns, ns.getConf(), cs).withTableConfiguration(acuConf)
- .withFileLenCache(fileLenCache).build();
-
- try {
+ try (FileSKVIterator index =
+ FileOperations.getInstance().newIndexReaderBuilder().forFile(dataFile, ns, ns.getConf(), cs)
+ .withTableConfiguration(acuConf).withFileLenCache(fileLenCache).build()) {
while (index.hasTop()) {
Key key = index.getTopKey();
totalIndexEntries++;
key.getRow(row);
- // TODO this could use a binary search
for (Entry<KeyExtent,MLong> entry : counts.entrySet()) {
if (entry.getKey().contains(row)) {
entry.getValue().l++;
@@ -296,14 +293,6 @@
index.next();
}
- } finally {
- try {
- if (index != null) {
- index.close();
- }
- } catch (IOException e) {
- log.debug("Failed to close " + mapFile, e);
- }
}
Map<KeyExtent,Long> results = new TreeMap<>();
@@ -354,12 +343,11 @@
}
public static List<KeyExtent> findOverlappingTablets(ClientContext context,
- KeyExtentCache extentCache, Path file, FileSystem fs, Cache<String,Long> fileLenCache,
- CryptoService cs) throws IOException {
+ KeyExtentCache extentCache, UnreferencedTabletFile file, FileSystem fs,
+ Cache<String,Long> fileLenCache, CryptoService cs) throws IOException {
try (FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
- .forFile(file.toString(), fs, fs.getConf(), cs)
- .withTableConfiguration(context.getConfiguration()).withFileLenCache(fileLenCache)
- .seekToBeginning().build()) {
+ .forFile(file, fs, fs.getConf(), cs).withTableConfiguration(context.getConfiguration())
+ .withFileLenCache(fileLenCache).seekToBeginning().build()) {
return findOverlappingTablets(extentCache, reader);
}
}
@@ -379,7 +367,7 @@
Map<String,Long> absFileLens = new HashMap<>();
fileLens.forEach((k, v) -> absFileLens.put(pathToCacheId(new Path(dir, k)), v));
- Cache<String,Long> fileLenCache = CacheBuilder.newBuilder().build();
+ Cache<String,Long> fileLenCache = Caffeine.newBuilder().build();
fileLenCache.putAll(absFileLens);
@@ -411,13 +399,7 @@
fileDestinations.values().stream().flatMap(List::stream)
.filter(dest -> dest.getRangeType() == RangeType.FILE)
.flatMap(dest -> Stream.of(dest.getStartRow(), dest.getEndRow())).filter(Objects::nonNull)
- .map(Text::new).sorted().distinct().forEach(row -> {
- try {
- extentCache.lookup(row);
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- });
+ .map(Text::new).sorted().distinct().forEach(extentCache::lookup);
SortedMap<KeyExtent,Files> mapping = new TreeMap<>();
@@ -548,22 +530,22 @@
context.instanceOperations().getSystemConfiguration(), tableProps, tableId);
for (FileStatus fileStatus : files) {
- Path filePath = fileStatus.getPath();
+ UnreferencedTabletFile file = UnreferencedTabletFile.of(fs, fileStatus.getPath());
CompletableFuture<Map<KeyExtent,Bulk.FileInfo>> future = CompletableFuture.supplyAsync(() -> {
try {
long t1 = System.currentTimeMillis();
List<KeyExtent> extents =
- findOverlappingTablets(context, extentCache, filePath, fs, fileLensCache, cs);
+ findOverlappingTablets(context, extentCache, file, fs, fileLensCache, cs);
// make sure file isn't going to too many tablets
- checkTabletCount(maxTablets, extents.size(), filePath.toString());
- Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), filePath,
+ checkTabletCount(maxTablets, extents.size(), file.toString());
+ Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), file,
fileStatus.getLen(), extents, fs, fileLensCache, cs);
Map<KeyExtent,Bulk.FileInfo> pathLocations = new HashMap<>();
for (KeyExtent ke : extents) {
- pathLocations.put(ke, new Bulk.FileInfo(filePath, estSizes.getOrDefault(ke, 0L)));
+ pathLocations.put(ke, new Bulk.FileInfo(file.getPath(), estSizes.getOrDefault(ke, 0L)));
}
long t2 = System.currentTimeMillis();
- log.debug("Mapped {} to {} tablets in {}ms", filePath, pathLocations.size(), t2 - t1);
+ log.debug("Mapped {} to {} tablets in {}ms", file, pathLocations.size(), t2 - t1);
return pathLocations;
} catch (Exception e) {
throw new CompletionException(e);
@@ -581,9 +563,9 @@
pathMapping.forEach((ext, fi) -> mappings.computeIfAbsent(ext, k -> new Files()).add(fi));
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
} catch (ExecutionException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
@@ -606,7 +588,7 @@
if (ke.contains(oke)) {
mappings.get(ke).merge(mappings.remove(oke));
} else if (!oke.contains(ke)) {
- throw new RuntimeException("Error during bulk import: Unable to merge overlapping "
+ throw new IllegalStateException("Error during bulk import: Unable to merge overlapping "
+ "tablets where neither tablet contains the other. This may be caused by "
+ "a concurrent merge. Key extents " + oke + " and " + ke + " overlap, but "
+ "neither contains the other.");
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
index 0f8d46c..533a43c 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
@@ -19,6 +19,7 @@
package org.apache.accumulo.core.clientImpl.bulk;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
import java.io.BufferedReader;
import java.io.BufferedWriter;
@@ -27,10 +28,6 @@
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.io.OutputStreamWriter;
-import java.lang.reflect.Type;
-import java.util.Base64;
-import java.util.Base64.Decoder;
-import java.util.Base64.Encoder;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
@@ -41,17 +38,10 @@
import org.apache.accumulo.core.clientImpl.bulk.Bulk.Mapping;
import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.util.json.ByteArrayToBase64TypeAdapter;
import org.apache.hadoop.fs.Path;
import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonDeserializationContext;
-import com.google.gson.JsonDeserializer;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonParseException;
-import com.google.gson.JsonPrimitive;
-import com.google.gson.JsonSerializationContext;
-import com.google.gson.JsonSerializer;
import com.google.gson.reflect.TypeToken;
import com.google.gson.stream.JsonWriter;
@@ -60,28 +50,7 @@
*/
public class BulkSerialize {
- private static class ByteArrayToBase64TypeAdapter
- implements JsonSerializer<byte[]>, JsonDeserializer<byte[]> {
-
- Decoder decoder = Base64.getUrlDecoder();
- Encoder encoder = Base64.getUrlEncoder();
-
- @Override
- public byte[] deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context)
- throws JsonParseException {
- return decoder.decode(json.getAsString());
- }
-
- @Override
- public JsonElement serialize(byte[] src, Type typeOfSrc, JsonSerializationContext context) {
- return new JsonPrimitive(encoder.encodeToString(src));
- }
- }
-
- static Gson createGson() {
- return new GsonBuilder()
- .registerTypeHierarchyAdapter(byte[].class, new ByteArrayToBase64TypeAdapter()).create();
- }
+ private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
public interface Output {
OutputStream create(Path path) throws IOException;
@@ -100,7 +69,6 @@
try (OutputStream fsOut = output.create(lmFile); JsonWriter writer =
new JsonWriter(new BufferedWriter(new OutputStreamWriter(fsOut, UTF_8)))) {
- Gson gson = createGson();
writer.setIndent(" ");
writer.beginArray();
Set<Entry<KeyExtent,Files>> es = loadMapping.entrySet();
@@ -130,7 +98,7 @@
final Path renamingFile = new Path(bulkDir, Constants.BULK_RENAME_FILE);
try (OutputStream fsOut = output.create(renamingFile);
BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fsOut))) {
- new Gson().toJson(oldToNewNameMap, writer);
+ GSON.get().toJson(oldToNewNameMap, writer);
}
}
@@ -141,7 +109,6 @@
public static Map<String,String> readRenameMap(String bulkDir, Input input) throws IOException {
final Path renamingFile = new Path(bulkDir, Constants.BULK_RENAME_FILE);
Map<String,String> oldToNewNameMap;
- Gson gson = createGson();
try (InputStream fis = input.open(renamingFile);
BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) {
oldToNewNameMap = gson.fromJson(reader, new TypeToken<Map<String,String>>() {}.getType());
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
index f17639a..e338fe9 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
@@ -23,7 +23,6 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
-import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
import java.util.Objects;
@@ -88,9 +87,9 @@
@VisibleForTesting
protected Stream<KeyExtent> lookupExtents(Text row) {
- return TabletsMetadata.builder(ctx).forTable(tableId).overlapping(row, true, null)
- .checkConsistency().fetch(PREV_ROW).build().stream().limit(100)
- .map(TabletMetadata::getExtent);
+ TabletsMetadata tabletsMetadata = TabletsMetadata.builder(ctx).forTable(tableId)
+ .overlapping(row, true, null).checkConsistency().fetch(PREV_ROW).build();
+ return tabletsMetadata.stream().limit(100).map(TabletMetadata::getExtent);
}
@Override
@@ -129,15 +128,8 @@
for (Text lookupRow : lookupRows) {
if (getFromCache(lookupRow) == null) {
while (true) {
- try {
- Iterator<KeyExtent> iter = lookupExtents(lookupRow).iterator();
- while (iter.hasNext()) {
- KeyExtent ke2 = iter.next();
- if (inCache(ke2)) {
- break;
- }
- updateCache(ke2);
- }
+ try (Stream<KeyExtent> keyExtentStream = lookupExtents(lookupRow)) {
+ keyExtentStream.takeWhile(ke2 -> !inCache(ke2)).forEach(this::updateCache);
break;
} catch (TabletDeletedException tde) {
// tablets were merged away in the table, start over and try again
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
index 30e1041..36b2d5b 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
@@ -19,7 +19,6 @@
package org.apache.accumulo.core.clientImpl.bulk;
import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.clientImpl.bulk.BulkSerialize.createGson;
import java.io.BufferedReader;
import java.io.IOException;
@@ -32,6 +31,7 @@
import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.util.json.ByteArrayToBase64TypeAdapter;
import com.google.gson.Gson;
import com.google.gson.stream.JsonReader;
@@ -41,14 +41,14 @@
*/
public class LoadMappingIterator
implements Iterator<Map.Entry<KeyExtent,Bulk.Files>>, AutoCloseable {
- private TableId tableId;
- private JsonReader reader;
- private Gson gson = createGson();
+ private final TableId tableId;
+ private final JsonReader reader;
+ private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
private Map<String,String> renameMap;
- LoadMappingIterator(TableId tableId, InputStream loadMapFile) throws IOException {
+ LoadMappingIterator(TableId tableId, InputStream loadMappingFile) throws IOException {
this.tableId = tableId;
- this.reader = new JsonReader(new BufferedReader(new InputStreamReader(loadMapFile, UTF_8)));
+ this.reader = new JsonReader(new BufferedReader(new InputStreamReader(loadMappingFile, UTF_8)));
this.reader.beginArray();
}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java
deleted file mode 100644
index db69081..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java
+++ /dev/null
@@ -1,52 +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
- *
- * https://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.clientImpl.mapred;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.hadoop.mapred.InputSplit;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * The Class BatchInputSplit. Encapsulates Accumulo ranges for use in Map Reduce jobs. Can contain
- * several Ranges per InputSplit.
- *
- * @deprecated since 2.0.0
- */
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
- justification = "Intended to share code between mapred and mapreduce")
-@Deprecated(since = "2.0.0")
-public class BatchInputSplit extends org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit
- implements InputSplit {
-
- public BatchInputSplit() {}
-
- public BatchInputSplit(BatchInputSplit split) throws IOException {
- super(split);
- }
-
- public BatchInputSplit(String table, TableId tableId, Collection<Range> ranges,
- String[] location) {
- super(table, tableId, ranges, location);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java
deleted file mode 100644
index cc01471..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java
+++ /dev/null
@@ -1,200 +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
- *
- * https://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.clientImpl.mapreduce;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-
-/**
- * The Class BatchInputSplit. Encapsulates a set of Accumulo ranges on a single tablet for use in
- * Map Reduce jobs. Can contain several Ranges per split.
- *
- * @deprecated since 2.0.0
- */
-@Deprecated(since = "2.0.0")
-public class BatchInputSplit extends org.apache.accumulo.core.client.mapreduce.RangeInputSplit {
- private Collection<Range> ranges;
- private float[] rangeProgress = null;
-
- public BatchInputSplit() {
- ranges = Collections.emptyList();
- }
-
- public BatchInputSplit(BatchInputSplit split) throws IOException {
- super(split);
- this.setRanges(split.getRanges());
- }
-
- public BatchInputSplit(String table, TableId tableId, Collection<Range> ranges,
- String[] locations) {
- super(table, tableId.canonical(), new Range(), locations);
- this.ranges = ranges;
- }
-
- /**
- * Save progress on each call to this function, implied by value of currentKey, and return average
- * ranges in the split
- */
- @Override
- public float getProgress(Key currentKey) {
- if (rangeProgress == null) {
- rangeProgress = new float[ranges.size()];
- }
-
- float total = 0; // progress per range could be on different scales, this number is "fuzzy"
-
- if (currentKey == null) {
- for (float progress : rangeProgress) {
- total += progress;
- }
- } else {
- int i = 0;
- for (Range range : ranges) {
- if (range.contains(currentKey)) {
- // find the current range and report as if that is the single range
- if (range.getStartKey() != null && range.getEndKey() != null) {
- if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
- // just look at the row progress
- rangeProgress[i] = SplitUtils.getProgress(range.getStartKey().getRowData(),
- range.getEndKey().getRowData(), currentKey.getRowData());
- } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM)
- != 0) {
- // just look at the column family progress
- rangeProgress[i] = SplitUtils.getProgress(range.getStartKey().getColumnFamilyData(),
- range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
- } else if (range.getStartKey().compareTo(range.getEndKey(),
- PartialKey.ROW_COLFAM_COLQUAL) != 0) {
- // just look at the column qualifier progress
- rangeProgress[i] = SplitUtils.getProgress(
- range.getStartKey().getColumnQualifierData(),
- range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
- }
- }
- total += rangeProgress[i];
- }
- i++;
- }
- }
-
- return total / ranges.size();
- }
-
- /**
- * 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() {
- long sum = 0;
- for (Range range : ranges) {
- sum += SplitUtils.getRangeLength(range);
- }
- return sum;
- }
-
- @Override
- public void readFields(DataInput in) throws IOException {
- super.readFields(in);
-
- int numRanges = in.readInt();
- ranges = new ArrayList<>(numRanges);
- for (int i = 0; i < numRanges; ++i) {
- Range r = new Range();
- r.readFields(in);
- ranges.add(r);
- }
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
- super.write(out);
-
- out.writeInt(ranges.size());
- for (Range r : ranges) {
- r.write(out);
- }
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder(256);
- sb.append("BatchInputSplit:");
- sb.append(" Ranges: ").append(Arrays.asList(ranges));
- sb.append(super.toString());
- return sb.toString();
- }
-
- public void setRanges(Collection<Range> ranges) {
- this.ranges = ranges;
- }
-
- public Collection<Range> getRanges() {
- return ranges;
- }
-
- @Override
- public Range getRange() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void setRange(Range range) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Boolean isIsolatedScan() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void setIsolatedScan(Boolean isolatedScan) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Boolean isOffline() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void setOffline(Boolean offline) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Boolean usesLocalIterators() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void setUsesLocalIterators(Boolean localIterators) {
- throw new UnsupportedOperationException();
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/DelegationTokenStub.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/DelegationTokenStub.java
deleted file mode 100644
index 7871624..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/DelegationTokenStub.java
+++ /dev/null
@@ -1,83 +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
- *
- * https://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.clientImpl.mapreduce;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-
-/**
- * An internal stub class for passing DelegationToken information out of the Configuration back up
- * to the appropriate implementation for mapreduce or mapred.
- *
- * @deprecated since 2.0.0
- */
-@Deprecated(since = "2.0.0")
-public class DelegationTokenStub implements AuthenticationToken {
-
- private String serviceName;
-
- public DelegationTokenStub(String serviceName) {
- requireNonNull(serviceName);
- this.serviceName = serviceName;
- }
-
- public String getServiceName() {
- return serviceName;
- }
-
- @Override
- public void write(DataOutput out) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void readFields(DataInput in) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void destroy() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public boolean isDestroyed() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void init(Properties properties) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Set<TokenProperty> getProperties() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public AuthenticationToken clone() {
- throw new UnsupportedOperationException();
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/SplitUtils.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/SplitUtils.java
deleted file mode 100644
index 9634659..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/SplitUtils.java
+++ /dev/null
@@ -1,90 +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
- *
- * https://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.clientImpl.mapreduce;
-
-import java.math.BigInteger;
-
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Range;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-
-/**
- * @deprecated since 2.0.0
- */
-@Deprecated(since = "2.0.0")
-public class SplitUtils {
-
- /**
- * Central place to set common split configuration not handled by split constructors. The
- * intention is to make it harder to miss optional setters in future refactor.
- */
- public static void updateSplit(org.apache.accumulo.core.client.mapreduce.RangeInputSplit split,
- org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig, Level logLevel) {
- split.setFetchedColumns(tableConfig.getFetchedColumns());
- split.setIterators(tableConfig.getIterators());
- split.setLogLevel(logLevel);
- split.setSamplerConfiguration(tableConfig.getSamplerConfiguration());
- }
-
- 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(SplitUtils.extractBytes(start, maxDepth));
- BigInteger endBI = new BigInteger(SplitUtils.extractBytes(end, maxDepth));
- BigInteger positionBI = new BigInteger(SplitUtils.extractBytes(position, maxDepth));
- return (float) (positionBI.subtract(startBI).doubleValue()
- / endBI.subtract(startBI).doubleValue());
- }
-
- public static long getRangeLength(Range range) {
- 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;
- }
-
- 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;
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java
deleted file mode 100644
index 6d4b3c6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java
+++ /dev/null
@@ -1,520 +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
- *
- * https://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.clientImpl.mapreduce.lib;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Base64;
-import java.util.Scanner;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Credentials;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-@SuppressWarnings("deprecation")
-public class ConfiguratorBase {
-
- protected static final Logger log = Logger.getLogger(ConfiguratorBase.class);
-
- /**
- * Specifies that connection info was configured
- *
- * @since 1.6.0
- */
- public enum ConnectorInfo {
- IS_CONFIGURED, PRINCIPAL, TOKEN
- }
-
- public enum TokenSource {
- FILE, INLINE, JOB;
-
- private String prefix;
-
- private TokenSource() {
- prefix = name().toLowerCase() + ":";
- }
-
- public String prefix() {
- return prefix;
- }
- }
-
- /**
- * Configuration keys for available Instance types.
- *
- * @since 1.6.0
- */
- public enum InstanceOpts {
- TYPE, NAME, ZOO_KEEPERS, CLIENT_CONFIG
- }
-
- /**
- * Configuration keys for general configuration options.
- *
- * @since 1.6.0
- */
- public enum GeneralOpts {
- LOG_LEVEL, VISIBILITY_CACHE_SIZE
- }
-
- /**
- * Provides a configuration key for a given feature enum, prefixed by the implementingClass
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param e the enum used to provide the unique part of the configuration key
- * @return the configuration key
- * @since 1.6.0
- */
- protected static String enumToConfKey(Class<?> implementingClass, Enum<?> e) {
- return implementingClass.getSimpleName() + "." + e.getDeclaringClass().getSimpleName() + "."
- + StringUtils.camelize(e.name().toLowerCase());
- }
-
- /**
- * Provides a configuration key for a given feature enum.
- *
- * @param e the enum used to provide the unique part of the configuration key
- * @return the configuration key
- */
- protected static String enumToConfKey(Enum<?> e) {
- return e.getDeclaringClass().getSimpleName() + "."
- + StringUtils.camelize(e.name().toLowerCase());
- }
-
- /**
- * 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 implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @param principal a valid Accumulo user name
- * @param token the user's password
- * @since 1.6.0
- */
- public static void setConnectorInfo(Class<?> implementingClass, Configuration conf,
- String principal, AuthenticationToken token) {
- if (isConnectorInfoSet(implementingClass, conf)) {
- throw new IllegalStateException("Connector info for " + implementingClass.getSimpleName()
- + " can only be set once per job");
- }
- checkArgument(principal != null, "principal is null");
- checkArgument(token != null, "token is null");
- conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
- conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
- if (token instanceof DelegationTokenImpl) {
- // Avoid serializing the DelegationToken secret in the configuration -- the Job will do that
- // work for us securely
- DelegationTokenImpl delToken = (DelegationTokenImpl) token;
- conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), TokenSource.JOB.prefix()
- + token.getClass().getName() + ":" + delToken.getServiceName().toString());
- } else {
- conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN),
- TokenSource.INLINE.prefix() + token.getClass().getName() + ":"
- + Base64.getEncoder().encodeToString(AuthenticationTokenSerializer.serialize(token)));
- }
- }
-
- private static String cachedTokenFileName(Class<?> implementingClass) {
- return implementingClass.getSimpleName() + ".tokenfile";
- }
-
- /**
- * Sets the connector information needed to communicate with Accumulo in this job.
- *
- * <p>
- * Pulls a token file into the Distributed Cache that contains the authentication token in an
- * attempt to be more secure than storing the password in the Configuration. Token file created
- * with "bin/accumulo create-token".
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @param principal a valid Accumulo user name
- * @param tokenFile the path to the token file in DFS
- * @since 1.6.0
- */
- public static void setConnectorInfo(Class<?> implementingClass, Configuration conf,
- String principal, String tokenFile) {
- if (isConnectorInfoSet(implementingClass, conf)) {
- throw new IllegalStateException("Connector info for " + implementingClass.getSimpleName()
- + " can only be set once per job");
- }
-
- checkArgument(principal != null, "principal is null");
- checkArgument(tokenFile != null, "tokenFile is null");
-
- DistributedCacheHelper.addCacheFile(tokenFile, cachedTokenFileName(implementingClass), conf);
-
- conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
- conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
- conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN),
- TokenSource.FILE.prefix() + tokenFile);
- }
-
- /**
- * Determines if the connector info has already been set for this instance.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @return true if the connector info has already been set, false otherwise
- * @since 1.6.0
- * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
- */
- public static Boolean isConnectorInfoSet(Class<?> implementingClass, Configuration conf) {
- return conf.getBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), false);
- }
-
- /**
- * Gets the user name from the configuration.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @return the principal
- * @since 1.6.0
- * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
- */
- public static String getPrincipal(Class<?> implementingClass, Configuration conf) {
- return conf.get(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL));
- }
-
- /**
- * Gets the authenticated token from either the specified token file or directly from the
- * configuration, whichever was used when the job was configured.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @return the principal's authentication token
- * @since 1.6.0
- * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
- * @see #setConnectorInfo(Class, Configuration, String, String)
- */
- public static AuthenticationToken getAuthenticationToken(Class<?> implementingClass,
- Configuration conf) {
- String token = conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN));
- if (token == null || token.isEmpty()) {
- return null;
- }
- if (token.startsWith(TokenSource.INLINE.prefix())) {
- String[] args = token.substring(TokenSource.INLINE.prefix().length()).split(":", 2);
- if (args.length == 2) {
- return AuthenticationTokenSerializer.deserialize(args[0],
- Base64.getDecoder().decode(args[1]));
- }
- } else if (token.startsWith(TokenSource.FILE.prefix())) {
- String tokenFileName = token.substring(TokenSource.FILE.prefix().length());
- return getTokenFromFile(implementingClass, conf, getPrincipal(implementingClass, conf),
- tokenFileName);
- } else if (token.startsWith(TokenSource.JOB.prefix())) {
- String[] args = token.substring(TokenSource.JOB.prefix().length()).split(":", 2);
- if (args.length == 2) {
- String className = args[0], serviceName = args[1];
- if (DelegationTokenImpl.class.getName().equals(className)) {
- return new org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub(serviceName);
- }
- }
- }
-
- throw new IllegalStateException("Token was not properly serialized into the configuration");
- }
-
- /**
- * Reads from the token file in distributed cache. Currently, the token file stores data separated
- * by colons e.g. principal:token_class:token
- *
- * @param conf the Hadoop context for the configured job
- * @return path to the token file as a String
- * @since 1.6.0
- * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
- */
- public static AuthenticationToken getTokenFromFile(Class<?> implementingClass, Configuration conf,
- String principal, String tokenFile) {
-
- try (InputStream inputStream = DistributedCacheHelper.openCachedFile(tokenFile,
- cachedTokenFileName(implementingClass), conf)) {
-
- try (Scanner fileScanner = new Scanner(inputStream, UTF_8)) {
- while (fileScanner.hasNextLine()) {
- Credentials creds = Credentials.deserialize(fileScanner.nextLine());
- if (principal.equals(creds.getPrincipal())) {
- return creds.getToken();
- }
- }
- throw new IllegalArgumentException("No token found for " + principal);
- }
-
- } catch (IOException e) {
- throw new IllegalStateException("Error closing token file stream", e);
- }
-
- }
-
- /**
- * Configures a ZooKeeperInstance for this job.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @param clientConfig client configuration for specifying connection timeouts, SSL connection
- * options, etc.
- * @since 1.6.0
- */
- public static void setZooKeeperInstance(Class<?> implementingClass, Configuration conf,
- org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
- String key = enumToConfKey(implementingClass, InstanceOpts.TYPE);
- if (!conf.get(key, "").isEmpty()) {
- throw new IllegalStateException(
- "Instance info can only be set once per job; it has already been configured with "
- + conf.get(key));
- }
- conf.set(key, "ZooKeeperInstance");
- if (clientConfig != null) {
- conf.set(enumToConfKey(implementingClass, InstanceOpts.CLIENT_CONFIG),
- clientConfig.serialize());
- }
- }
-
- /**
- * Initializes an Accumulo Instance based on the configuration.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @return an Accumulo instance
- * @since 1.6.0
- */
- public static org.apache.accumulo.core.client.Instance getInstance(Class<?> implementingClass,
- Configuration conf) {
- String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE), "");
- if ("ZooKeeperInstance".equals(instanceType)) {
- return new org.apache.accumulo.core.client.ZooKeeperInstance(
- getClientConfiguration(implementingClass, conf));
- } else if (instanceType.isEmpty()) {
- throw new IllegalStateException(
- "Instance has not been configured for " + implementingClass.getSimpleName());
- } else {
- throw new IllegalStateException("Unrecognized instance type " + instanceType);
- }
- }
-
- /**
- * Obtain a ClientConfiguration based on the configuration.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- *
- * @return A ClientConfiguration
- * @since 1.7.0
- */
- public static org.apache.accumulo.core.client.ClientConfiguration
- getClientConfiguration(Class<?> implementingClass, Configuration conf) {
- String clientConfigString =
- conf.get(enumToConfKey(implementingClass, InstanceOpts.CLIENT_CONFIG));
- if (null != clientConfigString) {
- return org.apache.accumulo.core.client.ClientConfiguration.deserialize(clientConfigString);
- }
-
- String instanceName = conf.get(enumToConfKey(implementingClass, InstanceOpts.NAME));
- String zookeepers = conf.get(enumToConfKey(implementingClass, InstanceOpts.ZOO_KEEPERS));
- org.apache.accumulo.core.client.ClientConfiguration clientConf =
- org.apache.accumulo.core.client.ClientConfiguration.loadDefault();
- if (null != instanceName) {
- clientConf.withInstance(instanceName);
- }
- if (null != zookeepers) {
- clientConf.withZkHosts(zookeepers);
- }
- return clientConf;
- }
-
- /**
- * Sets the log level for this job.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @param level the logging level
- * @since 1.6.0
- */
- public static void setLogLevel(Class<?> implementingClass, Configuration conf, Level level) {
- checkArgument(level != null, "level is null");
- Logger.getLogger(implementingClass).setLevel(level);
- conf.setInt(enumToConfKey(implementingClass, GeneralOpts.LOG_LEVEL), level.toInt());
- }
-
- /**
- * Gets the log level from this configuration.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @return the log level
- * @since 1.6.0
- * @see #setLogLevel(Class, Configuration, Level)
- */
- public static Level getLogLevel(Class<?> implementingClass, Configuration conf) {
- return Level.toLevel(
- conf.getInt(enumToConfKey(implementingClass, GeneralOpts.LOG_LEVEL), Level.INFO.toInt()));
- }
-
- /**
- * Sets the valid visibility count for this job.
- *
- * @param conf the Hadoop configuration object to configure
- * @param visibilityCacheSize the LRU cache size
- */
- public static void setVisibilityCacheSize(Configuration conf, int visibilityCacheSize) {
- conf.setInt(enumToConfKey(GeneralOpts.VISIBILITY_CACHE_SIZE), visibilityCacheSize);
- }
-
- /**
- * Gets the valid visibility count for this job.
- *
- * @param conf the Hadoop configuration object to configure
- * @return the valid visibility count
- */
- public static int getVisibilityCacheSize(Configuration conf) {
- return conf.getInt(enumToConfKey(GeneralOpts.VISIBILITY_CACHE_SIZE),
- Constants.DEFAULT_VISIBILITY_CACHE_SIZE);
- }
-
- /**
- * Unwraps the provided {@link AuthenticationToken} if it is an instance of DelegationTokenStub,
- * reconstituting it from the provided {@link JobConf}.
- *
- * @param job The job
- * @param token The authentication token
- */
- public static AuthenticationToken unwrapAuthenticationToken(JobConf job,
- AuthenticationToken token) {
- requireNonNull(job);
- requireNonNull(token);
- if (token instanceof org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) {
- org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub delTokenStub =
- (org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) token;
- Token<? extends TokenIdentifier> hadoopToken =
- job.getCredentials().getToken(new Text(delTokenStub.getServiceName()));
- AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier();
- try {
- identifier
- .readFields(new DataInputStream(new ByteArrayInputStream(hadoopToken.getIdentifier())));
- return new DelegationTokenImpl(hadoopToken.getPassword(), identifier);
- } catch (IOException e) {
- throw new RuntimeException("Could not construct DelegationToken from JobConf Credentials",
- e);
- }
- }
- return token;
- }
-
- /**
- * Unwraps the provided {@link AuthenticationToken} if it is an instance of DelegationTokenStub,
- * reconstituting it from the provided {@link JobConf}.
- *
- * @param job The job
- * @param token The authentication token
- */
- public static AuthenticationToken unwrapAuthenticationToken(JobContext job,
- AuthenticationToken token) {
- requireNonNull(job);
- requireNonNull(token);
- if (token instanceof org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) {
- org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub delTokenStub =
- (org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) token;
- Token<? extends TokenIdentifier> hadoopToken =
- job.getCredentials().getToken(new Text(delTokenStub.getServiceName()));
- AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier();
- try {
- identifier
- .readFields(new DataInputStream(new ByteArrayInputStream(hadoopToken.getIdentifier())));
- return new DelegationTokenImpl(hadoopToken.getPassword(), identifier);
- } catch (IOException e) {
- throw new RuntimeException("Could not construct DelegationToken from JobConf Credentials",
- e);
- }
- }
- return token;
- }
-
- public static ClientContext client(Class<?> CLASS, Configuration conf)
- throws AccumuloException, AccumuloSecurityException {
- return ((org.apache.accumulo.core.clientImpl.ConnectorImpl) getInstance(CLASS, conf)
- .getConnector(getPrincipal(CLASS, conf), getAuthenticationToken(CLASS, conf)))
- .getAccumuloClient();
- }
-
- public static ClientContext client(Class<?> CLASS,
- org.apache.accumulo.core.client.mapreduce.RangeInputSplit split, Configuration conf)
- throws IOException {
- try {
- org.apache.accumulo.core.client.Instance instance =
- split.getInstance(getClientConfiguration(CLASS, conf));
- if (instance == null) {
- instance = getInstance(CLASS, conf);
- }
-
- String principal = split.getPrincipal();
- if (principal == null) {
- principal = getPrincipal(CLASS, conf);
- }
-
- AuthenticationToken token = split.getToken();
- if (token == null) {
- token = getAuthenticationToken(CLASS, conf);
- }
-
- return ((org.apache.accumulo.core.clientImpl.ConnectorImpl) instance.getConnector(principal,
- token)).getAccumuloClient();
- } catch (AccumuloException | AccumuloSecurityException e) {
- throw new IOException(e);
- }
-
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/FileOutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/FileOutputConfigurator.java
deleted file mode 100644
index c9a622f..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/FileOutputConfigurator.java
+++ /dev/null
@@ -1,242 +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
- *
- * https://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.clientImpl.mapreduce.lib;
-
-import java.util.Arrays;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * @since 1.6.0
- */
-public class FileOutputConfigurator extends ConfiguratorBase {
-
- /**
- * Configuration keys for {@link AccumuloConfiguration}.
- *
- * @since 1.6.0
- */
- public static enum Opts {
- ACCUMULO_PROPERTIES
- }
-
- /**
- * The supported Accumulo properties we set in this OutputFormat, that change the behavior of the
- * RecordWriter.<br>
- * These properties correspond to the supported public static setter methods available to this
- * class.
- *
- * @param property the Accumulo property to check
- * @since 1.6.0
- */
- protected static Boolean isSupportedAccumuloProperty(Property property) {
- switch (property) {
- case TABLE_FILE_COMPRESSION_TYPE:
- case TABLE_FILE_COMPRESSED_BLOCK_SIZE:
- case TABLE_FILE_BLOCK_SIZE:
- case TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX:
- case TABLE_FILE_REPLICATION:
- return true;
- default:
- return false;
- }
- }
-
- /**
- * Helper for transforming Accumulo configuration properties into something that can be stored
- * safely inside the Hadoop Job configuration.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @param property the supported Accumulo property
- * @param value the value of the property to set
- * @since 1.6.0
- */
- private static <T> void setAccumuloProperty(Class<?> implementingClass, Configuration conf,
- Property property, T value) {
- if (isSupportedAccumuloProperty(property)) {
- String val = String.valueOf(value);
- if (property.getType().isValidFormat(val)) {
- conf.set(
- enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + property.getKey(),
- val);
- } else {
- throw new IllegalArgumentException(
- "Value is not appropriate for property type '" + property.getType() + "'");
- }
- } else {
- throw new IllegalArgumentException("Unsupported configuration property " + property.getKey());
- }
- }
-
- /**
- * This helper method provides an AccumuloConfiguration object constructed from the Accumulo
- * defaults, and overridden with Accumulo properties that have been stored in the Job's
- * configuration.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @since 1.6.0
- */
- public static AccumuloConfiguration getAccumuloConfiguration(Class<?> implementingClass,
- Configuration conf) {
- String prefix = enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + ".";
- ConfigurationCopy acuConf = new ConfigurationCopy(DefaultConfiguration.getInstance());
- for (Entry<String,String> entry : conf) {
- if (entry.getKey().startsWith(prefix)) {
- String propString = entry.getKey().substring(prefix.length());
- Property prop = Property.getPropertyByKey(propString);
- if (prop != null) {
- acuConf.set(prop, entry.getValue());
- } else if (Property.isValidTablePropertyKey(propString)) {
- acuConf.set(propString, entry.getValue());
- } else {
- throw new IllegalArgumentException("Unknown accumulo file property " + propString);
- }
- }
- }
- return acuConf;
- }
-
- /**
- * Sets the compression type to use for data blocks. Specifying a compression may require
- * additional libraries to be available to your Job.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @param compressionType one of "none", "gz", "bzip2", "lzo", "lz4", "snappy", or "zstd"
- * @since 1.6.0
- */
- public static void setCompressionType(Class<?> implementingClass, Configuration conf,
- String compressionType) {
- if (compressionType == null || !Arrays
- .asList("none", "gz", "bzip2", "lzo", "lz4", "snappy", "zstd").contains(compressionType)) {
- throw new IllegalArgumentException(
- "Compression type must be one of: none, gz, bzip2, lzo, lz4, snappy, zstd");
- }
- setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSION_TYPE,
- compressionType);
- }
-
- /**
- * Sets the size for data blocks within each file.<br>
- * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as
- * a group.
- *
- * <p>
- * Making this value smaller may increase seek performance, but at the cost of increasing the size
- * of the indexes (which can also affect seek performance).
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @param dataBlockSize the block size, in bytes
- * @since 1.6.0
- */
- public static void setDataBlockSize(Class<?> implementingClass, Configuration conf,
- long dataBlockSize) {
- setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE,
- dataBlockSize);
- }
-
- /**
- * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by
- * the underlying file system.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @param fileBlockSize the block size, in bytes
- * @since 1.6.0
- */
- public static void setFileBlockSize(Class<?> implementingClass, Configuration conf,
- long fileBlockSize) {
- setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_BLOCK_SIZE, fileBlockSize);
- }
-
- /**
- * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy
- * within the file, while larger blocks mean a more shallow index hierarchy within the file. This
- * can affect the performance of queries.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop configuration object to configure
- * @param indexBlockSize the block size, in bytes
- * @since 1.6.0
- */
- public static void setIndexBlockSize(Class<?> implementingClass, Configuration conf,
- long indexBlockSize) {
- setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX,