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,
- indexBlockSize);
- }
-
- /**
- * Sets the file system replication factor for the resulting file, overriding the file system
- * default.
- *
- * @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 replication the number of replicas for produced files
- * @since 1.6.0
- */
- public static void setReplication(Class<?> implementingClass, Configuration conf,
- int replication) {
- setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_REPLICATION, replication);
- }
-
- /**
- * @since 1.8.0
- */
- public static void setSampler(Class<?> implementingClass, Configuration conf,
- SamplerConfiguration samplerConfig) {
- Map<String,String> props = new SamplerConfigurationImpl(samplerConfig).toTablePropertiesMap();
-
- Set<Entry<String,String>> es = props.entrySet();
- for (Entry<String,String> entry : es) {
- conf.set(enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + entry.getKey(),
- entry.getValue());
- }
- }
-
- public static void setSummarizers(Class<?> implementingClass, Configuration conf,
- SummarizerConfiguration[] sumarizerConfigs) {
- Map<String,String> props = SummarizerConfiguration.toTableProperties(sumarizerConfigs);
-
- for (Entry<String,String> entry : props.entrySet()) {
- conf.set(enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + entry.getKey(),
- entry.getValue());
- }
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java
deleted file mode 100644
index b2a6e43..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java
+++ /dev/null
@@ -1,924 +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.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Base64;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.StringTokenizer;
-
-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.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.RowIterator;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-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;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.NamespacePermission;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.TextUtil;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.MapWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.collect.Maps;
-
-@SuppressWarnings("deprecation")
-public class InputConfigurator extends ConfiguratorBase {
-
- /**
- * Configuration keys for {@link Scanner}.
- *
- * @since 1.6.0
- */
- public enum ScanOpts {
- TABLE_NAME,
- AUTHORIZATIONS,
- RANGES,
- COLUMNS,
- ITERATORS,
- TABLE_CONFIGS,
- SAMPLER_CONFIG,
- CLASSLOADER_CONTEXT
- }
-
- /**
- * Configuration keys for various features.
- *
- * @since 1.6.0
- */
- public enum Features {
- AUTO_ADJUST_RANGES,
- SCAN_ISOLATION,
- USE_LOCAL_ITERATORS,
- SCAN_OFFLINE,
- BATCH_SCANNER,
- BATCH_SCANNER_THREADS
- }
-
- /**
- * Sets the name of the context classloader to use for scans
- *
- * @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 context the name of the context classloader
- * @since 1.8.0
- */
- public static void setClassLoaderContext(Class<?> implementingClass, Configuration conf,
- String context) {
- checkArgument(context != null, "context is null");
- conf.set(enumToConfKey(implementingClass, ScanOpts.CLASSLOADER_CONTEXT), context);
- }
-
- /**
- * Gets the name of the context classloader to use for scans
- *
- * @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 classloader context name
- * @since 1.8.0
- */
- public static String getClassLoaderContext(Class<?> implementingClass, Configuration conf) {
- return conf.get(enumToConfKey(implementingClass, ScanOpts.CLASSLOADER_CONTEXT), null);
- }
-
- /**
- * Sets the name of the input table, over which this job will scan.
- *
- * @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 tableName the table to use when the tablename is null in the write call
- * @since 1.6.0
- */
- public static void setInputTableName(Class<?> implementingClass, Configuration conf,
- String tableName) {
- checkArgument(tableName != null, "tableName is null");
- conf.set(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME), tableName);
- }
-
- /**
- * Sets the name of the input table, over which this job will scan.
- *
- * @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 String getInputTableName(Class<?> implementingClass, Configuration conf) {
- return conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME));
- }
-
- /**
- * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization.
- * Defaults to the empty set.
- *
- * @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 auths the user's authorizations
- * @since 1.6.0
- */
- public static void setScanAuthorizations(Class<?> implementingClass, Configuration conf,
- Authorizations auths) {
- if (auths != null && !auths.isEmpty()) {
- conf.set(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS), auths.serialize());
- }
- }
-
- /**
- * Gets the authorizations to set for the scans 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 Accumulo scan authorizations
- * @since 1.6.0
- * @see #setScanAuthorizations(Class, Configuration, Authorizations)
- */
- public static Authorizations getScanAuthorizations(Class<?> implementingClass,
- Configuration conf) {
- String authString = conf.get(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS));
- return authString == null ? Authorizations.EMPTY
- : new Authorizations(authString.getBytes(UTF_8));
- }
-
- /**
- * Sets the input ranges to scan on all input tables for this job. If not set, the entire table
- * will be scanned.
- *
- * @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 ranges the ranges that will be mapped over
- * @throws IllegalArgumentException if the ranges cannot be encoded into base 64
- * @since 1.6.0
- */
- public static void setRanges(Class<?> implementingClass, Configuration conf,
- Collection<Range> ranges) {
- checkArgument(ranges != null, "ranges is null");
-
- ArrayList<String> rangeStrings = new ArrayList<>(ranges.size());
- try {
- for (Range r : ranges) {
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- r.write(new DataOutputStream(baos));
- rangeStrings.add(Base64.getEncoder().encodeToString(baos.toByteArray()));
- }
- conf.setStrings(enumToConfKey(implementingClass, ScanOpts.RANGES),
- rangeStrings.toArray(new String[0]));
- } catch (IOException ex) {
- throw new IllegalArgumentException("Unable to encode ranges to Base64", ex);
- }
- }
-
- /**
- * Gets the ranges to scan over from a 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
- * @return the ranges
- * @throws IOException if the ranges have been encoded improperly
- * @since 1.6.0
- * @see #setRanges(Class, Configuration, Collection)
- */
- public static List<Range> getRanges(Class<?> implementingClass, Configuration conf)
- throws IOException {
-
- Collection<String> encodedRanges =
- conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.RANGES));
- List<Range> ranges = new ArrayList<>();
- for (String rangeString : encodedRanges) {
- ByteArrayInputStream bais = new ByteArrayInputStream(Base64.getDecoder().decode(rangeString));
- Range range = new Range();
- range.readFields(new DataInputStream(bais));
- ranges.add(range);
- }
- return ranges;
- }
-
- /**
- * Gets a list of the iterator settings (for iterators to apply to a scanner) 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 a list of iterators
- * @since 1.6.0
- * @see #addIterator(Class, Configuration, IteratorSetting)
- */
- public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
- String iterators = conf.get(enumToConfKey(implementingClass, ScanOpts.ITERATORS));
-
- // If no iterators are present, return an empty list
- if (iterators == null || iterators.isEmpty()) {
- return new ArrayList<>();
- }
-
- // Compose the set of iterators encoded in the job configuration
- StringTokenizer tokens = new StringTokenizer(iterators, StringUtils.COMMA_STR);
- List<IteratorSetting> list = new ArrayList<>();
- try {
- while (tokens.hasMoreTokens()) {
- String itstring = tokens.nextToken();
- ByteArrayInputStream bais = new ByteArrayInputStream(Base64.getDecoder().decode(itstring));
- list.add(new IteratorSetting(new DataInputStream(bais)));
- bais.close();
- }
- } catch (IOException e) {
- throw new IllegalArgumentException("couldn't decode iterator settings");
- }
- return list;
- }
-
- /**
- * Restricts the columns that will be mapped over for the single input table on 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 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.
- * @throws IllegalArgumentException if the column family is null
- * @since 1.6.0
- */
- public static void fetchColumns(Class<?> implementingClass, Configuration conf,
- Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
- checkArgument(columnFamilyColumnQualifierPairs != null,
- "columnFamilyColumnQualifierPairs is null");
- String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
- conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings);
- }
-
- public static String[]
- serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
- checkArgument(columnFamilyColumnQualifierPairs != null,
- "columnFamilyColumnQualifierPairs is null");
- ArrayList<String> columnStrings = new ArrayList<>(columnFamilyColumnQualifierPairs.size());
- for (Pair<Text,Text> column : columnFamilyColumnQualifierPairs) {
-
- if (column.getFirst() == null) {
- throw new IllegalArgumentException("Column family can not be null");
- }
-
- String col = Base64.getEncoder().encodeToString(TextUtil.getBytes(column.getFirst()));
- if (column.getSecond() != null) {
- col += ":" + Base64.getEncoder().encodeToString(TextUtil.getBytes(column.getSecond()));
- }
- columnStrings.add(col);
- }
-
- return columnStrings.toArray(new String[0]);
- }
-
- /**
- * Gets the columns to be mapped over from 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
- * @return a set of columns
- * @since 1.6.0
- * @see #fetchColumns(Class, Configuration, Collection)
- */
- public static Set<Pair<Text,Text>> getFetchedColumns(Class<?> implementingClass,
- Configuration conf) {
- checkArgument(conf != null, "conf is null");
- String confValue = conf.get(enumToConfKey(implementingClass, ScanOpts.COLUMNS));
- List<String> serialized = new ArrayList<>();
- if (confValue != null) {
- // Split and include any trailing empty strings to allow empty column families
- Collections.addAll(serialized, confValue.split(",", -1));
- }
- return deserializeFetchedColumns(serialized);
- }
-
- public static Set<Pair<Text,Text>> deserializeFetchedColumns(Collection<String> serialized) {
- Set<Pair<Text,Text>> columns = new HashSet<>();
-
- if (serialized == null) {
- return columns;
- }
-
- for (String col : serialized) {
- int idx = col.indexOf(":");
- Text cf = new Text(idx < 0 ? Base64.getDecoder().decode(col)
- : Base64.getDecoder().decode(col.substring(0, idx)));
- Text cq = idx < 0 ? null : new Text(Base64.getDecoder().decode(col.substring(idx + 1)));
- columns.add(new Pair<>(cf, cq));
- }
- return columns;
- }
-
- /**
- * Encode an iterator on the input for the single input table associated with 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 cfg the configuration of the iterator
- * @throws IllegalArgumentException if the iterator can't be serialized into the configuration
- * @since 1.6.0
- */
- public static void addIterator(Class<?> implementingClass, Configuration conf,
- IteratorSetting cfg) {
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- String newIter;
- try {
- cfg.write(new DataOutputStream(baos));
- newIter = Base64.getEncoder().encodeToString(baos.toByteArray());
- baos.close();
- } catch (IOException e) {
- throw new IllegalArgumentException("unable to serialize IteratorSetting");
- }
-
- String confKey = enumToConfKey(implementingClass, ScanOpts.ITERATORS);
- String iterators = conf.get(confKey);
- // No iterators specified yet, create a new string
- if (iterators == null || iterators.isEmpty()) {
- iterators = newIter;
- } else {
- // append the next iterator & reset
- iterators = iterators.concat(StringUtils.COMMA_STR + newIter);
- }
- // Store the iterators w/ the job
- conf.set(confKey, iterators);
- }
-
- /**
- * Controls the automatic adjustment of ranges for this job. This feature merges overlapping
- * ranges, then splits them to align with tablet boundaries. Disabling this feature will cause
- * exactly one Map task to be created for each specified range. The default setting is enabled. *
- *
- * <p>
- * By default, this feature is <b>enabled</b>.
- *
- * @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 enableFeature the feature is enabled if true, disabled otherwise
- * @see #setRanges(Class, Configuration, Collection)
- * @since 1.6.0
- */
- public static void setAutoAdjustRanges(Class<?> implementingClass, Configuration conf,
- boolean enableFeature) {
- conf.setBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), enableFeature);
- }
-
- /**
- * Determines whether a configuration has auto-adjust ranges enabled.
- *
- * @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 false if the feature is disabled, true otherwise
- * @since 1.6.0
- * @see #setAutoAdjustRanges(Class, Configuration, boolean)
- */
- public static Boolean getAutoAdjustRanges(Class<?> implementingClass, Configuration conf) {
- return conf.getBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), true);
- }
-
- /**
- * Controls the use of the {@link IsolatedScanner} in this job.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @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 enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.6.0
- */
- public static void setScanIsolation(Class<?> implementingClass, Configuration conf,
- boolean enableFeature) {
- conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), enableFeature);
- }
-
- /**
- * Determines whether a configuration has isolation enabled.
- *
- * @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 feature is enabled, false otherwise
- * @since 1.6.0
- * @see #setScanIsolation(Class, Configuration, boolean)
- */
- public static Boolean isIsolated(Class<?> implementingClass, Configuration conf) {
- return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), false);
- }
-
- /**
- * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature
- * will cause the iterator stack to be constructed within the Map task, rather than within the
- * Accumulo TServer. To use this feature, all classes needed for those iterators must be available
- * on the classpath for the task.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @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 enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.6.0
- */
- public static void setLocalIterators(Class<?> implementingClass, Configuration conf,
- boolean enableFeature) {
- conf.setBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), enableFeature);
- }
-
- /**
- * Determines whether a configuration uses local iterators.
- *
- * @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 feature is enabled, false otherwise
- * @since 1.6.0
- * @see #setLocalIterators(Class, Configuration, boolean)
- */
- public static Boolean usesLocalIterators(Class<?> implementingClass, Configuration conf) {
- return conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false);
- }
-
- /**
- * 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 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 enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.6.0
- */
- public static void setOfflineTableScan(Class<?> implementingClass, Configuration conf,
- boolean enableFeature) {
- conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), enableFeature);
- }
-
- /**
- * Determines whether a configuration has the offline table scan feature enabled.
- *
- * @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 feature is enabled, false otherwise
- * @since 1.6.0
- * @see #setOfflineTableScan(Class, Configuration, boolean)
- */
- public static Boolean isOfflineScan(Class<?> implementingClass, Configuration conf) {
- return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), false);
- }
-
- /**
- * Controls the use of the {@link BatchScanner} in this job. Using this feature will group ranges
- * by their source tablet per InputSplit and use BatchScanner to read them.
- *
- * <p>
- * By default, this feature is <b>disabled</b>.
- *
- * @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 enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.7.0
- */
- public static void setBatchScan(Class<?> implementingClass, Configuration conf,
- boolean enableFeature) {
- conf.setBoolean(enumToConfKey(implementingClass, Features.BATCH_SCANNER), enableFeature);
- }
-
- /**
- * Determines whether a configuration has the BatchScanner feature enabled.
- *
- * @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 feature is enabled, false otherwise
- * @since 1.7.0
- * @see #setBatchScan(Class, Configuration, boolean)
- */
- public static Boolean isBatchScan(Class<?> implementingClass, Configuration conf) {
- return conf.getBoolean(enumToConfKey(implementingClass, Features.BATCH_SCANNER), false);
- }
-
- /**
- * Sets configurations for multiple tables at a time.
- *
- * @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 configs an array of InputTableConfig objects to associate with the job
- * @since 1.6.0
- */
- public static void setInputTableConfigs(Class<?> implementingClass, Configuration conf,
- Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configs) {
- MapWritable mapWritable = new MapWritable();
- for (Map.Entry<String,
- org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfig : configs
- .entrySet()) {
- mapWritable.put(new Text(tableConfig.getKey()), tableConfig.getValue());
- }
-
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- try {
- mapWritable.write(new DataOutputStream(baos));
- } catch (IOException e) {
- throw new IllegalStateException("Table configuration could not be serialized.");
- }
-
- String confKey = enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS);
- conf.set(confKey, Base64.getEncoder().encodeToString(baos.toByteArray()));
- }
-
- /**
- * Returns all InputTableConfig objects associated with 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
- * @return all of the table query configs for the job
- * @since 1.6.0
- */
- public static Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig>
- getInputTableConfigs(Class<?> implementingClass, Configuration conf) {
- Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configs =
- new HashMap<>();
- Map.Entry<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> defaultConfig =
- getDefaultInputTableConfig(implementingClass, conf);
- if (defaultConfig != null) {
- configs.put(defaultConfig.getKey(), defaultConfig.getValue());
- }
- String configString = conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS));
- MapWritable mapWritable = new MapWritable();
- if (configString != null) {
- try {
- byte[] bytes = Base64.getDecoder().decode(configString);
- ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
- mapWritable.readFields(new DataInputStream(bais));
- bais.close();
- } catch (IOException e) {
- throw new IllegalStateException("The table query configurations could not be deserialized"
- + " from the given configuration");
- }
- }
- for (Map.Entry<Writable,Writable> entry : mapWritable.entrySet()) {
- configs.put(entry.getKey().toString(),
- (org.apache.accumulo.core.client.mapreduce.InputTableConfig) entry.getValue());
- }
-
- return configs;
- }
-
- /**
- * Returns the InputTableConfig for the given table
- *
- * @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 tableName the table name for which to fetch the table query config
- * @return the table query config for the given table name (if it exists) and null if it does not
- * @since 1.6.0
- */
- public static org.apache.accumulo.core.client.mapreduce.InputTableConfig
- getInputTableConfig(Class<?> implementingClass, Configuration conf, String tableName) {
- Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> queryConfigs =
- getInputTableConfigs(implementingClass, conf);
- return queryConfigs.get(tableName);
- }
-
- private static String extractNamespace(final String tableName) {
- final int delimiterPos = tableName.indexOf('.');
- if (delimiterPos < 1) {
- return ""; // default namespace
- } else {
- return tableName.substring(0, delimiterPos);
- }
- }
-
- /**
- * Validates that the user has permissions on the requested tables
- *
- * @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.7.0
- */
- public static void validatePermissions(Class<?> implementingClass, Configuration conf)
- throws IOException {
- Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> inputTableConfigs =
- getInputTableConfigs(implementingClass, conf);
- try {
- AccumuloClient client = client(implementingClass, conf);
- if (getInputTableConfigs(implementingClass, conf).isEmpty()) {
- throw new IOException("No table set.");
- }
-
- String principal = getPrincipal(implementingClass, conf);
- if (principal == null) {
- principal = client.whoami();
- }
-
- for (Map.Entry<String,
- org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfig : inputTableConfigs
- .entrySet()) {
- final String tableName = tableConfig.getKey();
- final String namespace = extractNamespace(tableName);
- final boolean hasTableRead = client.securityOperations().hasTablePermission(principal,
- tableName, TablePermission.READ);
- final boolean hasNamespaceRead = client.securityOperations()
- .hasNamespacePermission(principal, namespace, NamespacePermission.READ);
- if (!hasTableRead && !hasNamespaceRead) {
- throw new IOException("Unable to access table");
- }
- }
- for (Map.Entry<String,
- org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfigEntry : inputTableConfigs
- .entrySet()) {
- org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig =
- tableConfigEntry.getValue();
- if (!tableConfig.shouldUseLocalIterators()) {
- if (tableConfig.getIterators() != null) {
- for (IteratorSetting iter : tableConfig.getIterators()) {
- if (!client.tableOperations().testClassLoad(tableConfigEntry.getKey(),
- iter.getIteratorClass(), SortedKeyValueIterator.class.getName())) {
- throw new AccumuloException("Servers are unable to load " + iter.getIteratorClass()
- + " as a " + SortedKeyValueIterator.class.getName());
- }
- }
- }
- }
- }
- } catch (AccumuloException | TableNotFoundException | AccumuloSecurityException e) {
- throw new IOException(e);
- }
- }
-
- /**
- * Returns the InputTableConfig for the configuration based on the properties set using the
- * single-table input methods.
- *
- * @param implementingClass the class whose name will be used as a prefix for the property
- * configuration key
- * @param conf the Hadoop instance for which to retrieve the configuration
- * @return the config object built from the single input table properties set on the job
- * @since 1.6.0
- */
- protected static Map.Entry<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig>
- getDefaultInputTableConfig(Class<?> implementingClass, Configuration conf) {
- String tableName = getInputTableName(implementingClass, conf);
- if (tableName != null) {
- org.apache.accumulo.core.client.mapreduce.InputTableConfig queryConfig =
- new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
- List<IteratorSetting> itrs = getIterators(implementingClass, conf);
- if (itrs != null) {
- queryConfig.setIterators(itrs);
- }
- Set<Pair<Text,Text>> columns = getFetchedColumns(implementingClass, conf);
- if (columns != null) {
- queryConfig.fetchColumns(columns);
- }
- List<Range> ranges = null;
- try {
- ranges = getRanges(implementingClass, conf);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- if (ranges != null) {
- queryConfig.setRanges(ranges);
- }
-
- SamplerConfiguration samplerConfig = getSamplerConfiguration(implementingClass, conf);
- if (samplerConfig != null) {
- queryConfig.setSamplerConfiguration(samplerConfig);
- }
-
- queryConfig.setAutoAdjustRanges(getAutoAdjustRanges(implementingClass, conf))
- .setUseIsolatedScanners(isIsolated(implementingClass, conf))
- .setUseLocalIterators(usesLocalIterators(implementingClass, conf))
- .setOfflineScan(isOfflineScan(implementingClass, conf));
- return Maps.immutableEntry(tableName, queryConfig);
- }
- return null;
- }
-
- public static Map<String,Map<KeyExtent,List<Range>>> binOffline(TableId tableId,
- List<Range> ranges, ClientContext context) throws AccumuloException, TableNotFoundException {
-
- Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-
- if (context.getTableState(tableId) != TableState.OFFLINE) {
- context.clearTableListCache();
- if (context.getTableState(tableId) != TableState.OFFLINE) {
- throw new AccumuloException(
- "Table is online tableId:" + tableId + " cannot scan table in offline mode ");
- }
- }
-
- for (Range range : ranges) {
- Text startRow;
-
- if (range.getStartKey() != null) {
- startRow = range.getStartKey().getRow();
- } else {
- startRow = new Text();
- }
-
- Range metadataRange =
- new Range(new KeyExtent(tableId, startRow, null).toMetaRow(), true, null, false);
- Scanner scanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
- TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
- scanner.fetchColumnFamily(LastLocationColumnFamily.NAME);
- scanner.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
- scanner.fetchColumnFamily(FutureLocationColumnFamily.NAME);
- scanner.setRange(metadataRange);
-
- RowIterator rowIter = new RowIterator(scanner);
- KeyExtent lastExtent = null;
- while (rowIter.hasNext()) {
- Iterator<Map.Entry<Key,Value>> row = rowIter.next();
- String last = "";
- KeyExtent extent = null;
- String location = null;
-
- while (row.hasNext()) {
- Map.Entry<Key,Value> entry = row.next();
- Key key = entry.getKey();
-
- if (key.getColumnFamily().equals(LastLocationColumnFamily.NAME)) {
- last = entry.getValue().toString();
- }
-
- if (key.getColumnFamily().equals(CurrentLocationColumnFamily.NAME)
- || key.getColumnFamily().equals(FutureLocationColumnFamily.NAME)) {
- location = entry.getValue().toString();
- }
-
- if (TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
- extent = KeyExtent.fromMetaPrevRow(entry);
- }
-
- }
-
- if (location != null) {
- return null;
- }
-
- if (!extent.tableId().equals(tableId)) {
- throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
- }
-
- if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
- throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
- }
-
- binnedRanges.computeIfAbsent(last, k -> new HashMap<>())
- .computeIfAbsent(extent, k -> new ArrayList<>()).add(range);
-
- if (extent.endRow() == null
- || range.afterEndKey(new Key(extent.endRow()).followingKey(PartialKey.ROW))) {
- break;
- }
-
- lastExtent = extent;
- }
-
- }
- return binnedRanges;
- }
-
- private static String toBase64(Writable writable) {
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- DataOutputStream dos = new DataOutputStream(baos);
- try {
- writable.write(dos);
- dos.close();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
-
- return Base64.getEncoder().encodeToString(baos.toByteArray());
- }
-
- private static <T extends Writable> T fromBase64(T writable, String enc) {
- ByteArrayInputStream bais = new ByteArrayInputStream(Base64.getDecoder().decode(enc));
- DataInputStream dis = new DataInputStream(bais);
- try {
- writable.readFields(dis);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- return writable;
- }
-
- public static void setSamplerConfiguration(Class<?> implementingClass, Configuration conf,
- SamplerConfiguration samplerConfig) {
- requireNonNull(samplerConfig);
-
- String key = enumToConfKey(implementingClass, ScanOpts.SAMPLER_CONFIG);
- String val = toBase64(new SamplerConfigurationImpl(samplerConfig));
-
- conf.set(key, val);
- }
-
- private static SamplerConfiguration getSamplerConfiguration(Class<?> implementingClass,
- Configuration conf) {
- String key = enumToConfKey(implementingClass, ScanOpts.SAMPLER_CONFIG);
-
- String encodedSC = conf.get(key);
- if (encodedSC == null) {
- return null;
- }
-
- return fromBase64(new SamplerConfigurationImpl(), encodedSC).toSamplerConfiguration();
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/OutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/OutputConfigurator.java
deleted file mode 100644
index e557131..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/OutputConfigurator.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.clientImpl.mapreduce.lib;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.hadoop.conf.Configuration;
-
-public class OutputConfigurator extends ConfiguratorBase {
-
- /**
- * Configuration keys for {@link BatchWriter}.
- *
- * @since 1.6.0
- */
- public static enum WriteOpts {
- DEFAULT_TABLE_NAME, BATCH_WRITER_CONFIG
- }
-
- /**
- * Configuration keys for various features.
- *
- * @since 1.6.0
- */
- public static enum Features {
- CAN_CREATE_TABLES, SIMULATION_MODE
- }
-
- /**
- * 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 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 tableName the table to use when the tablename is null in the write call
- * @since 1.6.0
- */
- public static void setDefaultTableName(Class<?> implementingClass, Configuration conf,
- String tableName) {
- if (tableName != null) {
- conf.set(enumToConfKey(implementingClass, WriteOpts.DEFAULT_TABLE_NAME), tableName);
- }
- }
-
- /**
- * Gets the default table 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 default table name
- * @since 1.6.0
- * @see #setDefaultTableName(Class, Configuration, String)
- */
- public static String getDefaultTableName(Class<?> implementingClass, Configuration conf) {
- return conf.get(enumToConfKey(implementingClass, WriteOpts.DEFAULT_TABLE_NAME));
- }
-
- /**
- * 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 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 bwConfig the configuration for the {@link BatchWriter}
- * @since 1.6.0
- */
- public static void setBatchWriterOptions(Class<?> implementingClass, Configuration conf,
- BatchWriterConfig bwConfig) {
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- String serialized;
- try {
- bwConfig.write(new DataOutputStream(baos));
- serialized = new String(baos.toByteArray(), UTF_8);
- baos.close();
- } catch (IOException e) {
- throw new IllegalArgumentException(
- "unable to serialize " + BatchWriterConfig.class.getName());
- }
- conf.set(enumToConfKey(implementingClass, WriteOpts.BATCH_WRITER_CONFIG), serialized);
- }
-
- /**
- * Gets the {@link BatchWriterConfig} settings.
- *
- * @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 configuration object
- * @since 1.6.0
- * @see #setBatchWriterOptions(Class, Configuration, BatchWriterConfig)
- */
- public static BatchWriterConfig getBatchWriterOptions(Class<?> implementingClass,
- Configuration conf) {
- String serialized = conf.get(enumToConfKey(implementingClass, WriteOpts.BATCH_WRITER_CONFIG));
- BatchWriterConfig bwConfig = new BatchWriterConfig();
- if (serialized == null || serialized.isEmpty()) {
- return bwConfig;
- } else {
- try {
- ByteArrayInputStream bais = new ByteArrayInputStream(serialized.getBytes(UTF_8));
- bwConfig.readFields(new DataInputStream(bais));
- bais.close();
- return bwConfig;
- } catch (IOException e) {
- throw new IllegalArgumentException(
- "unable to serialize " + BatchWriterConfig.class.getName());
- }
- }
- }
-
- /**
- * 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 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 enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.6.0
- */
- public static void setCreateTables(Class<?> implementingClass, Configuration conf,
- boolean enableFeature) {
- conf.setBoolean(enumToConfKey(implementingClass, Features.CAN_CREATE_TABLES), enableFeature);
- }
-
- /**
- * Determines whether tables are permitted to be created as needed.
- *
- * @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 feature is disabled, false otherwise
- * @since 1.6.0
- * @see #setCreateTables(Class, Configuration, boolean)
- */
- public static Boolean canCreateTables(Class<?> implementingClass, Configuration conf) {
- return conf.getBoolean(enumToConfKey(implementingClass, Features.CAN_CREATE_TABLES), false);
- }
-
- /**
- * 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 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 enableFeature the feature is enabled if true, disabled otherwise
- * @since 1.6.0
- */
- public static void setSimulationMode(Class<?> implementingClass, Configuration conf,
- boolean enableFeature) {
- conf.setBoolean(enumToConfKey(implementingClass, Features.SIMULATION_MODE), enableFeature);
- }
-
- /**
- * Determines whether this feature is enabled.
- *
- * @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 feature is enabled, false otherwise
- * @since 1.6.0
- * @see #setSimulationMode(Class, Configuration, boolean)
- */
- public static Boolean getSimulationMode(Class<?> implementingClass, Configuration conf) {
- return conf.getBoolean(enumToConfKey(implementingClass, Features.SIMULATION_MODE), false);
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandConfigurer.java b/core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandConfigurer.java
new file mode 100644
index 0000000..6efa34a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandConfigurer.java
@@ -0,0 +1,69 @@
+/*
+ * 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.compaction;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer;
+import org.apache.accumulo.core.conf.Property;
+
+/**
+ * The compaction configurer is used by the shell compact command. It exists in accumulo-core, so it
+ * is on the class path for the shell and servers that run compactions.
+ */
+public class ShellCompactCommandConfigurer implements CompactionConfigurer {
+
+ private Map<String,String> overrides = new HashMap<>();
+
+ @Override
+ public void init(InitParameters iparams) {
+ Set<Entry<String,String>> es = iparams.getOptions().entrySet();
+ for (Entry<String,String> entry : es) {
+
+ switch (CompactionSettings.valueOf(entry.getKey())) {
+ case OUTPUT_COMPRESSION_OPT:
+ overrides.put(Property.TABLE_FILE_COMPRESSION_TYPE.getKey(), entry.getValue());
+ break;
+ case OUTPUT_BLOCK_SIZE_OPT:
+ overrides.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), entry.getValue());
+ break;
+ case OUTPUT_INDEX_BLOCK_SIZE_OPT:
+ overrides.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX.getKey(), entry.getValue());
+ break;
+ case OUTPUT_HDFS_BLOCK_SIZE_OPT:
+ overrides.put(Property.TABLE_FILE_BLOCK_SIZE.getKey(), entry.getValue());
+ break;
+ case OUTPUT_REPLICATION_OPT:
+ overrides.put(Property.TABLE_FILE_REPLICATION.getKey(), entry.getValue());
+ break;
+ default:
+ throw new IllegalArgumentException("Unknown option " + entry.getKey());
+ }
+ }
+ }
+
+ @Override
+ public Overrides override(InputParameters params) {
+ return new Overrides(overrides);
+ }
+
+}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java b/core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandSelector.java
similarity index 80%
rename from server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java
rename to core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandSelector.java
index 6ab51d4..90d5413 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategy.java
+++ b/core/src/main/java/org/apache/accumulo/core/compaction/ShellCompactCommandSelector.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.accumulo.tserver.compaction.strategies;
+package org.apache.accumulo.core.compaction;
import java.util.ArrayList;
import java.util.Collection;
@@ -32,13 +32,10 @@
import java.util.stream.Collectors;
import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
-import org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer;
import org.apache.accumulo.core.client.admin.compaction.CompactionSelector;
import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
import org.apache.accumulo.core.client.summary.Summary;
-import org.apache.accumulo.core.compaction.CompactionSettings;
import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -46,9 +43,10 @@
import org.apache.hadoop.fs.Path;
/**
- * The compaction strategy used by the shell compact command.
+ * The compaction selector is used by the shell compact command. It exists in accumulo-core, so it
+ * is on the class path for the shell and servers that run compactions.
*/
-public class ConfigurableCompactionStrategy implements CompactionSelector, CompactionConfigurer {
+public class ShellCompactCommandSelector implements CompactionSelector {
private abstract static class Test {
abstract Set<CompactableFile> getFilesToCompact(SelectionParameters params);
@@ -168,45 +166,9 @@
private List<Test> tests = new ArrayList<>();
private boolean andTest = true;
private int minFiles = 1;
- private Map<String,String> overrides = new HashMap<>();
@Override
- public void init(
- org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer.InitParameters iparams) {
- Set<Entry<String,String>> es = iparams.getOptions().entrySet();
- for (Entry<String,String> entry : es) {
-
- switch (CompactionSettings.valueOf(entry.getKey())) {
- case OUTPUT_COMPRESSION_OPT:
- overrides.put(Property.TABLE_FILE_COMPRESSION_TYPE.getKey(), entry.getValue());
- break;
- case OUTPUT_BLOCK_SIZE_OPT:
- overrides.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), entry.getValue());
- break;
- case OUTPUT_INDEX_BLOCK_SIZE_OPT:
- overrides.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX.getKey(), entry.getValue());
- break;
- case OUTPUT_HDFS_BLOCK_SIZE_OPT:
- overrides.put(Property.TABLE_FILE_BLOCK_SIZE.getKey(), entry.getValue());
- break;
- case OUTPUT_REPLICATION_OPT:
- overrides.put(Property.TABLE_FILE_REPLICATION.getKey(), entry.getValue());
- break;
- default:
- throw new IllegalArgumentException("Unknown option " + entry.getKey());
- }
- }
-
- }
-
- @Override
- public Overrides override(InputParameters params) {
- return new Overrides(overrides);
- }
-
- @Override
- public void init(
- org.apache.accumulo.core.client.admin.compaction.CompactionSelector.InitParameters iparams) {
+ public void init(InitParameters iparams) {
boolean selectNoSummary = false;
boolean selectExtraSummary = false;
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
index 636b18d..8a6e48b 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
@@ -45,7 +45,6 @@
import java.util.stream.StreamSupport;
import org.apache.accumulo.core.conf.PropertyType.PortRange;
-import org.apache.accumulo.core.spi.scan.SimpleScanDispatcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -411,11 +410,6 @@
* Re-reads the max threads from the configuration that created this class
*/
public int getCurrentMaxThreads() {
- Integer depThreads = getDeprecatedScanThreads(name, isScanServer);
- if (depThreads != null) {
- return depThreads;
- }
-
if (isScanServer) {
String prop =
Property.SSERV_SCAN_EXECUTORS_PREFIX.getKey() + name + "." + SCAN_EXEC_THREADS;
@@ -436,43 +430,6 @@
*/
public abstract boolean isPropertySet(Property prop);
- // deprecation property warning could get spammy in tserver so only warn once
- boolean depPropWarned = false;
-
- @SuppressWarnings("deprecation")
- Integer getDeprecatedScanThreads(String name, boolean isScanServer) {
-
- Property prop;
- Property deprecatedProp;
-
- if (name.equals(SimpleScanDispatcher.DEFAULT_SCAN_EXECUTOR_NAME)) {
- prop = isScanServer ? Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS
- : Property.TSERV_SCAN_EXECUTORS_DEFAULT_THREADS;
- deprecatedProp = Property.TSERV_READ_AHEAD_MAXCONCURRENT;
- } else if (name.equals("meta")) {
- prop = isScanServer ? Property.SSERV_SCAN_EXECUTORS_META_THREADS
- : Property.TSERV_SCAN_EXECUTORS_META_THREADS;
- deprecatedProp = Property.TSERV_METADATA_READ_AHEAD_MAXCONCURRENT;
- } else {
- return null;
- }
-
- if (!isPropertySet(prop) && isPropertySet(deprecatedProp)) {
- if (!depPropWarned) {
- depPropWarned = true;
- log.warn("Property {} is deprecated, use {} instead.", deprecatedProp.getKey(),
- prop.getKey());
- }
- return Integer.valueOf(get(deprecatedProp));
- } else if (isPropertySet(prop) && isPropertySet(deprecatedProp) && !depPropWarned) {
- depPropWarned = true;
- log.warn("Deprecated property {} ignored because {} is set", deprecatedProp.getKey(),
- prop.getKey());
- }
-
- return null;
- }
-
private static class RefCount<T> {
T obj;
long count;
@@ -591,12 +548,7 @@
String val = subEntry.getValue();
if (opt.equals(SCAN_EXEC_THREADS)) {
- Integer depThreads = getDeprecatedScanThreads(name, isScanServer);
- if (depThreads == null) {
- threads = Integer.parseInt(val);
- } else {
- threads = depThreads;
- }
+ threads = Integer.parseInt(val);
} else if (opt.equals(SCAN_EXEC_PRIORITY)) {
prio = Integer.parseInt(val);
} else if (opt.equals(SCAN_EXEC_PRIORITIZER)) {
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ClientConfigGenerate.java b/core/src/main/java/org/apache/accumulo/core/conf/ClientConfigGenerate.java
index 0cd2175..f95688a 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ClientConfigGenerate.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ClientConfigGenerate.java
@@ -29,8 +29,8 @@
import com.google.common.collect.Sets;
/**
- * Generates client-properties.md for documentation on Accumulo website and
- * accumulo-client.properties for Accumulo distribution tarball
+ * Generates client properties documentation on Accumulo website and accumulo-client.properties for
+ * Accumulo distribution tarball
*/
public class ClientConfigGenerate {
@@ -86,9 +86,9 @@
@Override
void pageHeader() {
doc.println("---");
- doc.println("title: Client Properties (2.x)");
+ doc.println("title: Client Properties (3.x)");
doc.println("category: configuration");
- doc.println("order: 3");
+ doc.println("order: 4");
doc.println("---\n");
doc.println("<!-- WARNING: Do not edit this file. It is a generated file"
+ " that is copied from Accumulo build (from core/target/generated-docs) -->");
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java b/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
index 34780c8..39c3e75 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
@@ -28,7 +28,6 @@
import java.util.Objects;
import java.util.Properties;
-import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.client.admin.TableOperations.ImportMappingOptions;
import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
import org.apache.accumulo.core.client.security.tokens.CredentialProviderToken;
@@ -135,16 +134,7 @@
"The maximum duration to leave idle transports open in the client's transport pool", "2.1.0",
false),
- // Trace
- @Deprecated(since = "2.1.0", forRemoval = true)
- TRACE_SPAN_RECEIVERS("trace.span.receivers", "org.apache.accumulo.tracer.ZooTraceClient",
- "A list of span receiver classes to send trace spans"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- TRACE_ZOOKEEPER_PATH("trace.zookeeper.path", Constants.ZTRACERS, PropertyType.PATH,
- "The zookeeper node where tracers are registered", "2.0.0", false);
-
- @Deprecated(since = "2.1.0", forRemoval = true)
- public static final String TRACE_SPAN_RECEIVER_PREFIX = "trace.span.receiver";
+ ;
private final String key;
private final String defaultValue;
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigCheckUtil.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigCheckUtil.java
index eba5cc4..fcb255c 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigCheckUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigCheckUtil.java
@@ -18,7 +18,6 @@
*/
package org.apache.accumulo.core.conf;
-import java.io.IOException;
import java.util.Map.Entry;
import java.util.Objects;
@@ -44,6 +43,7 @@
* {@link Property#INSTANCE_ZK_TIMEOUT} within a valid range.
*
* @param entries iterable through configuration keys and values
+ * @param source the namespace, table id, site or system config where for diagnostic messages
* @throws ConfigCheckException if a fatal configuration error is found
*/
public static void validate(Iterable<Entry<String,String>> entries, @NonNull String source) {
@@ -160,7 +160,7 @@
Class<?> requiredBaseClass) {
try {
ConfigurationTypeHelper.getClassInstance(null, className, requiredBaseClass);
- } catch (IOException | ReflectiveOperationException e) {
+ } catch (ReflectiveOperationException e) {
fatal(confOption + " has an invalid class name: " + className);
} catch (ClassCastException e) {
fatal(confOption + " must implement " + requiredBaseClass
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
index 51a6e24..dc5320e 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
@@ -62,9 +62,9 @@
void pageHeader() {
doc.println("---");
- doc.println("title: Server Properties (2.x)");
+ doc.println("title: Server Properties (3.x)");
doc.println("category: configuration");
- doc.println("order: 5");
+ doc.println("order: 6");
doc.println("---\n");
doc.println("<!-- WARNING: Do not edit this file. It is a generated file"
+ " that is copied from Accumulo build (from core/target/generated-docs) -->\n");
@@ -100,8 +100,8 @@
description += "**Available since:** ";
if (prop.getKey().startsWith("manager.")
&& (prop.availableSince().startsWith("1.") || prop.availableSince().startsWith("2.0"))) {
- description += "2.1.0 (since " + prop.availableSince() + " as *master."
- + prop.getKey().substring(8) + "*)<br>";
+ description += "2.1.0 (formerly *master." + prop.getKey().substring(8) + "* since "
+ + prop.availableSince() + ")<br>";
} else {
description += prop.availableSince() + "<br>";
}
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
index 8893ae8..ef5e0b3 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
@@ -24,7 +24,6 @@
import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.concurrent.TimeUnit.SECONDS;
-import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@@ -173,7 +172,7 @@
try {
instance = getClassInstance(context, clazzName, base);
- } catch (RuntimeException | IOException | ReflectiveOperationException e) {
+ } catch (RuntimeException | ReflectiveOperationException e) {
log.error("Failed to load class {} in classloader context {}", clazzName, context, e);
}
@@ -193,7 +192,7 @@
* @return a new instance of the class
*/
public static <T> T getClassInstance(String context, String clazzName, Class<T> base)
- throws IOException, ReflectiveOperationException {
+ throws ReflectiveOperationException {
T instance;
Class<? extends T> clazz = ClassLoaderUtil.loadClass(context, clazzName, base);
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java b/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java
index 1152103..8cea352 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java
@@ -22,14 +22,10 @@
import java.util.ArrayList;
import java.util.List;
-import java.util.Spliterator;
-import java.util.Spliterators;
import java.util.function.BiConsumer;
import java.util.function.Predicate;
import java.util.function.UnaryOperator;
-import java.util.stream.StreamSupport;
-import org.apache.commons.configuration2.AbstractConfiguration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -52,15 +48,10 @@
private static final Logger log = LoggerFactory.getLogger(DeprecatedPropertyUtil.class);
- @SuppressWarnings("deprecation")
- public static final PropertyRenamer MASTER_MANAGER_RENAMER = PropertyRenamer
- .renamePrefix(Property.MASTER_PREFIX.getKey(), Property.MANAGER_PREFIX.getKey());
-
/**
* Ordered list of renamers
*/
- protected static final List<PropertyRenamer> renamers =
- new ArrayList<>(List.of(MASTER_MANAGER_RENAMER));
+ protected static final List<PropertyRenamer> renamers = new ArrayList<>();
/**
* Checks if {@code propertyName} is a deprecated property name and return its replacement name,
@@ -97,22 +88,10 @@
}
/**
- * Ensures that for any deprecated properties, both the deprecated and replacement property name
- * are not both used in {@code config}.
- *
- * @param config the configuration to check for invalid use of deprecated and replacement
- * properties
+ * @return The list of property renamers
*/
- static void sanityCheckManagerProperties(AbstractConfiguration config) {
- boolean foundMasterPrefix = StreamSupport
- .stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.ORDERED), false)
- .anyMatch(MASTER_MANAGER_RENAMER.keyFilter);
- boolean foundManagerPrefix = StreamSupport
- .stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.ORDERED), false)
- .anyMatch(k -> k.startsWith(Property.MANAGER_PREFIX.getKey()));
- if (foundMasterPrefix && foundManagerPrefix) {
- throw new IllegalStateException("Found both old 'master.*' and new 'manager.*' "
- + "naming conventions in the same startup configuration");
- }
+ public static List<PropertyRenamer> getPropertyRenamers() {
+ return renamers;
}
+
}
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 6295850..ba65025 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -27,19 +27,18 @@
import java.util.HashSet;
import java.util.function.Predicate;
-import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.classloader.ClassLoaderUtil;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
import org.apache.accumulo.core.data.constraints.NoDeleteConstraint;
import org.apache.accumulo.core.file.rfile.RFile;
import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
import org.apache.accumulo.core.iteratorsImpl.system.DeletingIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
import org.apache.accumulo.core.spi.fs.RandomVolumeChooser;
import org.apache.accumulo.core.spi.scan.ScanDispatcher;
import org.apache.accumulo.core.spi.scan.ScanPrioritizer;
+import org.apache.accumulo.core.spi.scan.ScanServerSelector;
import org.apache.accumulo.core.spi.scan.SimpleScanDispatcher;
import org.apache.accumulo.core.util.format.DefaultFormatter;
import org.slf4j.LoggerFactory;
@@ -47,6 +46,23 @@
import com.google.common.base.Preconditions;
public enum Property {
+ COMPACTION_PREFIX("compaction.", null, PropertyType.PREFIX,
+ "Both major and minor compaction properties can be included under this prefix.", "3.1.0"),
+ COMPACTION_SERVICE_PREFIX(COMPACTION_PREFIX + "service.", null, PropertyType.PREFIX,
+ "This prefix should be used to define all properties for the compaction services."
+ + "See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %}.\n"
+ + "A new external compaction service would be defined like the following:\n"
+ + "`compaction.service.newService.planner="
+ + "\"org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner\".`\n"
+ + "`compaction.service.newService.opts.queues=\""
+ + "[{\"name\": \"small\", \"maxSize\":\"32M\"},"
+ + "{ \"name\":\"medium\", \"maxSize\":\"512M\"},{\"name\":\"large\"}]`\n"
+ + "`compaction.service.newService.opts.maxOpen=50`.\n"
+ + "Additional options can be defined using the `compaction.service.<service>.opts.<option>` property.",
+ "3.1.0"),
+ COMPACTION_WARN_TIME(COMPACTION_PREFIX + "warn.time", "10m", PropertyType.TIMEDURATION,
+ "When a compaction has not made progress for this time period, a warning will be logged.",
+ "3.1.0"),
// SSL properties local to each node (see also instance.ssl.enabled which must be consistent
// across all nodes in an instance)
RPC_PREFIX("rpc.", null, PropertyType.PREFIX,
@@ -80,9 +96,9 @@
"1.6.0"),
RPC_SSL_CIPHER_SUITES("rpc.ssl.cipher.suites", "", PropertyType.STRING,
"Comma separated list of cipher suites that can be used by accepted connections.", "1.6.1"),
- RPC_SSL_ENABLED_PROTOCOLS("rpc.ssl.server.enabled.protocols", "TLSv1.2", PropertyType.STRING,
+ RPC_SSL_ENABLED_PROTOCOLS("rpc.ssl.server.enabled.protocols", "TLSv1.3", PropertyType.STRING,
"Comma separated list of protocols that can be used to accept connections.", "1.6.2"),
- RPC_SSL_CLIENT_PROTOCOL("rpc.ssl.client.protocol", "TLSv1.2", PropertyType.STRING,
+ RPC_SSL_CLIENT_PROTOCOL("rpc.ssl.client.protocol", "TLSv1.3", PropertyType.STRING,
"The protocol used to connect to a secure server. Must be in the list of enabled protocols "
+ "on the server side `rpc.ssl.server.enabled.protocols`.",
"1.6.2"),
@@ -208,26 +224,12 @@
+ "encryption, replace this classname with an implementation of the"
+ "org.apache.accumulo.core.spi.crypto.CryptoFactory interface.",
"2.1.0"),
-
// general properties
GENERAL_PREFIX("general.", null, PropertyType.PREFIX,
"Properties in this category affect the behavior of accumulo overall, but"
+ " do not have to be consistent throughout a cloud.",
"1.3.5"),
- @Deprecated(since = "2.0.0")
- GENERAL_DYNAMIC_CLASSPATHS(
- org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.DYNAMIC_CLASSPATH_PROPERTY_NAME,
- org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.DEFAULT_DYNAMIC_CLASSPATH_VALUE,
- PropertyType.STRING,
- "A list of all of the places where changes "
- + "in jars or classes will force a reload of the classloader. Built-in dynamic class "
- + "loading will be removed in a future version. If this is needed, consider overriding "
- + "the Java system class loader with one that has this feature "
- + "(https://docs.oracle.com/javase/8/docs/api/java/lang/ClassLoader.html#getSystemClassLoader--). "
- + "Additionally, this property no longer does property interpolation of environment "
- + "variables, such as '$ACCUMULO_HOME'. Use commons-configuration syntax,"
- + "'${env:ACCUMULO_HOME}' instead.",
- "1.3.5"),
+
GENERAL_CONTEXT_CLASSLOADER_FACTORY("general.context.class.loader.factory", "",
PropertyType.CLASSNAME,
"Name of classloader factory to be used to create classloaders for named contexts,"
@@ -268,11 +270,6 @@
"2.1.0"),
GENERAL_THREADPOOL_SIZE("general.server.threadpool.size", "1", PropertyType.COUNT,
"The number of threads to use for server-internal scheduled tasks.", "2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = GENERAL_THREADPOOL_SIZE)
- GENERAL_SIMPLETIMER_THREADPOOL_SIZE("general.server.simpletimer.threadpool.size", "1",
- PropertyType.COUNT, "The number of threads to use for server-internal scheduled tasks.",
- "1.7.0"),
// If you update the default type, be sure to update the default used for initialization failures
// in VolumeManagerImpl
@Experimental
@@ -297,6 +294,29 @@
GENERAL_IDLE_PROCESS_INTERVAL("general.metrics.process.idle", "5m", PropertyType.TIMEDURATION,
"Amount of time a process must be idle before it is considered to be idle by the metrics system.",
"2.1.3"),
+ GENERAL_LOW_MEM_DETECTOR_INTERVAL("general.low.mem.detector.interval", "5s",
+ PropertyType.TIMEDURATION, "The time interval between low memory checks.", "3.0.0"),
+ GENERAL_LOW_MEM_DETECTOR_THRESHOLD("general.low.mem.detector.threshold", "0.05",
+ PropertyType.FRACTION,
+ "The LowMemoryDetector will report when free memory drops below this percentage of total memory.",
+ "3.0.0"),
+ GENERAL_LOW_MEM_SCAN_PROTECTION("general.low.mem.protection.scan", "false", PropertyType.BOOLEAN,
+ "Scans may be paused or return results early when the server "
+ + "is low on memory and this property is set to true. Enabling this property will incur a slight "
+ + "scan performance penalty when the server is not low on memory.",
+ "3.0.0"),
+ GENERAL_LOW_MEM_MINC_PROTECTION("general.low.mem.protection.compaction.minc", "false",
+ PropertyType.BOOLEAN,
+ "Minor compactions may be paused when the server "
+ + "is low on memory and this property is set to true. Enabling this property will incur a slight "
+ + "compaction performance penalty when the server is not low on memory.",
+ "3.0.0"),
+ GENERAL_LOW_MEM_MAJC_PROTECTION("general.low.mem.protection.compaction.majc", "false",
+ PropertyType.BOOLEAN,
+ "Major compactions may be paused when the server "
+ + "is low on memory and this property is set to true. Enabling this property will incur a slight "
+ + "compaction performance penalty when the server is not low on memory.",
+ "3.0.0"),
GENERAL_MAX_SCANNER_RETRY_PERIOD("general.max.scanner.retry.period", "5s",
PropertyType.TIMEDURATION,
"The maximum amount of time that a Scanner should wait before retrying a failed RPC.",
@@ -314,19 +334,12 @@
+ " was changed and it now can accept multiple class names. The metrics spi was introduced in 2.1.3,"
+ " the deprecated factory is org.apache.accumulo.core.metrics.MeterRegistryFactory.",
"2.1.0"),
+ GENERAL_PROCESS_BIND_ADDRESS("general.process.bind.addr", "0.0.0.0", PropertyType.STRING,
+ "The local IP address to which this server should bind for sending and receiving network traffic.",
+ "3.0.0"),
// properties that are specific to manager server behavior
MANAGER_PREFIX("manager.", null, PropertyType.PREFIX,
- "Properties in this category affect the behavior of the manager server. "
- + "Since 2.1.0, all properties in this category replace the old `master.*` names.",
- "2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = Property.MANAGER_PREFIX)
- MASTER_PREFIX("master.", null, PropertyType.PREFIX,
- "Properties in this category affect the behavior of the manager (formerly named master) server. "
- + "Since 2.1.0, all properties in this category are deprecated and replaced with corresponding "
- + "`manager.*` properties. The old `master.*` names can still be used until at release 3.0, but a warning "
- + "will be emitted. Configuration files should be updated to use the new property names.",
- "1.3.5"),
+ "Properties in this category affect the behavior of the manager server.", "2.1.0"),
MANAGER_CLIENTPORT("manager.port.client", "9999", PropertyType.PORT,
"The port used for handling client connections on the manager.", "1.3.5"),
MANAGER_TABLET_BALANCER("manager.tablet.balancer",
@@ -338,28 +351,11 @@
PropertyType.TIMEDURATION,
"Time to wait between scanning tablet states to identify tablets that need to be assigned, un-assigned, migrated, etc.",
"2.1.2"),
- MANAGER_BULK_RETRIES("manager.bulk.retries", "3", PropertyType.COUNT,
- "The number of attempts to bulk import a RFile before giving up.", "1.4.0"),
- MANAGER_BULK_THREADPOOL_SIZE("manager.bulk.threadpool.size", "5", PropertyType.COUNT,
- "The number of threads to use when coordinating a bulk import.", "1.4.0"),
- MANAGER_BULK_THREADPOOL_TIMEOUT("manager.bulk.threadpool.timeout", "0s",
- PropertyType.TIMEDURATION,
- "The time after which bulk import threads terminate with no work available. Zero (0) will keep the threads alive indefinitely.",
- "2.1.0"),
MANAGER_BULK_TIMEOUT("manager.bulk.timeout", "5m", PropertyType.TIMEDURATION,
"The time to wait for a tablet server to process a bulk import request.", "1.4.3"),
MANAGER_RENAME_THREADS("manager.rename.threadpool.size", "20", PropertyType.COUNT,
"The number of threads to use when renaming user files during table import or bulk ingest.",
"2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = MANAGER_RENAME_THREADS)
- MANAGER_BULK_RENAME_THREADS("manager.bulk.rename.threadpool.size", "20", PropertyType.COUNT,
- "The number of threads to use when moving user files to bulk ingest "
- + "directories under accumulo control.",
- "1.7.0"),
- MANAGER_BULK_TSERVER_REGEX("manager.bulk.tserver.regex", "", PropertyType.STRING,
- "Regular expression that defines the set of Tablet Servers that will perform bulk imports.",
- "2.0.0"),
MANAGER_MINTHREADS("manager.server.threads.minimum", "20", PropertyType.COUNT,
"The minimum number of threads to use to handle incoming requests.", "1.4.0"),
MANAGER_MINTHREADS_TIMEOUT("manager.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
@@ -380,11 +376,6 @@
MANAGER_WAL_CLOSER_IMPLEMENTATION("manager.wal.closer.implementation",
"org.apache.accumulo.server.manager.recovery.HadoopLogCloser", PropertyType.CLASSNAME,
"A class that implements a mechanism to steal write access to a write-ahead log.", "2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = Property.MANAGER_WAL_CLOSER_IMPLEMENTATION)
- MANAGER_WALOG_CLOSER_IMPLEMETATION("manager.walog.closer.implementation",
- "org.apache.accumulo.server.manager.recovery.HadoopLogCloser", PropertyType.CLASSNAME,
- "A class that implements a mechanism to steal write access to a write-ahead log.", "1.5.0"),
MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL("manager.fate.metrics.min.update.interval", "60s",
PropertyType.TIMEDURATION, "Limit calls from metric sinks to zookeeper to update interval.",
"1.9.3"),
@@ -392,29 +383,12 @@
"The number of threads used to run fault-tolerant executions (FATE)."
+ " These are primarily table operations like merge.",
"1.4.3"),
- @Deprecated(since = "2.1.0")
- MANAGER_REPLICATION_SCAN_INTERVAL("manager.replication.status.scan.interval", "30s",
- PropertyType.TIMEDURATION,
- "Amount of time to sleep before scanning the status section of the "
- + "replication table for new data.",
- "1.7.0"),
- @Deprecated(since = "2.1.0")
- MANAGER_REPLICATION_COORDINATOR_PORT("manager.replication.coordinator.port", "10001",
- PropertyType.PORT, "Port for the replication coordinator service.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- MANAGER_REPLICATION_COORDINATOR_MINTHREADS("manager.replication.coordinator.minthreads", "4",
- PropertyType.COUNT, "Minimum number of threads dedicated to answering coordinator requests.",
- "1.7.0"),
- @Deprecated(since = "2.1.0")
- MANAGER_REPLICATION_COORDINATOR_THREADCHECK("manager.replication.coordinator.threadcheck.time",
- "5s", PropertyType.TIMEDURATION,
- "The time between adjustments of the coordinator thread pool.", "1.7.0"),
MANAGER_STATUS_THREAD_POOL_SIZE("manager.status.threadpool.size", "0", PropertyType.COUNT,
"The number of threads to use when fetching the tablet server status for balancing. Zero "
+ "indicates an unlimited number of threads will be used.",
"1.8.0"),
MANAGER_METADATA_SUSPENDABLE("manager.metadata.suspendable", "false", PropertyType.BOOLEAN,
- "Allow tablets for the " + MetadataTable.NAME
+ "Allow tablets for the " + AccumuloTable.METADATA.tableName()
+ " table to be suspended via table.suspend.duration.",
"1.8.0"),
MANAGER_STARTUP_TSERVER_AVAIL_MIN_COUNT("manager.startup.tserver.avail.min.count", "0",
@@ -431,6 +405,15 @@
+ "indefinitely. Default is 0 to block indefinitely. Only valid when tserver available "
+ "threshold is set greater than 0.",
"1.10.0"),
+ SPLIT_PREFIX("split.", null, PropertyType.PREFIX,
+ "System wide properties related to splitting tablets.", "3.1.0"),
+ SPLIT_MAXOPEN("split.files.max", "300", PropertyType.COUNT,
+ "To find a tablets split points, all RFiles are opened and their indexes"
+ + " are read. This setting determines how many RFiles can be opened at once."
+ + " When there are more RFiles than this setting multiple passes must be"
+ + " made, which is slower. However opening too many RFiles at once can cause"
+ + " problems.",
+ "3.1.0"),
// properties that are specific to scan server behavior
@Experimental
SSERV_PREFIX("sserver.", null, PropertyType.PREFIX,
@@ -448,6 +431,13 @@
SSERV_DEFAULT_BLOCKSIZE("sserver.default.blocksize", "1M", PropertyType.BYTES,
"Specifies a default blocksize for the scan server caches.", "2.1.0"),
@Experimental
+ SSERV_GROUP_NAME("sserver.group", ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME,
+ PropertyType.STRING,
+ "Optional group name that will be made available to the "
+ + "ScanServerSelector client plugin. Groups support at least two use cases:"
+ + " dedicating resources to scans and/or using different hardware for scans.",
+ "3.0.0"),
+ @Experimental
SSERV_CACHED_TABLET_METADATA_EXPIRATION("sserver.cache.metadata.expiration", "5m",
PropertyType.TIMEDURATION,
"The time after which cached tablet metadata will be expired if not previously refreshed.",
@@ -534,6 +524,8 @@
TSERV_TOTAL_MUTATION_QUEUE_MAX("tserver.total.mutation.queue.max", "5%", PropertyType.MEMORY,
"The amount of memory used to store write-ahead-log mutations before flushing them.",
"1.7.0"),
+ @ReplacedBy(property = SPLIT_MAXOPEN)
+ @Deprecated(since = "3.1")
TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN("tserver.tablet.split.midpoint.files.max", "300",
PropertyType.COUNT,
"To find a tablets split points, all RFiles are opened and their indexes"
@@ -547,29 +539,12 @@
+ "logs over this threshold is minor compacted. Also any tablet referencing this many "
+ "logs or more will be compacted.",
"2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = Property.TSERV_WAL_MAX_REFERENCED)
- TSERV_WALOG_MAX_REFERENCED("tserver.walog.max.referenced", "3", PropertyType.COUNT,
- "When a tablet server has more than this many write ahead logs, any tablet referencing older "
- + "logs over this threshold is minor compacted. Also any tablet referencing this many "
- + "logs or more will be compacted.",
- "2.0.0"),
TSERV_WAL_MAX_SIZE("tserver.wal.max.size", "1G", PropertyType.BYTES,
"The maximum size for each write-ahead log. See comment for property"
+ " `tserver.memory.maps.max`.",
"2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = Property.TSERV_WAL_MAX_SIZE)
- TSERV_WALOG_MAX_SIZE("tserver.walog.max.size", "1G", PropertyType.BYTES,
- "The maximum size for each write-ahead log. See comment for property"
- + " `tserver.memory.maps.max`.",
- "1.3.5"),
TSERV_WAL_MAX_AGE("tserver.wal.max.age", "24h", PropertyType.TIMEDURATION,
"The maximum age for each write-ahead log.", "2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = Property.TSERV_WAL_MAX_AGE)
- TSERV_WALOG_MAX_AGE("tserver.walog.max.age", "24h", PropertyType.TIMEDURATION,
- "The maximum age for each write-ahead log.", "1.6.6"),
TSERV_WAL_TOLERATED_CREATION_FAILURES("tserver.wal.tolerated.creation.failures", "50",
PropertyType.COUNT,
"The maximum number of failures tolerated when creating a new write-ahead"
@@ -577,35 +552,14 @@
+ " number of failures consecutively trying to create a new write-ahead log"
+ " causes the TabletServer to exit.",
"2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = Property.TSERV_WAL_TOLERATED_CREATION_FAILURES)
- TSERV_WALOG_TOLERATED_CREATION_FAILURES("tserver.walog.tolerated.creation.failures", "50",
- PropertyType.COUNT,
- "The maximum number of failures tolerated when creating a new write-ahead"
- + " log. Negative values will allow unlimited creation failures. Exceeding this"
- + " number of failures consecutively trying to create a new write-ahead log"
- + " causes the TabletServer to exit.",
- "1.7.1"),
TSERV_WAL_TOLERATED_WAIT_INCREMENT("tserver.wal.tolerated.wait.increment", "1000ms",
PropertyType.TIMEDURATION,
"The amount of time to wait between failures to create or write a write-ahead log.", "2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = Property.TSERV_WAL_TOLERATED_WAIT_INCREMENT)
- TSERV_WALOG_TOLERATED_WAIT_INCREMENT("tserver.walog.tolerated.wait.increment", "1000ms",
- PropertyType.TIMEDURATION,
- "The amount of time to wait between failures to create or write a write-ahead log.", "1.7.1"),
// Never wait longer than 5 mins for a retry
TSERV_WAL_TOLERATED_MAXIMUM_WAIT_DURATION("tserver.wal.maximum.wait.duration", "5m",
PropertyType.TIMEDURATION,
"The maximum amount of time to wait after a failure to create or write a write-ahead log.",
"2.1.0"),
- // Never wait longer than 5 mins for a retry
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = Property.TSERV_WAL_TOLERATED_MAXIMUM_WAIT_DURATION)
- TSERV_WALOG_TOLERATED_MAXIMUM_WAIT_DURATION("tserver.walog.maximum.wait.duration", "5m",
- PropertyType.TIMEDURATION,
- "The maximum amount of time to wait after a failure to create or write a write-ahead log.",
- "1.7.1"),
TSERV_SCAN_MAX_OPENFILES("tserver.scan.files.open.max", "100", PropertyType.COUNT,
"Maximum total RFiles that all tablets in a tablet server can open for scans.", "1.4.0"),
TSERV_MAX_IDLE("tserver.files.open.idle", "1m", PropertyType.TIMEDURATION,
@@ -663,19 +617,27 @@
"The maximum number of concurrent tablet migrations for a tablet server.", "1.3.5"),
TSERV_MAJC_DELAY("tserver.compaction.major.delay", "30s", PropertyType.TIMEDURATION,
"Time a tablet server will sleep between checking which tablets need compaction.", "1.3.5"),
+ @Deprecated(since = "3.1")
+ @ReplacedBy(property = COMPACTION_SERVICE_PREFIX)
TSERV_COMPACTION_SERVICE_PREFIX("tserver.compaction.major.service.", null, PropertyType.PREFIX,
"Prefix for compaction services.", "2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_ROOT_PLANNER("tserver.compaction.major.service.root.planner",
DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME,
"Compaction planner for root tablet service.", "2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_ROOT_RATE_LIMIT("tserver.compaction.major.service.root.rate.limit", "0B",
PropertyType.BYTES,
"Maximum number of bytes to read or write per second over all major"
- + " compactions in this compaction service, or 0B for unlimited.",
+ + " compactions in this compaction service, or 0B for unlimited. This property has"
+ + " been deprecated in anticipation of it being removed in a future release that"
+ + " removes the rate limiting feature.",
"2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_ROOT_MAX_OPEN(
"tserver.compaction.major.service.root.planner.opts.maxOpen", "30", PropertyType.COUNT,
"The maximum number of files a compaction will open.", "2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_ROOT_EXECUTORS(
"tserver.compaction.major.service.root.planner.opts.executors",
"[{'name':'small','type':'internal','maxSize':'32M','numThreads':1},{'name':'huge','type':'internal','numThreads':1}]"
@@ -683,17 +645,23 @@
PropertyType.STRING,
"See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %}.",
"2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_META_PLANNER("tserver.compaction.major.service.meta.planner",
DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME,
"Compaction planner for metadata table.", "2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_META_RATE_LIMIT("tserver.compaction.major.service.meta.rate.limit", "0B",
PropertyType.BYTES,
"Maximum number of bytes to read or write per second over all major"
- + " compactions in this compaction service, or 0B for unlimited.",
+ + " compactions in this compaction service, or 0B for unlimited. This property has"
+ + " been deprecated in anticipation of it being removed in a future release that"
+ + " removes the rate limiting feature.",
"2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_META_MAX_OPEN(
"tserver.compaction.major.service.meta.planner.opts.maxOpen", "30", PropertyType.COUNT,
"The maximum number of files a compaction will open.", "2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_META_EXECUTORS(
"tserver.compaction.major.service.meta.planner.opts.executors",
"[{'name':'small','type':'internal','maxSize':'32M','numThreads':2},{'name':'huge','type':'internal','numThreads':2}]"
@@ -701,20 +669,26 @@
PropertyType.JSON,
"See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %}.",
"2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_DEFAULT_PLANNER(
"tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME + ".planner",
DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME,
"Planner for default compaction service.", "2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT(
"tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME + ".rate.limit", "0B",
PropertyType.BYTES,
"Maximum number of bytes to read or write per second over all major"
- + " compactions in this compaction service, or 0B for unlimited.",
+ + " compactions in this compaction service, or 0B for unlimited. This property has"
+ + " been deprecated in anticipation of it being removed in a future release that"
+ + " removes the rate limiting feature.",
"2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN(
"tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME
+ ".planner.opts.maxOpen",
"10", PropertyType.COUNT, "The maximum number of files a compaction will open.", "2.1.0"),
+ @Deprecated(since = "3.1")
TSERV_COMPACTION_SERVICE_DEFAULT_EXECUTORS(
"tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME
+ ".planner.opts.executors",
@@ -723,29 +697,10 @@
PropertyType.STRING,
"See {% jlink -f org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %}.",
"2.1.0"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- @ReplacedBy(property = Property.TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN)
- TSERV_MAJC_THREAD_MAXOPEN("tserver.compaction.major.thread.files.open.max", "10",
- PropertyType.COUNT, "Max number of RFiles a major compaction thread can open at once.",
- "1.4.0"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- @ReplacedBy(property = Property.TSERV_COMPACTION_SERVICE_DEFAULT_EXECUTORS)
- TSERV_MAJC_MAXCONCURRENT("tserver.compaction.major.concurrent.max", "3", PropertyType.COUNT,
- "The maximum number of concurrent major compactions for a tablet server.", "1.3.5"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- @ReplacedBy(property = Property.TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT)
- TSERV_MAJC_THROUGHPUT("tserver.compaction.major.throughput", "0B", PropertyType.BYTES,
- "Maximum number of bytes to read or write per second over all major"
- + " compactions within each compaction service, or 0B for unlimited.",
- "1.8.0"),
TSERV_MINC_MAXCONCURRENT("tserver.compaction.minor.concurrent.max", "4", PropertyType.COUNT,
"The maximum number of concurrent minor compactions for a tablet server.", "1.3.5"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- TSERV_MAJC_TRACE_PERCENT("tserver.compaction.major.trace.percent", "0.1", PropertyType.FRACTION,
- "The percent of major compactions to trace.", "1.7.0"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- TSERV_MINC_TRACE_PERCENT("tserver.compaction.minor.trace.percent", "0.1", PropertyType.FRACTION,
- "The percent of minor compactions to trace.", "1.7.0"),
+ @Deprecated(since = "3.1")
+ @ReplacedBy(property = COMPACTION_WARN_TIME)
TSERV_COMPACTION_WARN_TIME("tserver.compaction.warn.time", "10m", PropertyType.TIMEDURATION,
"When a compaction has not made progress for this time period, a warning will be logged.",
"1.6.0"),
@@ -753,12 +708,6 @@
"The number of concurrent threads that will load bloom filters in the background. "
+ "Setting this to zero will make bloom filters load in the foreground.",
"1.3.5"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- TSERV_MONITOR_FS("tserver.monitor.fs", "false", PropertyType.BOOLEAN,
- "When enabled the tserver will monitor file systems and kill itself when"
- + " one switches from rw to ro. This is usually and indication that Linux has"
- + " detected a bad disk.",
- "1.3.5"),
TSERV_MEMDUMP_DIR("tserver.dir.memdump", "/tmp", PropertyType.PATH,
"A long running scan could possibly hold memory that has been minor"
+ " compacted. To prevent this, the in memory map is dumped to a local file"
@@ -766,23 +715,6 @@
+ " minor compacted file because it may have been modified by iterators. The"
+ " file dumped to the local dir is an exact copy of what was in memory.",
"1.3.5"),
- TSERV_BULK_PROCESS_THREADS("tserver.bulk.process.threads", "1", PropertyType.COUNT,
- "The manager will task a tablet server with pre-processing a bulk import"
- + " RFile prior to assigning it to the appropriate tablet servers. This"
- + " configuration value controls the number of threads used to process the files.",
- "1.4.0"),
- TSERV_BULK_ASSIGNMENT_THREADS("tserver.bulk.assign.threads", "1", PropertyType.COUNT,
- "The manager delegates bulk import RFile processing and assignment to"
- + " tablet servers. After file has been processed, the tablet server will"
- + " assign the file to the appropriate tablets on all servers. This property"
- + " controls the number of threads used to communicate to the other servers.",
- "1.4.0"),
- TSERV_BULK_RETRY("tserver.bulk.retry.max", "5", PropertyType.COUNT,
- "The number of times the tablet server will attempt to assign a RFile to"
- + " a tablet as it migrates and splits.",
- "1.4.0"),
- TSERV_BULK_TIMEOUT("tserver.bulk.timeout", "5m", PropertyType.TIMEDURATION,
- "The time to wait for a tablet server to process a bulk import request.", "1.4.3"),
TSERV_HEALTH_CHECK_FREQ("tserver.health.check.interval", "30m", PropertyType.TIMEDURATION,
"The time between tablet server health checks.", "2.1.0"),
TSERV_MINTHREADS("tserver.server.threads.minimum", "20", PropertyType.COUNT,
@@ -818,16 +750,8 @@
"1.5.0"),
TSERV_WAL_SORT_MAX_CONCURRENT("tserver.wal.sort.concurrent.max", "2", PropertyType.COUNT,
"The maximum number of threads to use to sort logs during recovery.", "2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = Property.TSERV_WAL_SORT_MAX_CONCURRENT)
- TSERV_RECOVERY_MAX_CONCURRENT("tserver.recovery.concurrent.max", "2", PropertyType.COUNT,
- "The maximum number of threads to use to sort logs during recovery.", "1.5.0"),
TSERV_WAL_SORT_BUFFER_SIZE("tserver.wal.sort.buffer.size", "10%", PropertyType.MEMORY,
"The amount of memory to use when sorting logs during recovery.", "2.1.0"),
- @Deprecated(since = "2.1.0")
- @ReplacedBy(property = Property.TSERV_WAL_SORT_BUFFER_SIZE)
- TSERV_SORT_BUFFER_SIZE("tserver.sort.buffer.size", "10%", PropertyType.MEMORY,
- "The amount of memory to use when sorting logs during recovery.", "1.5.0"),
TSERV_WAL_SORT_FILE_PREFIX("tserver.wal.sort.file.", null, PropertyType.PREFIX,
"The rfile properties to use when sorting logs during recovery. Most of the properties"
+ " that begin with 'table.file' can be used here. For example, to set the compression"
@@ -836,7 +760,8 @@
@Deprecated(since = "2.1.3")
TSERV_WORKQ_THREADS("tserver.workq.threads", "2", PropertyType.COUNT,
"The number of threads for the distributed work queue. These threads are"
- + " used for copying failed bulk import RFiles. This property will be removed when bulk import V1 is removed.",
+ + " used for copying failed bulk import RFiles. Note that as of version 3.1.0 this property"
+ + " is not used and will be removed in a future release.",
"1.4.2"),
TSERV_WAL_SYNC("tserver.wal.sync", "true", PropertyType.BOOLEAN,
"Use the SYNC_BLOCK create flag to sync WAL writes to disk. Prevents"
@@ -848,17 +773,6 @@
+ " warning along with the current stack trace. Meant to help debug stuck"
+ " assignments.",
"1.6.2"),
- @Deprecated(since = "2.1.0")
- TSERV_REPLICATION_REPLAYERS("tserver.replication.replayer.", null, PropertyType.PREFIX,
- "Allows configuration of implementation used to apply replicated data.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- TSERV_REPLICATION_DEFAULT_HANDLER("tserver.replication.default.replayer",
- "org.apache.accumulo.tserver.replication.BatchWriterReplicationReplayer",
- PropertyType.CLASSNAME, "Default AccumuloReplicationReplayer implementation.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- TSERV_REPLICATION_BW_REPLAYER_MEMORY("tserver.replication.batchwriter.replayer.memory", "50M",
- PropertyType.BYTES, "Memory to provide to batchwriter to replay mutations for replication.",
- "1.7.0"),
TSERV_ASSIGNMENT_MAXCONCURRENT("tserver.assignment.concurrent.max", "2", PropertyType.COUNT,
"The number of threads available to load tablets. Recoveries are still performed serially.",
"1.7.0"),
@@ -886,6 +800,7 @@
"The number of threads on each tablet server available to retrieve"
+ " summary data, that is not currently in cache, from RFiles.",
"2.0.0"),
+ @Deprecated(since = "3.1")
TSERV_LAST_LOCATION_MODE("tserver.last.location.mode", "compaction",
PropertyType.LAST_LOCATION_MODE,
"Describes how the system will record the 'last' location for tablets, which can be used for"
@@ -894,9 +809,10 @@
+ " 'assignment' is the mode, then the most recently assigned location will be recorded."
+ " The manager.startup.tserver properties might also need to be set to ensure the"
+ " tserver is available before tablets are initially assigned if the 'last' location is"
- + " to be used.",
+ + " to be used. This property has been deprecated in anticipation of it being removed in"
+ + " a future release that removes major compactions from the TabletServer, rendering this"
+ + " feature moot.",
"2.1.1"),
-
// accumulo garbage collector properties
GC_PREFIX("gc.", null, PropertyType.PREFIX,
"Properties in this category affect the behavior of the accumulo garbage collector.",
@@ -914,17 +830,6 @@
"The listening port for the garbage collector's monitor service.", "1.3.5"),
GC_DELETE_THREADS("gc.threads.delete", "16", PropertyType.COUNT,
"The number of threads used to delete RFiles and write-ahead logs.", "1.3.5"),
- @Experimental
- GC_REMOVE_IN_USE_CANDIDATES("gc.remove.in.use.candidates", "false", PropertyType.BOOLEAN,
- "GC will remove deletion candidates that are in-use from the metadata location. "
- + "This is expected to increase the speed of subsequent GC runs.",
- "2.1.3"),
- @Deprecated(since = "2.1.1", forRemoval = true)
- GC_TRASH_IGNORE("gc.trash.ignore", "false", PropertyType.BOOLEAN,
- "Do not use the Trash, even if it is configured.", "1.5.0"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- GC_TRACE_PERCENT("gc.trace.percent", "0.01", PropertyType.FRACTION,
- "Percent of gc cycles to trace.", "1.7.0"),
GC_SAFEMODE("gc.safemode", "false", PropertyType.BOOLEAN,
"Provides listing of files to be deleted but does not delete any files.", "2.1.0"),
GC_USE_FULL_COMPACTION("gc.post.metadata.action", "flush", PropertyType.GC_POST_ACTION,
@@ -966,7 +871,7 @@
"A comma-separated list of disallowed SSL Ciphers, see"
+ " monitor.ssl.include.ciphers to allow ciphers.",
"1.6.1"),
- MONITOR_SSL_INCLUDE_PROTOCOLS("monitor.ssl.include.protocols", "TLSv1.2", PropertyType.STRING,
+ MONITOR_SSL_INCLUDE_PROTOCOLS("monitor.ssl.include.protocols", "TLSv1.3", PropertyType.STRING,
"A comma-separate list of allowed SSL protocols.", "1.5.3"),
MONITOR_LOCK_CHECK_INTERVAL("monitor.lock.check.interval", "5s", PropertyType.TIMEDURATION,
"The amount of time to sleep between checking for the Monitor ZooKeeper lock.", "1.5.1"),
@@ -979,41 +884,6 @@
+ " The resources that are used by default can be seen in"
+ " `accumulo/server/monitor/src/main/resources/templates/default.ftl`.",
"2.0.0"),
- @Deprecated(since = "2.1.0")
- TRACE_PREFIX("trace.", null, PropertyType.PREFIX,
- "Properties in this category affect the behavior of distributed tracing.", "1.3.5"),
- @Deprecated(since = "2.1.0")
- TRACE_SPAN_RECEIVERS("trace.span.receivers", "org.apache.accumulo.tracer.ZooTraceClient",
- PropertyType.CLASSNAMELIST, "A list of span receiver classes to send trace spans.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- TRACE_SPAN_RECEIVER_PREFIX("trace.span.receiver.", null, PropertyType.PREFIX,
- "Prefix for span receiver configuration properties.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- TRACE_ZK_PATH("trace.zookeeper.path", Constants.ZTRACERS, PropertyType.STRING,
- "The zookeeper node where tracers are registered.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- TRACE_PORT("trace.port.client", "12234", PropertyType.PORT,
- "The listening port for the trace server.", "1.3.5"),
- @Deprecated(since = "2.1.0")
- TRACE_TABLE("trace.table", "trace", PropertyType.STRING,
- "The name of the table to store distributed traces.", "1.3.5"),
- @Deprecated(since = "2.1.0")
- TRACE_USER("trace.user", "root", PropertyType.STRING,
- "The name of the user to store distributed traces.", "1.3.5"),
- @Sensitive
- @Deprecated(since = "2.1.0")
- TRACE_PASSWORD("trace.password", "secret", PropertyType.STRING,
- "The password for the user used to store distributed traces.", "1.3.5"),
- @Sensitive
- @Deprecated(since = "2.1.0")
- TRACE_TOKEN_PROPERTY_PREFIX("trace.token.property.", null, PropertyType.PREFIX,
- "The prefix used to create a token for storing distributed traces. For"
- + " each property required by trace.token.type, place this prefix in front of it.",
- "1.5.0"),
- @Deprecated(since = "2.1.0")
- TRACE_TOKEN_TYPE("trace.token.type", PasswordToken.class.getName(), PropertyType.CLASSNAME,
- "An AuthenticationToken type supported by the authorizer.", "1.5.0"),
-
// per table properties
TABLE_PREFIX("table.", null, PropertyType.PREFIX,
"Properties in this category affect tablet server treatment of tablets,"
@@ -1042,23 +912,10 @@
"Minimum ratio of total input size to maximum input RFile size for"
+ " running a major compaction.",
"1.3.5"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- TABLE_MAJC_COMPACTALL_IDLETIME("table.compaction.major.everything.idle", "1h",
- PropertyType.TIMEDURATION,
- "After a tablet has been idle (no mutations) for this time period it may"
- + " have all of its RFiles compacted into one. There is no guarantee an idle"
- + " tablet will be compacted. Compactions of idle tablets are only started"
- + " when regular compactions are not running. Idle compactions only take"
- + " place for tablets that have one or more RFiles.",
- "1.3.5"),
TABLE_SPLIT_THRESHOLD("table.split.threshold", "1G", PropertyType.BYTES,
"A tablet is split when the combined size of RFiles exceeds this amount.", "1.3.5"),
TABLE_MAX_END_ROW_SIZE("table.split.endrow.size.max", "10k", PropertyType.BYTES,
"Maximum size of end row.", "1.7.0"),
- @Deprecated(since = "2.0.0")
- @ReplacedBy(property = Property.TSERV_WAL_MAX_REFERENCED)
- TABLE_MINC_LOGS_MAX("table.compaction.minor.logs.threshold", "3", PropertyType.COUNT,
- "This property is deprecated and replaced.", "1.3.5"),
TABLE_MINC_COMPACT_IDLETIME("table.compaction.minor.idle", "5m", PropertyType.TIMEDURATION,
"After a tablet has been idle (no mutations) for this time period it may have its "
+ "in-memory map flushed to disk in a minor compaction. There is no guarantee an idle "
@@ -1078,10 +935,13 @@
+ "specified time. If a system compaction cancels a hold and runs, then the user compaction"
+ " can reselect and hold files after the system compaction runs.",
"2.1.0"),
+ @Deprecated(since = "3.1")
TABLE_COMPACTION_SELECTOR("table.compaction.selector", "", PropertyType.CLASSNAME,
"A configurable selector for a table that can periodically select file for mandatory "
- + "compaction, even if the files do not meet the compaction ratio.",
+ + "compaction, even if the files do not meet the compaction ratio. This option was deprecated in "
+ + "3.1, see the CompactionKind.SELECTOR enum javadoc for details.",
"2.1.0"),
+ @Deprecated(since = "3.1")
TABLE_COMPACTION_SELECTOR_OPTS("table.compaction.selector.opts.", null, PropertyType.PREFIX,
"Options for the table compaction dispatcher.", "2.1.0"),
TABLE_COMPACTION_CONFIGURER("table.compaction.configurer", "", PropertyType.CLASSNAME,
@@ -1089,15 +949,6 @@
"2.1.0"),
TABLE_COMPACTION_CONFIGURER_OPTS("table.compaction.configurer.opts.", null, PropertyType.PREFIX,
"Options for the table compaction configuror.", "2.1.0"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- @ReplacedBy(property = TABLE_COMPACTION_SELECTOR)
- TABLE_COMPACTION_STRATEGY("table.majc.compaction.strategy", "", PropertyType.CLASSNAME,
- "See {% jlink -f org.apache.accumulo.core.spi.compaction %}.", "1.6.0"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- @ReplacedBy(property = TABLE_COMPACTION_SELECTOR_OPTS)
- TABLE_COMPACTION_STRATEGY_PREFIX("table.majc.compaction.strategy.opts.", null,
- PropertyType.PREFIX,
- "Properties in this category are used to configure the compaction strategy.", "1.6.0"),
// Crypto-related properties
@Experimental
TABLE_CRYPTO_PREFIX("table.crypto.opts.", null, PropertyType.PREFIX,
@@ -1266,31 +1117,10 @@
"1.3.5"),
TABLE_FORMATTER_CLASS("table.formatter", DefaultFormatter.class.getName(), PropertyType.STRING,
"The Formatter class to apply on results in the shell.", "1.4.0"),
- @Deprecated(since = "2.1.0")
- TABLE_INTERPRETER_CLASS("table.interepreter",
- org.apache.accumulo.core.util.interpret.DefaultScanInterpreter.class.getName(),
- PropertyType.STRING,
- "The ScanInterpreter class to apply on scan arguments in the shell. "
- + "Note that this property is deprecated and will be removed in a future version.",
- "1.5.0"),
TABLE_CLASSLOADER_CONTEXT("table.class.loader.context", "", PropertyType.STRING,
"The context to use for loading per-table resources, such as iterators"
+ " from the configured factory in `general.context.class.loader.factory`.",
"2.1.0"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- @ReplacedBy(property = TABLE_CLASSLOADER_CONTEXT)
- TABLE_CLASSPATH("table.classpath.context", "", PropertyType.STRING,
- "Per table classpath context.", "1.5.0"),
- @Deprecated(since = "2.1.0")
- TABLE_REPLICATION("table.replication", "false", PropertyType.BOOLEAN,
- "Is replication enabled for the given table.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- TABLE_REPLICATION_TARGET("table.replication.target.", null, PropertyType.PREFIX,
- "Enumerate a mapping of other systems which this table should replicate"
- + " their data to. The key suffix is the identifying cluster name and the"
- + " value is an identifier for a location on the target system, e.g. the ID"
- + " of the table on the target to replicate to.",
- "1.7.0"),
TABLE_SAMPLER("table.sampler", "", PropertyType.CLASSNAME,
"The name of a class that implements org.apache.accumulo.core.Sampler."
+ " Setting this option enables storing a sample of data which can be"
@@ -1331,122 +1161,6 @@
+ "constraint.",
"2.0.0"),
- // VFS ClassLoader properties
-
- // this property shouldn't be used directly; it exists solely to document the default value
- // defined by its use in AccumuloVFSClassLoader when generating the property documentation
- @Deprecated(since = "2.1.0", forRemoval = true)
- VFS_CLASSLOADER_SYSTEM_CLASSPATH_PROPERTY(
- org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.VFS_CLASSLOADER_SYSTEM_CLASSPATH_PROPERTY,
- "", PropertyType.STRING,
- "Configuration for a system level vfs classloader. Accumulo jar can be"
- + " configured here and loaded out of HDFS.",
- "1.5.0"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- VFS_CONTEXT_CLASSPATH_PROPERTY(
- org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.VFS_CONTEXT_CLASSPATH_PROPERTY,
- null, PropertyType.PREFIX,
- "Properties in this category are define a classpath. These properties"
- + " start with the category prefix, followed by a context name. The value is"
- + " a comma separated list of URIs. Supports full regex on filename alone."
- + " For example, general.vfs.context.classpath.cx1=hdfs://nn1:9902/mylibdir/*.jar."
- + " You can enable post delegation for a context, which will load classes from the"
- + " context first instead of the parent first. Do this by setting"
- + " `general.vfs.context.classpath.<name>.delegation=post`, where `<name>` is"
- + " your context name. If delegation is not specified, it defaults to loading"
- + " from parent classloader first.",
- "1.5.0"),
-
- // this property shouldn't be used directly; it exists solely to document the default value
- // defined by its use in AccumuloVFSClassLoader when generating the property documentation
- @Deprecated(since = "2.1.0", forRemoval = true)
- VFS_CLASSLOADER_CACHE_DIR(
- org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.VFS_CACHE_DIR,
- "${java.io.tmpdir}", PropertyType.ABSOLUTEPATH,
- "The base directory to use for the vfs cache. The actual cached files will be located"
- + " in a subdirectory, `accumulo-vfs-cache-<jvmProcessName>-${user.name}`, where"
- + " `<jvmProcessName>` is determined by the JVM's internal management engine."
- + " The cache will keep a soft reference to all of the classes loaded in the VM."
- + " This should be on local disk on each node with sufficient space.",
- "1.5.0"),
-
- // General properties for configuring replication
- @Deprecated(since = "2.1.0")
- REPLICATION_PREFIX("replication.", null, PropertyType.PREFIX,
- "Properties in this category affect the replication of data to other Accumulo instances.",
- "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_PEERS("replication.peer.", null, PropertyType.PREFIX,
- "Properties in this category control what systems data can be replicated to.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_PEER_USER("replication.peer.user.", null, PropertyType.PREFIX,
- "The username to provide when authenticating with the given peer.", "1.7.0"),
- @Sensitive
- @Deprecated(since = "2.1.0")
- REPLICATION_PEER_PASSWORD("replication.peer.password.", null, PropertyType.PREFIX,
- "The password to provide when authenticating with the given peer.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_PEER_KEYTAB("replication.peer.keytab.", null, PropertyType.PREFIX,
- "The keytab to use when authenticating with the given peer.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_NAME("replication.name", "", PropertyType.STRING,
- "Name of this cluster with respect to replication. Used to identify this"
- + " instance from other peers.",
- "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_MAX_WORK_QUEUE("replication.max.work.queue", "1000", PropertyType.COUNT,
- "Upper bound of the number of files queued for replication.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_WORK_ASSIGNMENT_SLEEP("replication.work.assignment.sleep", "30s",
- PropertyType.TIMEDURATION, "Amount of time to sleep between replication work assignment.",
- "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_WORKER_THREADS("replication.worker.threads", "4", PropertyType.COUNT,
- "Size of the threadpool that each tabletserver devotes to replicating data.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_RECEIPT_SERVICE_PORT("replication.receipt.service.port", "10002", PropertyType.PORT,
- "Listen port used by thrift service in tserver listening for replication.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_WORK_ATTEMPTS("replication.work.attempts", "10", PropertyType.COUNT,
- "Number of attempts to try to replicate some data before giving up and"
- + " letting it naturally be retried later.",
- "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_MIN_THREADS("replication.receiver.min.threads", "1", PropertyType.COUNT,
- "Minimum number of threads for replication.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_THREADCHECK("replication.receiver.threadcheck.time", "30s", PropertyType.TIMEDURATION,
- "The time between adjustments of the replication thread pool.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_MAX_UNIT_SIZE("replication.max.unit.size", "64M", PropertyType.BYTES,
- "Maximum size of data to send in a replication message.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_WORK_ASSIGNER("replication.work.assigner",
- "org.apache.accumulo.manager.replication.UnorderedWorkAssigner", PropertyType.CLASSNAME,
- "Replication WorkAssigner implementation to use.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_DRIVER_DELAY("replication.driver.delay", "0s", PropertyType.TIMEDURATION,
- "Amount of time to wait before the replication work loop begins in the manager.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_WORK_PROCESSOR_DELAY("replication.work.processor.delay", "0s",
- PropertyType.TIMEDURATION,
- "Amount of time to wait before first checking for replication work, not"
- + " useful outside of tests.",
- "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_WORK_PROCESSOR_PERIOD("replication.work.processor.period", "0s",
- PropertyType.TIMEDURATION,
- "Amount of time to wait before re-checking for replication work, not"
- + " useful outside of tests.",
- "1.7.0"),
- @Deprecated(since = "2.1.0", forRemoval = true)
- REPLICATION_TRACE_PERCENT("replication.trace.percent", "0.1", PropertyType.FRACTION,
- "The sampling percentage to use for replication traces.", "1.7.0"),
- @Deprecated(since = "2.1.0")
- REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
- "Amount of time for a single replication RPC call to last before failing"
- + " the attempt. See replication.work.attempts.",
- "1.7.4"),
// Compactor properties
@Experimental
COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
@@ -1481,6 +1195,9 @@
@Experimental
COMPACTOR_MAX_MESSAGE_SIZE("compactor.message.size.max", "10M", PropertyType.BYTES,
"The maximum size of a message that can be sent to a tablet server.", "2.1.0"),
+ @Experimental
+ COMPACTOR_QUEUE_NAME("compactor.queue", "", PropertyType.STRING,
+ "The queue for which this Compactor will perform compactions.", "3.0.0"),
// CompactionCoordinator properties
@Experimental
COMPACTION_COORDINATOR_PREFIX("compaction.coordinator.", null, PropertyType.PREFIX,
@@ -1529,54 +1246,7 @@
@Experimental
COMPACTION_COORDINATOR_TSERVER_COMPACTION_CHECK_INTERVAL(
"compaction.coordinator.tserver.check.interval", "1m", PropertyType.TIMEDURATION,
- "The interval at which to check the tservers for external compactions.", "2.1.0"),
- // deprecated properties grouped at the end to reference property that replaces them
- @Deprecated(since = "1.6.0")
- @ReplacedBy(property = INSTANCE_VOLUMES)
- INSTANCE_DFS_URI("instance.dfs.uri", "", PropertyType.URI,
- "A url accumulo should use to connect to DFS. If this is empty, accumulo"
- + " will obtain this information from the hadoop configuration. This property"
- + " will only be used when creating new files if instance.volumes is empty."
- + " After an upgrade to 1.6.0 Accumulo will start using absolute paths to"
- + " reference files. Files created before a 1.6.0 upgrade are referenced via"
- + " relative paths. Relative paths will always be resolved using this config"
- + " (if empty using the hadoop config).",
- "1.4.0"),
- @Deprecated(since = "1.6.0")
- @ReplacedBy(property = INSTANCE_VOLUMES)
- INSTANCE_DFS_DIR("instance.dfs.dir", "/accumulo", PropertyType.ABSOLUTEPATH,
- "HDFS directory in which accumulo instance will run. "
- + "Do not change after accumulo is initialized.",
- "1.3.5"),
- @Deprecated(since = "2.0.0")
- GENERAL_CLASSPATHS(org.apache.accumulo.start.classloader.AccumuloClassLoader.GENERAL_CLASSPATHS,
- "", PropertyType.STRING,
- "The class path should instead be configured"
- + " by the launch environment (for example, accumulo-env.sh). A list of all"
- + " of the places to look for a class. Order does matter, as it will look for"
- + " the jar starting in the first location to the last. Supports full regex"
- + " on filename alone.",
- "1.3.5"),
- @Deprecated(since = "1.7.0")
- @ReplacedBy(property = TABLE_DURABILITY)
- TSERV_WAL_SYNC_METHOD("tserver.wal.sync.method", "hsync", PropertyType.STRING,
- "Use table.durability instead.", "1.5.2"),
- @Deprecated(since = "1.7.0")
- @ReplacedBy(property = TABLE_DURABILITY)
- TABLE_WALOG_ENABLED("table.walog.enabled", "true", PropertyType.BOOLEAN,
- "Use table.durability=none instead.", "1.3.5"),
- @Deprecated(since = "2.0.0")
- @ReplacedBy(property = TSERV_SCAN_EXECUTORS_DEFAULT_THREADS)
- TSERV_READ_AHEAD_MAXCONCURRENT("tserver.readahead.concurrent.max", "16", PropertyType.COUNT,
- "The maximum number of concurrent read ahead that will execute. This "
- + "effectively limits the number of long running scans that can run concurrently "
- + "per tserver.",
- "1.3.5"),
- @Deprecated(since = "2.0.0")
- @ReplacedBy(property = TSERV_SCAN_EXECUTORS_META_THREADS)
- TSERV_METADATA_READ_AHEAD_MAXCONCURRENT("tserver.metadata.readahead.concurrent.max", "8",
- PropertyType.COUNT, "The maximum number of concurrent metadata read ahead that will execute.",
- "1.3.5");
+ "The interval at which to check the tservers for external compactions.", "2.1.0");
private final String key;
private final String defaultValue;
@@ -1733,6 +1403,8 @@
ReplacedBy rb = getAnnotation(ReplacedBy.class);
if (rb != null) {
replacedBy = rb.property();
+ } else {
+ isReplaced = false;
}
annotationsComputed = true;
}
@@ -1831,8 +1503,6 @@
&& (key.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey())
|| key.startsWith(Property.TABLE_ITERATOR_PREFIX.getKey())
|| key.startsWith(Property.TABLE_LOCALITY_GROUP_PREFIX.getKey())
- || key.startsWith(Property.TABLE_COMPACTION_STRATEGY_PREFIX.getKey())
- || key.startsWith(Property.TABLE_REPLICATION_TARGET.getKey())
|| key.startsWith(Property.TABLE_ARBITRARY_PROP_PREFIX.getKey())
|| key.startsWith(TABLE_SAMPLER_OPTS.getKey())
|| key.startsWith(TABLE_SUMMARIZER_PREFIX.getKey())
@@ -1892,15 +1562,14 @@
// white list prefixes
return key.startsWith(Property.TABLE_PREFIX.getKey())
|| key.startsWith(Property.TSERV_PREFIX.getKey())
+ || key.startsWith(Property.COMPACTION_SERVICE_PREFIX.getKey())
|| key.startsWith(Property.SSERV_PREFIX.getKey())
|| key.startsWith(Property.MANAGER_PREFIX.getKey())
- || key.startsWith(Property.MASTER_PREFIX.getKey())
|| key.startsWith(Property.GC_PREFIX.getKey())
|| key.startsWith(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey())
+ || key.equals(Property.COMPACTION_WARN_TIME.getKey())
|| key.equals(Property.GENERAL_FILE_NAME_ALLOCATION_BATCH_SIZE_MIN.getKey())
- || key.equals(Property.GENERAL_FILE_NAME_ALLOCATION_BATCH_SIZE_MAX.getKey())
- || key.startsWith(VFS_CONTEXT_CLASSPATH_PROPERTY.getKey())
- || key.startsWith(REPLICATION_PREFIX.getKey());
+ || key.equals(Property.GENERAL_FILE_NAME_ALLOCATION_BATCH_SIZE_MAX.getKey());
}
/**
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
index a0c7f55..0181db3 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/PropertyType.java
@@ -112,11 +112,8 @@
+ "Substitutions of the ACCUMULO_HOME environment variable can be done in the system "
+ "config file using '${env:ACCUMULO_HOME}' or similar."),
- // VFS_CLASSLOADER_CACHE_DIR's default value is a special case, for documentation purposes
- @SuppressWarnings("removal")
ABSOLUTEPATH("absolute path",
- x -> x == null || x.trim().isEmpty() || new Path(x.trim()).isAbsolute()
- || x.equals(Property.VFS_CLASSLOADER_CACHE_DIR.getDefaultValue()),
+ x -> x == null || x.trim().isEmpty() || new Path(x.trim()).isAbsolute(),
"An absolute filesystem path. The filesystem depends on the property."
+ " This is the same as path, but enforces that its root is explicitly specified."),
@@ -248,7 +245,7 @@
|| (suffixCheck.test(x) && new Bounds(lowerBound, upperBound).test(stripUnits.apply(x)));
}
- private static final Pattern SUFFIX_REGEX = Pattern.compile("[^\\d]*$");
+ private static final Pattern SUFFIX_REGEX = Pattern.compile("\\D*$"); // match non-digits at end
private static final Function<String,String> stripUnits =
x -> x == null ? null : SUFFIX_REGEX.matcher(x.trim()).replaceAll("");
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
index 196486f..ed1775c 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
@@ -88,14 +88,6 @@
}
public OverridesOption fromEnv() {
- URL siteUrl = SiteConfiguration.class.getClassLoader().getResource("accumulo-site.xml");
- if (siteUrl != null) {
- throw new IllegalArgumentException("Found deprecated config file 'accumulo-site.xml' on "
- + "classpath. Since 2.0.0, this file was replaced by 'accumulo.properties'. Run the "
- + "following command to convert an old 'accumulo-site.xml' file to the new format: "
- + "accumulo convert-config -x /old/accumulo-site.xml -p /new/accumulo.properties");
- }
-
String configFile = System.getProperty("accumulo.properties", "accumulo.properties");
if (configFile.startsWith("file://")) {
File f;
@@ -166,9 +158,6 @@
config.addConfiguration(overrideConfig);
config.addConfiguration(propsFileConfig);
- // Make sure any deprecated property names aren't using both the old and new name.
- DeprecatedPropertyUtil.sanityCheckManagerProperties(config);
-
var result = new HashMap<String,String>();
config.getKeys().forEachRemaining(orig -> {
String resolved = DeprecatedPropertyUtil.getReplacementName(orig, (log, replacement) -> {
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/cluster/ClusterConfigParser.java b/core/src/main/java/org/apache/accumulo/core/conf/cluster/ClusterConfigParser.java
index 5790fa7..e005701 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/cluster/ClusterConfigParser.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/cluster/ClusterConfigParser.java
@@ -72,14 +72,12 @@
(parentKey == null || parentKey.equals("")) ? "" : parentKey + addTheDot(parentKey);
if (value instanceof String) {
results.put(parent + key, (String) value);
- return;
} else if (value instanceof List) {
((List<?>) value).forEach(l -> {
if (l instanceof String) {
// remove the [] at the ends of toString()
String val = value.toString();
results.put(parent + key, val.substring(1, val.length() - 1).replace(", ", " "));
- return;
} else {
flatten(parent, key, l, results);
}
@@ -90,9 +88,8 @@
map.forEach((k, v) -> flatten(parent + key, k, v, results));
} else if (value instanceof Number) {
results.put(parent + key, value.toString());
- return;
} else {
- throw new RuntimeException("Unhandled object type: " + value.getClass());
+ throw new IllegalStateException("Unhandled object type: " + value.getClass());
}
}
@@ -109,7 +106,7 @@
out.printf(PROPERTY_FORMAT, section.toUpperCase() + "_HOSTS", config.get(section));
} else {
if (section.equals("manager") || section.equals("tserver")) {
- throw new RuntimeException("Required configuration section is missing: " + section);
+ throw new IllegalStateException("Required configuration section is missing: " + section);
}
System.err.println("WARN: " + section + " is missing");
}
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java
deleted file mode 100644
index 1106452..0000000
--- a/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java
+++ /dev/null
@@ -1,98 +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.constraints;
-
-import java.util.List;
-
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.security.AuthorizationContainer;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * This class is replaced by {@link org.apache.accumulo.core.data.constraints.Constraint}
- *
- * @deprecated since 2.1.0 Use {@link org.apache.accumulo.core.data.constraints.Constraint}
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_INTERFACE",
- justification = "Same name used for compatibility during deprecation cycle")
-public interface Constraint extends org.apache.accumulo.core.data.constraints.Constraint {
-
- /**
- * The environment within which a constraint exists.
- */
- interface Environment extends org.apache.accumulo.core.data.constraints.Constraint.Environment {
- /**
- * Gets the key extent of the environment.
- *
- * @return key extent
- */
- KeyExtent getExtent();
-
- /**
- * Gets the user within the environment.
- *
- * @return user
- */
- @Override
- String getUser();
-
- /**
- * Gets the authorizations in the environment.
- *
- * @return authorizations
- */
- @Override
- AuthorizationContainer getAuthorizationsContainer();
- }
-
- /**
- * Gets a short, one-sentence description of what a given violation code means.
- *
- * @param violationCode numeric violation code
- * @return matching violation description
- */
- @Override
- String getViolationDescription(short violationCode);
-
- /**
- * Checks a mutation for constraint violations. If the mutation contains no violations, returns
- * null. Otherwise, returns a list of violation codes.
- *
- * Violation codes must be non-negative. Negative violation codes are reserved for system use.
- *
- * @param env constraint environment
- * @param mutation mutation to check
- * @return list of violation codes, or null if none
- */
- List<Short> check(Environment env, Mutation mutation);
-
- /**
- * Implemented for backwards compatibility.
- *
- * @since 2.1.0
- */
- @Override
- default List<Short> check(org.apache.accumulo.core.data.constraints.Constraint.Environment env,
- Mutation mutation) {
- return check((Environment) env, mutation);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/DefaultKeySizeConstraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/DefaultKeySizeConstraint.java
deleted file mode 100644
index 0c09b46..0000000
--- a/core/src/main/java/org/apache/accumulo/core/constraints/DefaultKeySizeConstraint.java
+++ /dev/null
@@ -1,80 +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.constraints;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * A constraints that limits the size of keys to 1mb.
- *
- * @deprecated since 2.1.0 Use
- * {@link org.apache.accumulo.core.data.constraints.DefaultKeySizeConstraint}
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
- justification = "Same name used for compatibility during deprecation cycle")
-public class DefaultKeySizeConstraint extends
- org.apache.accumulo.core.data.constraints.DefaultKeySizeConstraint implements Constraint {
-
- protected static final short MAX__KEY_SIZE_EXCEEDED_VIOLATION = 1;
- protected static final long maxSize = 1048576; // 1MB default size
-
- @Override
- public String getViolationDescription(short violationCode) {
-
- switch (violationCode) {
- case MAX__KEY_SIZE_EXCEEDED_VIOLATION:
- return "Key was larger than 1MB";
- }
-
- return null;
- }
-
- static final List<Short> NO_VIOLATIONS = new ArrayList<>();
-
- @Override
- public List<Short> check(Constraint.Environment env, Mutation mutation) {
-
- // fast size check
- if (mutation.numBytes() < maxSize) {
- return NO_VIOLATIONS;
- }
-
- List<Short> violations = new ArrayList<>();
-
- for (ColumnUpdate cu : mutation.getUpdates()) {
- int size = mutation.getRow().length;
- size += cu.getColumnFamily().length;
- size += cu.getColumnQualifier().length;
- size += cu.getColumnVisibility().length;
-
- if (size > maxSize) {
- violations.add(MAX__KEY_SIZE_EXCEEDED_VIOLATION);
- }
- }
-
- return violations;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/NoDeleteConstraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/NoDeleteConstraint.java
deleted file mode 100644
index f0f5db5..0000000
--- a/core/src/main/java/org/apache/accumulo/core/constraints/NoDeleteConstraint.java
+++ /dev/null
@@ -1,60 +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.constraints;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * This constraint ensures mutations do not have deletes.
- *
- * @since 2.0.0
- * @deprecated since 2.1.0 Use {@link org.apache.accumulo.core.data.constraints.NoDeleteConstraint}
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
- justification = "Same name used for compatibility during deprecation cycle")
-public class NoDeleteConstraint extends org.apache.accumulo.core.data.constraints.NoDeleteConstraint
- implements Constraint {
-
- @Override
- public String getViolationDescription(short violationCode) {
- if (violationCode == 1) {
- return "Deletes are not allowed";
- }
- return null;
- }
-
- @Override
- public List<Short> check(Constraint.Environment env, Mutation mutation) {
- List<ColumnUpdate> updates = mutation.getUpdates();
- for (ColumnUpdate update : updates) {
- if (update.isDeleted()) {
- return Collections.singletonList((short) 1);
- }
- }
- return null;
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/VisibilityConstraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/VisibilityConstraint.java
deleted file mode 100644
index 2233dff..0000000
--- a/core/src/main/java/org/apache/accumulo/core/constraints/VisibilityConstraint.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.constraints;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-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.util.BadArgumentException;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * A constraint that checks the visibility of columns against the actor's authorizations. Violation
- * codes:
- * <ul>
- * <li>1 = failure to parse visibility expression</li>
- * <li>2 = insufficient authorization</li>
- * </ul>
- *
- * @deprecated since 2.1.0 Use
- * {@link org.apache.accumulo.core.data.constraints.VisibilityConstraint}
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
- justification = "Same name used for compatibility during deprecation cycle")
-public class VisibilityConstraint
- extends org.apache.accumulo.core.data.constraints.VisibilityConstraint implements Constraint {
-
- @Override
- public String getViolationDescription(short violationCode) {
- switch (violationCode) {
- case 1:
- return "Malformed column visibility";
- case 2:
- return "User does not have authorization on column visibility";
- }
-
- return null;
- }
-
- @Override
- public List<Short> check(Constraint.Environment env, Mutation mutation) {
- List<ColumnUpdate> updates = mutation.getUpdates();
-
- HashSet<String> ok = null;
- if (updates.size() > 1) {
- ok = new HashSet<>();
- }
-
- VisibilityEvaluator ve = null;
-
- for (ColumnUpdate update : updates) {
-
- byte[] cv = update.getColumnVisibility();
- if (cv.length > 0) {
- String key = null;
- if (ok != null && ok.contains(key = new String(cv, UTF_8))) {
- continue;
- }
-
- try {
-
- if (ve == null) {
- ve = new VisibilityEvaluator(env.getAuthorizationsContainer());
- }
-
- if (!ve.evaluate(new ColumnVisibility(cv))) {
- return Collections.singletonList((short) 2);
- }
-
- } catch (BadArgumentException | VisibilityParseException bae) {
- return Collections.singletonList((short) 1);
- }
-
- if (ok != null) {
- ok.add(key);
- }
- }
- }
-
- return null;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedInputStream.java b/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedInputStream.java
index 3e37f6f..f3dccea 100644
--- a/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedInputStream.java
+++ b/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedInputStream.java
@@ -38,7 +38,7 @@
public BlockedInputStream(InputStream in, int blockSize, int maxSize) {
if (blockSize == 0) {
- throw new RuntimeException("Invalid block size");
+ throw new IllegalArgumentException("Invalid block size");
}
if (in instanceof DataInputStream) {
this.in = (DataInputStream) in;
@@ -67,7 +67,7 @@
if (readPos == array.length) {
readPos = 0;
} else if (readPos > array.length) {
- throw new RuntimeException(
+ throw new IllegalStateException(
"Unexpected state, this should only ever increase or cycle on the boundary!");
}
return toRet;
@@ -121,7 +121,7 @@
finished = true;
return false;
} else if (size == 0) {
- throw new RuntimeException(
+ throw new IllegalStateException(
"Empty block written, this shouldn't happen with this BlockedOutputStream.");
}
diff --git a/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedOutputStream.java b/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedOutputStream.java
index 05de9f7..6fd59b7 100644
--- a/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedOutputStream.java
+++ b/core/src/main/java/org/apache/accumulo/core/crypto/streams/BlockedOutputStream.java
@@ -55,7 +55,7 @@
@Override
public synchronized void flush() throws IOException {
if (!bb.hasArray()) {
- throw new RuntimeException("BlockedOutputStream has no backing array.");
+ throw new IllegalStateException("BlockedOutputStream has no backing array.");
}
int size = bb.position();
if (size == 0) {
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java b/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
index 88fb8b5..4db8ca8 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
@@ -19,13 +19,14 @@
package org.apache.accumulo.core.data;
import static com.google.common.base.Preconditions.checkArgument;
+import static java.nio.charset.StandardCharsets.UTF_8;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
-import java.util.Set;
+import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.hadoop.io.Text;
/**
@@ -83,6 +84,50 @@
return Collections.unmodifiableList(conditions);
}
+ private String toString(ByteSequence bs) {
+ if (bs == null) {
+ return null;
+ }
+ return new String(bs.toArray(), UTF_8);
+ }
+
+ @Override
+ public String prettyPrint() {
+ StringBuilder sb = new StringBuilder(super.prettyPrint());
+ for (Condition c : conditions) {
+ sb.append(" condition: ");
+ sb.append(toString(c.getFamily()));
+ sb.append(":");
+ sb.append(toString(c.getQualifier()));
+ if (c.getValue() != null && !toString(c.getValue()).isBlank()) {
+ sb.append(" value: ");
+ sb.append(toString(c.getValue()));
+ }
+ if (c.getVisibility() != null && !toString(c.getVisibility()).isBlank()) {
+ sb.append(" visibility: '");
+ sb.append(toString(c.getVisibility()));
+ sb.append("'");
+ }
+ if (c.getTimestamp() != null) {
+ sb.append(" timestamp: ");
+ sb.append("'");
+ sb.append(c.getTimestamp());
+ sb.append("'");
+ }
+ if (c.getIterators().length != 0) {
+ sb.append(" iterator: ");
+ IteratorSetting[] iterators = c.getIterators();
+ for (IteratorSetting its : iterators) {
+ sb.append("'");
+ sb.append(its.toString());
+ sb.append("' ");
+ }
+ }
+ sb.append("\n");
+ }
+ return sb.toString();
+ }
+
@Override
public boolean equals(Object o) {
if (o == this) {
@@ -105,10 +150,4 @@
return result;
}
- @Deprecated
- @Override
- public void setReplicationSources(Set<String> sources) {
- throw new UnsupportedOperationException(
- "Conditional Mutations are not supported for replication");
- }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/data/InstanceId.java b/core/src/main/java/org/apache/accumulo/core/data/InstanceId.java
index 598d359..70e9019 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/InstanceId.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/InstanceId.java
@@ -20,10 +20,9 @@
import java.util.Objects;
import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
/**
* A strongly typed representation of an Accumulo instance ID. The constructor for this class will
@@ -36,7 +35,7 @@
// cache is for canonicalization/deduplication of created objects,
// to limit the number of InstanceId objects in the JVM at any given moment
// WeakReferences are used because we don't need them to stick around any longer than they need to
- static final Cache<String,InstanceId> cache = CacheBuilder.newBuilder().weakValues().build();
+ static final Cache<String,InstanceId> cache = Caffeine.newBuilder().weakValues().build();
private InstanceId(String canonical) {
super(canonical);
@@ -49,12 +48,7 @@
* @return InstanceId object
*/
public static InstanceId of(final String canonical) {
- try {
- return cache.get(canonical, () -> new InstanceId(canonical));
- } catch (ExecutionException e) {
- throw new AssertionError(
- "This should never happen: ID constructor should never return null.");
- }
+ return cache.get(canonical, k -> new InstanceId(canonical));
}
/**
diff --git a/core/src/main/java/org/apache/accumulo/core/data/LoadPlan.java b/core/src/main/java/org/apache/accumulo/core/data/LoadPlan.java
index aecd25f..311042d 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/LoadPlan.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/LoadPlan.java
@@ -125,7 +125,7 @@
"Start row is greater than or equal to end row : " + srs + " " + ers);
}
} else {
- throw new RuntimeException();
+ throw new IllegalStateException();
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
index 53f834c..b8c6c22 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
@@ -19,7 +19,6 @@
package org.apache.accumulo.core.data;
import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
import java.io.DataInput;
import java.io.DataOutput;
@@ -28,9 +27,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashSet;
import java.util.List;
-import java.util.Set;
import org.apache.accumulo.core.dataImpl.thrift.TMutation;
import org.apache.accumulo.core.security.ColumnVisibility;
@@ -109,9 +106,6 @@
private List<ColumnUpdate> updates;
- private static final Set<String> EMPTY = Collections.emptySet();
- private Set<String> replicationSources = EMPTY;
-
private static final byte[] EMPTY_BYTES = new byte[0];
private void serialize() {
@@ -242,10 +236,6 @@
this.entries = tmutation.entries;
this.values = ByteBufferUtil.toBytesList(tmutation.values);
- if (tmutation.isSetSources()) {
- this.replicationSources = new HashSet<>(tmutation.sources);
- }
-
if (this.row == null) {
throw new IllegalArgumentException("null row");
}
@@ -265,7 +255,6 @@
this.data = m.data;
this.entries = m.entries;
this.values = m.values;
- this.replicationSources = m.replicationSources;
}
/**
@@ -1360,58 +1349,6 @@
return entries;
}
- /**
- * Add a new element to the set of peers which this Mutation originated from
- *
- * @param peer the peer to add
- * @since 1.7.0
- * @deprecated The feature pertaining to this method was deprecated in 2.1.0, but this method was
- * overlooked when annotating the code. It is being marked as deprecated in 2.1.1 in
- * order to correct that oversight, and will be removed in 3.0.0 with the rest of the
- * code pertaining to this feature.
- */
- @Deprecated(since = "2.1.1")
- public void addReplicationSource(String peer) {
- if (replicationSources == null || replicationSources == EMPTY) {
- replicationSources = new HashSet<>();
- }
-
- replicationSources.add(peer);
- }
-
- /**
- * Set the replication peers which this Mutation originated from
- *
- * @param sources Set of peer names which have processed this update
- * @since 1.7.0
- * @deprecated The feature pertaining to this method was deprecated in 2.1.0, but this method was
- * overlooked when annotating the code. It is being marked as deprecated in 2.1.1 in
- * order to correct that oversight, and will be removed in 3.0.0 with the rest of the
- * code pertaining to this feature.
- */
- @Deprecated(since = "2.1.1")
- public void setReplicationSources(Set<String> sources) {
- requireNonNull(sources);
- this.replicationSources = sources;
- }
-
- /**
- * Return the replication sources for this Mutation
- *
- * @return An unmodifiable view of the replication sources
- * @deprecated The feature pertaining to this method was deprecated in 2.1.0, but this method was
- * overlooked when annotating the code. It is being marked as deprecated in 2.1.1 in
- * order to correct that oversight, and will be removed in 3.0.0 with the rest of the
- * code pertaining to this feature.
- */
- @Deprecated(since = "2.1.1")
- public Set<String> getReplicationSources() {
- if (replicationSources == null) {
- return EMPTY;
- }
- return Collections.unmodifiableSet(replicationSources);
- }
-
@Override
public void readFields(DataInput in) throws IOException {
@@ -1454,9 +1391,9 @@
if ((first & 0x02) == 0x02) {
int numMutations = WritableUtils.readVInt(in);
- this.replicationSources = new HashSet<>();
for (int i = 0; i < numMutations; i++) {
- replicationSources.add(WritableUtils.readString(in));
+ // consume the replication sources that may have been previously serialized
+ WritableUtils.readString(in);
}
}
}
@@ -1530,10 +1467,9 @@
final byte[] integerBuffer = new byte[5];
serialize();
byte hasValues = (values == null) ? 0 : (byte) 1;
- if (!replicationSources.isEmpty()) {
- // Use 2nd least-significant bit for whether or not we have replication sources
- hasValues = (byte) (0x02 | hasValues);
- }
+ // When replication sources were supported, we used the 2nd least-significant bit to denote
+ // their presence, but this is no longer used; kept here for historical explanation only
+ // hasValues = (byte) (0x02 | hasValues);
out.write((byte) (0x80 | hasValues));
UnsynchronizedBuffer.writeVInt(out, integerBuffer, row.length);
@@ -1550,12 +1486,6 @@
out.write(val);
}
}
- if ((0x02 & hasValues) == 0x02) {
- UnsynchronizedBuffer.writeVInt(out, integerBuffer, replicationSources.size());
- for (String source : replicationSources) {
- WritableUtils.writeString(out, source);
- }
- }
}
@Override
@@ -1591,9 +1521,6 @@
ByteBuffer otherData = m.serializedSnapshot();
if (Arrays.equals(row, m.row) && entries == m.entries && myData.equals(otherData)) {
// If two mutations don't have the same
- if (!replicationSources.equals(m.replicationSources)) {
- return false;
- }
if (values == null && m.values == null) {
return true;
}
@@ -1631,12 +1558,7 @@
this.serialize();
}
ByteBuffer data = serializedSnapshot();
- TMutation tmutation =
- new TMutation(ByteBuffer.wrap(row), data, ByteBufferUtil.toByteBuffers(values), entries);
- if (!this.replicationSources.isEmpty()) {
- tmutation.setSources(new ArrayList<>(replicationSources));
- }
- return tmutation;
+ return new TMutation(ByteBuffer.wrap(row), data, ByteBufferUtil.toByteBuffers(values), entries);
}
/**
diff --git a/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java b/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java
index 90a8c74..c201113 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java
@@ -18,10 +18,8 @@
*/
package org.apache.accumulo.core.data;
-import java.util.concurrent.ExecutionException;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
/**
* A strongly typed representation of a namespace ID. This class cannot be used to get a namespace
@@ -35,7 +33,7 @@
// cache is for canonicalization/deduplication of created objects,
// to limit the number of NamespaceId objects in the JVM at any given moment
// WeakReferences are used because we don't need them to stick around any longer than they need to
- static final Cache<String,NamespaceId> cache = CacheBuilder.newBuilder().weakValues().build();
+ static final Cache<String,NamespaceId> cache = Caffeine.newBuilder().weakValues().build();
private NamespaceId(String canonical) {
super(canonical);
@@ -48,11 +46,6 @@
* @return NamespaceId object
*/
public static NamespaceId of(final String canonical) {
- try {
- return cache.get(canonical, () -> new NamespaceId(canonical));
- } catch (ExecutionException e) {
- throw new AssertionError(
- "This should never happen: ID constructor should never return null.");
- }
+ return cache.get(canonical, k -> new NamespaceId(canonical));
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/data/TableId.java b/core/src/main/java/org/apache/accumulo/core/data/TableId.java
index 9493d8d..a97c9d2 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/TableId.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/TableId.java
@@ -18,10 +18,8 @@
*/
package org.apache.accumulo.core.data;
-import java.util.concurrent.ExecutionException;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
/**
* A strongly typed representation of a table ID. This class cannot be used to get a table ID from a
@@ -35,7 +33,7 @@
// cache is for canonicalization/deduplication of created objects,
// to limit the number of TableId objects in the JVM at any given moment
// WeakReferences are used because we don't need them to stick around any longer than they need to
- static final Cache<String,TableId> cache = CacheBuilder.newBuilder().weakValues().build();
+ static final Cache<String,TableId> cache = Caffeine.newBuilder().weakValues().build();
private TableId(final String canonical) {
super(canonical);
@@ -48,11 +46,6 @@
* @return TableId object
*/
public static TableId of(final String canonical) {
- try {
- return cache.get(canonical, () -> new TableId(canonical));
- } catch (ExecutionException e) {
- throw new AssertionError(
- "This should never happen: ID constructor should never return null.");
- }
+ return cache.get(canonical, k -> new TableId(canonical));
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/data/TabletId.java b/core/src/main/java/org/apache/accumulo/core/data/TabletId.java
index 51bfa1a..2d676d1 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/TabletId.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/TabletId.java
@@ -31,12 +31,6 @@
*/
TableId getTable();
- /**
- * @deprecated use {@link #getTable()} and {@link TableId#canonical()} instead
- */
- @Deprecated(since = "2.1.0")
- Text getTableId();
-
Text getEndRow();
Text getPrevEndRow();
diff --git a/core/src/main/java/org/apache/accumulo/core/data/constraints/VisibilityConstraint.java b/core/src/main/java/org/apache/accumulo/core/data/constraints/VisibilityConstraint.java
index 62531c5..87b60df 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/constraints/VisibilityConstraint.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/constraints/VisibilityConstraint.java
@@ -24,12 +24,11 @@
import java.util.HashSet;
import java.util.List;
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
+import org.apache.accumulo.core.data.ArrayByteSequence;
import org.apache.accumulo.core.data.ColumnUpdate;
import org.apache.accumulo.core.data.Mutation;
-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.util.BadArgumentException;
/**
* A constraint that checks the visibility of columns against the actor's authorizations. Violation
@@ -64,7 +63,7 @@
ok = new HashSet<>();
}
- VisibilityEvaluator ve = null;
+ AccessEvaluator ve = null;
for (ColumnUpdate update : updates) {
@@ -78,14 +77,15 @@
try {
if (ve == null) {
- ve = new VisibilityEvaluator(env.getAuthorizationsContainer());
+ var authContainer = env.getAuthorizationsContainer();
+ ve = AccessEvaluator.of(auth -> authContainer.contains(new ArrayByteSequence(auth)));
}
- if (!ve.evaluate(new ColumnVisibility(cv))) {
+ if (!ve.canAccess(cv)) {
return Collections.singletonList((short) 2);
}
- } catch (BadArgumentException | VisibilityParseException bae) {
+ } catch (InvalidAccessExpressionException iaee) {
return Collections.singletonList((short) 1);
}
diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
index c0a40dd..1069a5e 100644
--- a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
+++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
@@ -20,6 +20,8 @@
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Objects.requireNonNull;
+import static org.apache.accumulo.core.util.RowRangeUtil.requireKeyExtentDataRange;
+import static org.apache.accumulo.core.util.RowRangeUtil.stripZeroTail;
import java.io.ByteArrayOutputStream;
import java.io.DataInput;
@@ -46,8 +48,7 @@
import org.apache.accumulo.core.data.TabletId;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
-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.MetadataSchema.TabletsSection;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
import org.apache.accumulo.core.util.ByteBufferUtil;
@@ -389,6 +390,65 @@
return new Range(metadataPrevRow, prevEndRow() == null, toMetaRow(), true);
}
+ /**
+ * Creates a KeyExtent which represents the intersection of this KeyExtent and the passed in
+ * range.
+ * <p>
+ * <b>Note:</b> The range provided must be a range that is derived from a KeyExtent. This means
+ * the range must be in the format of a row range and also requires an exclusive start key, which
+ * is the format that {@link #toDataRange()} uses
+ *
+ * @param range range to clip to
+ * @return the intersection of this KeyExtent and the given range
+ * @throws IllegalArgumentException if the KeyExtent and range do not overlap
+ */
+ public KeyExtent clip(Range range) {
+ return clip(range, false);
+ }
+
+ /**
+ * Creates a KeyExtent which represents the intersection of this KeyExtent and the passed in
+ * range. Unlike {@link #clip(Range)}, this method can optionally return null if the given range
+ * and this KeyExtent do not overlap, instead of throwing an exception. The returnNullIfDisjoint
+ * parameter controls this behavior.
+ * <p>
+ * <b>Note:</b> The range provided must be a range that is derived from a KeyExtent. This means
+ * the range must be in the format of a row range and also requires an exclusive start key, which
+ * is the format that {@link #toDataRange()} uses
+ *
+ * @param range range to clip to
+ * @param returnNullIfDisjoint true to return null if ranges are disjoint, false to throw an
+ * exception
+ * @return the intersection of this KeyExtent and the given range, or null if given range and this
+ * KeyExtent do not overlap and returnNullIfDisjoint is true
+ * @throws IllegalArgumentException if the KeyExtent and range does not overlap and
+ * returnNullIfDisjoint is false
+ *
+ * @see KeyExtent#clip(Range)
+ **/
+ public KeyExtent clip(Range range, boolean returnNullIfDisjoint) {
+ // This will require a range that matches a row range generated by toDataRange()
+ // This range itself will be required to be an inclusive start and exclusive end
+ // The start and end rows will be required to be exclusive keys (ending in 0x00)
+ requireKeyExtentDataRange(range);
+
+ // If returnNullIfDisjoint is false then this will throw an exception if
+ // the ranges are disjoint, otherwise we can just return null
+ final Range clippedRange = this.toDataRange().clip(range, returnNullIfDisjoint);
+ if (clippedRange == null) {
+ return null;
+ }
+
+ // Build the new KeyExtent with the clipped range. We need to strip off the ending byte
+ // which will essentially reverse what toDataRange() does
+ Text endRow = clippedRange.getEndKey() != null
+ ? new Text(stripZeroTail(clippedRange.getEndKey().getRowData()).toArray()) : null;
+ Text prevEndRow = clippedRange.getStartKey() != null
+ ? new Text(stripZeroTail(clippedRange.getStartKey().getRowData()).toArray()) : null;
+
+ return new KeyExtent(tableId, endRow, prevEndRow);
+ }
+
private boolean startsAfter(KeyExtent other) {
KeyExtent nke = requireNonNull(other);
return tableId().compareTo(nke.tableId()) > 0 || (prevEndRow() != null && nke.endRow() != null
@@ -475,12 +535,16 @@
return prevExtent.endRow().equals(prevEndRow());
}
+ public boolean isSystemTable() {
+ return AccumuloTable.allTableIds().contains(tableId());
+ }
+
public boolean isMeta() {
- return tableId().equals(MetadataTable.ID) || isRootTablet();
+ return tableId().equals(AccumuloTable.METADATA.tableId()) || isRootTablet();
}
public boolean isRootTablet() {
- return tableId().equals(RootTable.ID);
+ return tableId().equals(AccumuloTable.ROOT.tableId());
}
public String obscured() {
@@ -488,7 +552,7 @@
try {
digester = MessageDigest.getInstance(OBSCURING_HASH_ALGORITHM);
} catch (NoSuchAlgorithmException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
if (endRow() != null && endRow().getLength() > 0) {
digester.update(endRow().getBytes(), 0, endRow().getLength());
diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
index e5a16dd..4700980 100644
--- a/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
@@ -42,12 +42,6 @@
}
@Override
- @Deprecated(since = "2.1.0")
- public Text getTableId() {
- return new Text(ke.tableId().canonical());
- }
-
- @Override
public Text getEndRow() {
return ke.endRow();
}
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java
index 3ea322e..78f107f 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java
@@ -20,6 +20,7 @@
import static java.nio.charset.StandardCharsets.UTF_8;
+import java.time.Duration;
import java.time.ZoneOffset;
import java.time.format.DateTimeFormatter;
import java.util.ArrayList;
@@ -32,16 +33,15 @@
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
-import java.util.concurrent.TimeUnit;
import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus;
import org.apache.accumulo.core.fate.zookeeper.FateLock;
import org.apache.accumulo.core.fate.zookeeper.FateLock.FateLockPath;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath;
import org.apache.accumulo.core.fate.zookeeper.ZooReader;
import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.ServiceLockPath;
import org.apache.accumulo.core.util.FastFormat;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
@@ -368,7 +368,7 @@
long timeCreated = zs.timeCreated(tid);
- zs.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+ zs.unreserve(tid, Duration.ZERO);
if (includeByStatus(status, filterStatus) && includeByTxid(tid, filterTxid)) {
statuses
@@ -471,7 +471,7 @@
break;
}
- zs.unreserve(txid, 0, TimeUnit.MILLISECONDS);
+ zs.unreserve(txid, Duration.ZERO);
return state;
}
@@ -515,7 +515,7 @@
break;
}
- zs.unreserve(txid, 0, TimeUnit.MILLISECONDS);
+ zs.unreserve(txid, Duration.ZERO);
return state;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java b/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java
index ca016d0..bd2bd52 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/AgeOffStore.java
@@ -19,13 +19,13 @@
package org.apache.accumulo.core.fate;
import java.io.Serializable;
+import java.time.Duration;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -108,7 +108,7 @@
}
} finally {
- store.unreserve(txid, 0, TimeUnit.MILLISECONDS);
+ store.unreserve(txid, Duration.ZERO);
}
} catch (Exception e) {
log.warn("Failed to age off FATE tx " + FateTxId.formatTid(txid), e);
@@ -138,7 +138,7 @@
break;
}
} finally {
- store.unreserve(txid, 0, TimeUnit.MILLISECONDS);
+ store.unreserve(txid, Duration.ZERO);
}
}
}
@@ -166,8 +166,8 @@
}
@Override
- public void unreserve(long tid, long deferTime, TimeUnit deferTimeUnit) {
- store.unreserve(tid, deferTime, deferTimeUnit);
+ public void unreserve(long tid, Duration deferTime) {
+ store.unreserve(tid, deferTime);
}
@Override
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java
index 4eb690e..83793ec 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/Fate.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/Fate.java
@@ -18,6 +18,7 @@
*/
package org.apache.accumulo.core.fate;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.FAILED;
@@ -29,12 +30,12 @@
import static org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus.UNKNOWN;
import static org.apache.accumulo.core.util.ShutdownUtil.isIOException;
+import java.time.Duration;
import java.util.EnumSet;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Function;
@@ -60,8 +61,8 @@
private final TStore<T> store;
private final T environment;
- private ScheduledThreadPoolExecutor fatePoolWatcher;
- private ExecutorService executor;
+ private final ScheduledThreadPoolExecutor fatePoolWatcher;
+ private final ExecutorService executor;
private static final EnumSet<TStatus> FINISHED_STATES = EnumSet.of(FAILED, SUCCESSFUL, UNKNOWN);
@@ -132,7 +133,7 @@
runnerLog.error("Uncaught exception in FATE runner thread.", e);
} finally {
if (tid != null) {
- store.unreserve(tid, deferTime, TimeUnit.MILLISECONDS);
+ store.unreserve(tid, Duration.ofMillis(deferTime));
}
}
}
@@ -161,7 +162,7 @@
while (true) {
// Nothing is going to work well at this point, so why even try. Just wait for the end,
// preventing this FATE thread from processing further work and likely failing.
- UtilWaitThread.sleepUninterruptibly(1, MINUTES);
+ sleepUninterruptibly(1, MINUTES);
}
}
}
@@ -227,24 +228,16 @@
/**
* Creates a Fault-tolerant executor.
- * <p>
- * Note: Users of this class should call {@link #startTransactionRunners(AccumuloConfiguration)}
- * to launch the worker threads after creating a Fate object.
*
* @param toLogStrFunc A function that converts Repo to Strings that are suitable for logging
*/
- public Fate(T environment, TStore<T> store, Function<Repo<T>,String> toLogStrFunc) {
+ public Fate(T environment, TStore<T> store, Function<Repo<T>,String> toLogStrFunc,
+ AccumuloConfiguration conf) {
this.store = FateLogger.wrap(store, toLogStrFunc);
this.environment = environment;
- }
-
- /**
- * Launches the specified number of worker threads.
- */
- public void startTransactionRunners(AccumuloConfiguration conf) {
final ThreadPoolExecutor pool = ThreadPools.getServerThreadPools().createExecutorService(conf,
Property.MANAGER_FATE_THREADPOOL_SIZE, true);
- fatePoolWatcher =
+ this.fatePoolWatcher =
ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(conf);
ThreadPools.watchCriticalScheduledTask(fatePoolWatcher.schedule(() -> {
// resize the pool if the property changed
@@ -269,7 +262,7 @@
}
}
}, 3, SECONDS));
- executor = pool;
+ this.executor = pool;
}
// get a transaction id back to the requester before doing any work
@@ -290,7 +283,7 @@
store.push(tid, repo);
} catch (StackOverflowException e) {
// this should not happen
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
@@ -303,7 +296,7 @@
store.setStatus(tid, SUBMITTED);
}
} finally {
- store.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+ store.unreserve(tid, Duration.ZERO);
}
}
@@ -339,7 +332,7 @@
return false;
}
} finally {
- store.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+ store.unreserve(tid, Duration.ZERO);
}
} else {
// reserved, lets retry.
@@ -370,7 +363,7 @@
break;
}
} finally {
- store.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+ store.unreserve(tid, Duration.ZERO);
}
}
@@ -383,7 +376,7 @@
}
return (String) store.getTransactionInfo(tid, TxInfo.RETURN_VALUE);
} finally {
- store.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+ store.unreserve(tid, Duration.ZERO);
}
}
@@ -397,7 +390,7 @@
}
return (Exception) store.getTransactionInfo(tid, TxInfo.EXCEPTION);
} finally {
- store.unreserve(tid, 0, TimeUnit.MILLISECONDS);
+ store.unreserve(tid, Duration.ZERO);
}
}
@@ -407,7 +400,9 @@
public void shutdown() {
keepRunning.set(false);
fatePoolWatcher.shutdown();
- executor.shutdown();
+ if (executor != null) {
+ executor.shutdown();
+ }
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java
index 4a216f1..0b48c3b 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/ReadOnlyTStore.java
@@ -19,9 +19,9 @@
package org.apache.accumulo.core.fate;
import java.io.Serializable;
+import java.time.Duration;
import java.util.EnumSet;
import java.util.List;
-import java.util.concurrent.TimeUnit;
/**
* Read only access to a Transaction Store.
@@ -79,9 +79,8 @@
* @param tid transaction id, previously reserved.
* @param deferTime time to keep this transaction out of the pool used in the {@link #reserve()
* reserve} method. must be non-negative.
- * @param deferTimeUnit the time unit of deferTime
*/
- void unreserve(long tid, long deferTime, TimeUnit deferTimeUnit);
+ void unreserve(long tid, Duration deferTime);
/**
* Get the current operation for the given transaction id.
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java
index ff7a297..c3de5f2 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java
@@ -18,9 +18,10 @@
*/
package org.apache.accumulo.core.fate;
+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 org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
@@ -28,7 +29,8 @@
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;
-import java.security.SecureRandom;
+import java.io.UncheckedIOException;
+import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
@@ -37,12 +39,12 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.concurrent.TimeUnit;
import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
import org.apache.accumulo.core.util.FastFormat;
+import org.apache.accumulo.core.util.time.NanoTime;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
@@ -62,8 +64,7 @@
private ZooReaderWriter zk;
private String lastReserved = "";
private Set<Long> reserved;
- private Map<Long,Long> deferred;
- private static final SecureRandom random = new SecureRandom();
+ private Map<Long,NanoTime> deferred;
private long statusChangeEvents = 0;
private int reservationsWaiting = 0;
@@ -74,7 +75,7 @@
oos.writeObject(o);
return baos.toByteArray();
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
}
@@ -85,8 +86,10 @@
try (ByteArrayInputStream bais = new ByteArrayInputStream(ser);
ObjectInputStream ois = new ObjectInputStream(bais)) {
return ois.readObject();
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ } catch (ReflectiveOperationException e) {
+ throw new IllegalStateException(e);
}
}
@@ -118,14 +121,14 @@
while (true) {
try {
// looking at the code for SecureRandom, it appears to be thread safe
- long tid = random.nextLong() & 0x7fffffffffffffffL;
+ long tid = RANDOM.get().nextLong() & 0x7fffffffffffffffL;
zk.putPersistentData(getTXPath(tid), TStatus.NEW.name().getBytes(UTF_8),
NodeExistsPolicy.FAIL);
return tid;
} catch (NodeExistsException nee) {
// exist, so just try another random #
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (KeeperException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
}
}
@@ -161,7 +164,7 @@
}
if (deferred.containsKey(tid)) {
- if ((deferred.get(tid) - System.nanoTime()) < 0) {
+ if (deferred.get(tid).elapsed().compareTo(Duration.ZERO) > 0) {
deferred.remove(tid);
} else {
continue;
@@ -188,7 +191,7 @@
} catch (NoNodeException nne) {
// node deleted after we got the list of children, its ok
unreserve(tid);
- } catch (Exception e) {
+ } catch (KeeperException | InterruptedException | RuntimeException e) {
unreserve(tid);
throw e;
}
@@ -200,19 +203,18 @@
if (deferred.isEmpty()) {
this.wait(5000);
} else {
- long currTime = System.nanoTime();
- long minWait =
- deferred.values().stream().mapToLong(l -> l - currTime).min().getAsLong();
- long waitTime = TimeUnit.MILLISECONDS.convert(minWait, TimeUnit.NANOSECONDS);
- if (waitTime > 0) {
- this.wait(Math.min(waitTime, 5000));
+ var now = NanoTime.now();
+ long minWait = deferred.values().stream()
+ .mapToLong(nanoTime -> nanoTime.subtract(now).toMillis()).min().orElseThrow();
+ if (minWait > 0) {
+ this.wait(Math.min(minWait, 5000));
}
}
}
}
}
- } catch (InterruptedException | KeeperException e) {
- throw new RuntimeException(e);
+ } catch (KeeperException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
}
@@ -225,7 +227,7 @@
try {
this.wait(1000);
} catch (InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
@@ -270,10 +272,9 @@
}
@Override
- public void unreserve(long tid, long deferTime, TimeUnit deferTimeUnit) {
- deferTime = TimeUnit.NANOSECONDS.convert(deferTime, deferTimeUnit);
+ public void unreserve(long tid, Duration deferTime) {
- if (deferTime < 0) {
+ if (deferTime.isNegative()) {
throw new IllegalArgumentException("deferTime < 0 : " + deferTime);
}
@@ -283,8 +284,8 @@
"Tried to unreserve id that was not reserved " + FateTxId.formatTid(tid));
}
- if (deferTime > 0) {
- deferred.put(tid, System.nanoTime() + deferTime);
+ if (deferTime.compareTo(Duration.ZERO) > 0) {
+ deferred.put(tid, NanoTime.nowPlus(deferTime));
}
this.notifyAll();
@@ -317,7 +318,7 @@
return null;
}
} catch (KeeperException.NoNodeException ex) {
- throw new RuntimeException(ex);
+ throw new IllegalStateException(ex);
}
byte[] ser = zk.getData(txpath + "/" + top);
@@ -328,8 +329,8 @@
log.debug("zookeeper error reading " + txpath + ": " + ex, ex);
sleepUninterruptibly(100, MILLISECONDS);
continue;
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (KeeperException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
}
return null;
@@ -369,8 +370,8 @@
zk.putPersistentSequential(txpath + "/repo_", serialize(repo));
} catch (StackOverflowException soe) {
throw soe;
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (KeeperException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
}
@@ -385,8 +386,8 @@
throw new IllegalStateException("Tried to pop when empty " + FateTxId.formatTid(tid));
}
zk.recursiveDelete(txpath + "/" + top, NodeMissingPolicy.SKIP);
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (KeeperException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
}
@@ -395,8 +396,8 @@
return TStatus.valueOf(new String(zk.getData(getTXPath(tid)), UTF_8));
} catch (NoNodeException nne) {
return TStatus.UNKNOWN;
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (KeeperException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
}
@@ -425,7 +426,7 @@
try {
this.wait(5000);
} catch (InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
}
@@ -439,8 +440,8 @@
try {
zk.putPersistentData(getTXPath(tid), status.name().getBytes(UTF_8),
NodeExistsPolicy.OVERWRITE);
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (KeeperException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
synchronized (this) {
@@ -455,8 +456,8 @@
try {
zk.recursiveDelete(getTXPath(tid), NodeMissingPolicy.SKIP);
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (KeeperException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
}
@@ -476,8 +477,8 @@
data[1] = ' ';
zk.putPersistentData(getTXPath(tid) + "/" + txInfo, data, NodeExistsPolicy.OVERWRITE);
}
- } catch (Exception e2) {
- throw new RuntimeException(e2);
+ } catch (KeeperException | InterruptedException e2) {
+ throw new IllegalStateException(e2);
}
}
@@ -499,8 +500,8 @@
}
} catch (NoNodeException nne) {
return null;
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (KeeperException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
}
@@ -513,8 +514,8 @@
l.add(parseTid(txid));
}
return l;
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (KeeperException | InterruptedException e) {
+ throw new IllegalStateException(e);
}
}
@@ -541,7 +542,7 @@
} catch (KeeperException.NoNodeException e) {
return Collections.emptyList();
} catch (KeeperException | InterruptedException e1) {
- throw new RuntimeException(e1);
+ throw new IllegalStateException(e1);
}
ops = new ArrayList<>(ops);
@@ -561,7 +562,7 @@
// children changed so start over
continue outer;
} catch (KeeperException | InterruptedException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/DistributedReadWriteLock.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/DistributedReadWriteLock.java
index 852a40b..0bf4af1 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/DistributedReadWriteLock.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/DistributedReadWriteLock.java
@@ -40,7 +40,7 @@
*/
public class DistributedReadWriteLock implements java.util.concurrent.locks.ReadWriteLock {
- static enum LockType {
+ public enum LockType {
READ, WRITE,
}
@@ -107,7 +107,11 @@
private static final Logger log = LoggerFactory.getLogger(DistributedReadWriteLock.class);
- static class ReadLock implements Lock {
+ public static interface DistributedLock extends Lock {
+ LockType getType();
+ }
+
+ static class ReadLock implements DistributedLock {
QueueLock qlock;
byte[] userData;
@@ -125,7 +129,8 @@
this.entry = entry;
}
- protected LockType lockType() {
+ @Override
+ public LockType getType() {
return LockType.READ;
}
@@ -154,9 +159,9 @@
@Override
public boolean tryLock() {
if (entry == -1) {
- entry = qlock.addEntry(new ParsedLock(this.lockType(), this.userData).getLockData());
+ entry = qlock.addEntry(new ParsedLock(this.getType(), this.userData).getLockData());
log.info("Added lock entry {} userData {} lockType {}", entry,
- new String(this.userData, UTF_8), lockType());
+ new String(this.userData, UTF_8), getType());
}
SortedMap<Long,byte[]> entries = qlock.getEarlierEntries(entry);
for (Entry<Long,byte[]> entry : entries.entrySet()) {
@@ -169,7 +174,7 @@
}
}
throw new IllegalStateException("Did not find our own lock in the queue: " + this.entry
- + " userData " + new String(this.userData, UTF_8) + " lockType " + lockType());
+ + " userData " + new String(this.userData, UTF_8) + " lockType " + getType());
}
@Override
@@ -193,7 +198,7 @@
return;
}
log.debug("Removing lock entry {} userData {} lockType {}", entry,
- new String(this.userData, UTF_8), lockType());
+ new String(this.userData, UTF_8), getType());
qlock.removeEntry(entry);
entry = -1;
}
@@ -215,22 +220,22 @@
}
@Override
- protected LockType lockType() {
+ public LockType getType() {
return LockType.WRITE;
}
@Override
public boolean tryLock() {
if (entry == -1) {
- entry = qlock.addEntry(new ParsedLock(this.lockType(), this.userData).getLockData());
+ entry = qlock.addEntry(new ParsedLock(this.getType(), this.userData).getLockData());
log.info("Added lock entry {} userData {} lockType {}", entry,
- new String(this.userData, UTF_8), lockType());
+ new String(this.userData, UTF_8), getType());
}
SortedMap<Long,byte[]> entries = qlock.getEarlierEntries(entry);
Iterator<Entry<Long,byte[]>> iterator = entries.entrySet().iterator();
if (!iterator.hasNext()) {
throw new IllegalStateException("Did not find our own lock in the queue: " + this.entry
- + " userData " + new String(this.userData, UTF_8) + " lockType " + lockType());
+ + " userData " + new String(this.userData, UTF_8) + " lockType " + getType());
}
return iterator.next().getKey().equals(entry);
}
@@ -244,7 +249,7 @@
this.data = Arrays.copyOf(data, data.length);
}
- public static Lock recoverLock(QueueLock qlock, byte[] data) {
+ public static DistributedLock recoverLock(QueueLock qlock, byte[] data) {
SortedMap<Long,byte[]> entries = qlock.getEarlierEntries(Long.MAX_VALUE);
for (Entry<Long,byte[]> entry : entries.entrySet()) {
ParsedLock parsed = new ParsedLock(entry.getValue());
@@ -261,12 +266,12 @@
}
@Override
- public Lock readLock() {
+ public DistributedLock readLock() {
return new ReadLock(qlock, data);
}
@Override
- public Lock writeLock() {
+ public DistributedLock writeLock() {
return new WriteLock(qlock, data);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/FateLock.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/FateLock.java
index fa0e4db..effa070 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/FateLock.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/FateLock.java
@@ -83,8 +83,8 @@
zoo.putPersistentData(path.toString(), new byte[] {}, NodeExistsPolicy.SKIP);
}
}
- } catch (Exception ex) {
- throw new RuntimeException(ex);
+ } catch (KeeperException | InterruptedException ex) {
+ throw new IllegalStateException(ex);
}
}
@@ -112,8 +112,8 @@
// ignored
}
}
- } catch (Exception ex) {
- throw new RuntimeException(ex);
+ } catch (KeeperException | InterruptedException ex) {
+ throw new IllegalStateException(ex);
}
return result;
}
@@ -128,8 +128,8 @@
} catch (NotEmptyException nee) {
// the path had other lock nodes, no big deal
}
- } catch (Exception ex) {
- throw new RuntimeException(ex);
+ } catch (KeeperException | InterruptedException ex) {
+ throw new IllegalStateException(ex);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
index fe8807c..fc2b6f1 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
@@ -19,19 +19,22 @@
package org.apache.accumulo.core.fate.zookeeper;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
-import java.security.SecureRandom;
import java.util.Collections;
import java.util.ConcurrentModificationException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.LockSupport;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.ServiceLockPath;
+import org.apache.accumulo.core.lock.ServiceLockData;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.WatchedEvent;
@@ -62,7 +65,6 @@
private final HashMap<String,List<String>> childrenCache;
private final ZooReader zReader;
- private static final SecureRandom random = new SecureRandom();
private volatile boolean closed = false;
@@ -270,7 +272,7 @@
}
LockSupport.parkNanos(sleepTime);
if (sleepTime < 10_000) {
- sleepTime = (int) (sleepTime + sleepTime * random.nextDouble());
+ sleepTime = (int) (sleepTime + sleepTime * RANDOM.get().nextDouble());
}
}
}
@@ -525,13 +527,21 @@
}
}
- public byte[] getLockData(ServiceLockPath path) {
+ public Optional<ServiceLockData> getLockData(ServiceLockPath path) {
List<String> children = ServiceLock.validateAndSort(path, getChildren(path.toString()));
if (children == null || children.isEmpty()) {
- return null;
+ return Optional.empty();
}
String lockNode = children.get(0);
- return get(path + "/" + lockNode);
+
+ byte[] lockData = get(path + "/" + lockNode);
+ if (log.isTraceEnabled()) {
+ log.trace("Data from lockNode {} is {}", lockNode, new String(lockData, UTF_8));
+ }
+ if (lockData == null) {
+ lockData = new byte[0];
+ }
+ return ServiceLockData.parse(lockData);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java
index 39fd5a2..e30c942 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java
@@ -27,10 +27,12 @@
/**
* A factory for {@link ZooCache} instances.
* <p>
- * Implementation note: We are using the instances map to track all the instances that have been
- * created, so we can explicitly close them when the last legacy client has gone away. This is part
- * of the "SingletonManager" code, and it is likely that ZooCacheFactory and ZooKeeperInstance can
- * be removed when legacy client code support is no longer required.
+ * Implementation note: We were using the instances map to track all the instances that have been
+ * created, so we could explicitly close them when the SingletonManager detected that the last
+ * legacy client (using Connector/ZooKeeperInstance) has gone away. This class may no longer be
+ * needed, since the legacy client code has been removed, so long as the ZooCache instances it is
+ * tracking are managed as resources within ClientContext or ServerContext, and explicitly closed
+ * when those are closed.
*/
public class ZooCacheFactory {
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java
index dbd80b2..1df0b54 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java
@@ -19,9 +19,8 @@
package org.apache.accumulo.core.fate.zookeeper;
import static java.util.Objects.requireNonNull;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
+import java.time.Duration;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
@@ -42,8 +41,9 @@
private static final Logger log = LoggerFactory.getLogger(ZooReader.class);
protected static final RetryFactory RETRY_FACTORY =
- Retry.builder().maxRetries(10).retryAfter(250, MILLISECONDS).incrementBy(250, MILLISECONDS)
- .maxWait(2, MINUTES).backOffFactor(1.5).logInterval(3, MINUTES).createFactory();
+ Retry.builder().maxRetries(10).retryAfter(Duration.ofMillis(250))
+ .incrementBy(Duration.ofMillis(250)).maxWait(Duration.ofMinutes(2)).backOffFactor(1.5)
+ .logInterval(Duration.ofMinutes(3)).createFactory();
protected final String keepers;
protected final int timeout;
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java
index 2f94173..91085f2 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java
@@ -20,10 +20,10 @@
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import java.io.IOException;
import java.net.UnknownHostException;
-import java.security.SecureRandom;
import java.util.HashMap;
import java.util.Map;
@@ -63,8 +63,6 @@
private static Map<String,ZooSessionInfo> sessions = new HashMap<>();
- private static final SecureRandom random = new SecureRandom();
-
static {
SingletonManager.register(new SingletonService() {
@@ -157,7 +155,7 @@
long duration = NANOSECONDS.toMillis(stopTime - startTime);
if (duration > 2L * timeout) {
- throw new RuntimeException("Failed to connect to zookeeper (" + host
+ throw new IllegalStateException("Failed to connect to zookeeper (" + host
+ ") within 2x zookeeper timeout period " + timeout);
}
@@ -171,7 +169,7 @@
}
UtilWaitThread.sleep(sleepTime);
if (sleepTime < 10000) {
- sleepTime = sleepTime + (long) (sleepTime * random.nextDouble());
+ sleepTime = sleepTime + (long) (sleepTime * RANDOM.get().nextDouble());
}
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java
index c968c33..47d906f 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java
@@ -27,6 +27,7 @@
import java.time.ZoneOffset;
import java.time.format.DateTimeFormatter;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import org.apache.accumulo.core.Constants;
@@ -94,15 +95,18 @@
}
}
- public static final List<ACL> PRIVATE;
- public static final List<ACL> PUBLIC;
+ // Need to use Collections.unmodifiableList() instead of List.of() or List.copyOf(), because
+ // ImmutableCollections.contains() doesn't handle nulls properly (JDK-8265905) and ZooKeeper (as
+ // of 3.8.1) calls acl.contains((Object) null) which throws a NPE when passed an immutable
+ // collection
+ public static final List<ACL> PRIVATE =
+ Collections.unmodifiableList(new ArrayList<>(Ids.CREATOR_ALL_ACL));
+ public static final List<ACL> PUBLIC;
static {
- PRIVATE = new ArrayList<>();
- PRIVATE.addAll(Ids.CREATOR_ALL_ACL);
- PUBLIC = new ArrayList<>();
- PUBLIC.addAll(PRIVATE);
- PUBLIC.add(new ACL(Perms.READ, Ids.ANYONE_ID_UNSAFE));
+ var publicTmp = new ArrayList<>(PRIVATE);
+ publicTmp.add(new ACL(Perms.READ, Ids.ANYONE_ID_UNSAFE));
+ PUBLIC = Collections.unmodifiableList(publicTmp);
}
public static String getRoot(final InstanceId instanceId) {
diff --git a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
index 1620f80..fd8d4cb 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java
@@ -19,12 +19,12 @@
package org.apache.accumulo.core.file;
import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.PrintStream;
-import java.security.SecureRandom;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@@ -50,11 +50,13 @@
import org.apache.accumulo.core.file.rfile.RFile;
import org.apache.accumulo.core.iterators.IteratorEnvironment;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
import org.apache.accumulo.core.util.threads.ThreadPools;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.util.bloom.Key;
import org.apache.hadoop.util.hash.Hash;
@@ -67,7 +69,6 @@
*/
public class BloomFilterLayer {
- private static final SecureRandom random = new SecureRandom();
private static final Logger LOG = LoggerFactory.getLogger(BloomFilterLayer.class);
public static final String BLOOM_FILE_NAME = "acu_bloom";
public static final int HASH_COUNT = 5;
@@ -217,7 +218,7 @@
final String context = ClassLoaderUtil.tableContext(acuconf);
loadTask = () -> {
- // no need to load the bloom filter if the map file is closed
+ // no need to load the bloom filter if the data file is closed
if (closed) {
return;
}
@@ -386,13 +387,13 @@
}
@Override
- public org.apache.accumulo.core.data.Key getFirstKey() throws IOException {
- return reader.getFirstKey();
+ public Text getFirstRow() throws IOException {
+ return reader.getFirstRow();
}
@Override
- public org.apache.accumulo.core.data.Key getLastKey() throws IOException {
- return reader.getLastKey();
+ public Text getLastRow() throws IOException {
+ return reader.getLastRow();
}
@Override
@@ -460,7 +461,7 @@
HashSet<Integer> valsSet = new HashSet<>();
for (int i = 0; i < 100000; i++) {
- valsSet.add(random.nextInt(Integer.MAX_VALUE));
+ valsSet.add(RANDOM.get().nextInt(Integer.MAX_VALUE));
}
ArrayList<Integer> vals = new ArrayList<>(valsSet);
@@ -480,8 +481,8 @@
String suffix = FileOperations.getNewFileExtension(acuconf);
String fname = "/tmp/test." + suffix;
FileSKVWriter bmfw = FileOperations.getInstance().newWriterBuilder()
- .forFile(fname, fs, conf, NoCryptoServiceFactory.NONE).withTableConfiguration(acuconf)
- .build();
+ .forFile(new ReferencedTabletFile(new Path(fname)), fs, conf, NoCryptoServiceFactory.NONE)
+ .withTableConfiguration(acuconf).build();
long t1 = System.currentTimeMillis();
@@ -503,8 +504,8 @@
t1 = System.currentTimeMillis();
FileSKVIterator bmfr = FileOperations.getInstance().newReaderBuilder()
- .forFile(fname, fs, conf, NoCryptoServiceFactory.NONE).withTableConfiguration(acuconf)
- .build();
+ .forFile(new ReferencedTabletFile(new Path(fname)), fs, conf, NoCryptoServiceFactory.NONE)
+ .withTableConfiguration(acuconf).build();
t2 = System.currentTimeMillis();
out.println("Opened " + fname + " in " + (t2 - t1));
@@ -512,7 +513,7 @@
int hits = 0;
for (int i = 0; i < 5000; i++) {
- int row = random.nextInt(Integer.MAX_VALUE);
+ int row = RANDOM.get().nextInt(Integer.MAX_VALUE);
String fi = String.format("%010d", row);
// bmfr.seek(new Range(new Text("r"+fi)));
org.apache.accumulo.core.data.Key k1 =
diff --git a/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java b/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
index fb16b85..f7ba7d2 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/DispatchingFileFactory.java
@@ -20,25 +20,19 @@
import java.io.IOException;
-import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.file.map.MapFileOperations;
import org.apache.accumulo.core.file.rfile.RFile;
import org.apache.accumulo.core.file.rfile.RFileOperations;
+import org.apache.accumulo.core.metadata.TabletFile;
import org.apache.accumulo.core.summary.SummaryWriter;
-import org.apache.hadoop.fs.Path;
class DispatchingFileFactory extends FileOperations {
private FileOperations findFileFactory(FileOptions options) {
- String file = options.getFilename();
+ TabletFile file = options.getFile();
- Path p = new Path(file);
- String name = p.getName();
+ String name = file.getPath().getName();
- if (name.startsWith(Constants.MAPFILE_EXTENSION + "_")) {
- return new MapFileOperations();
- }
String[] sp = name.split("\\.");
if (sp.length < 2) {
@@ -47,10 +41,7 @@
String extension = sp[sp.length - 1];
- if (extension.equals(Constants.MAPFILE_EXTENSION)
- || extension.equals(Constants.MAPFILE_EXTENSION + "_tmp")) {
- return new MapFileOperations();
- } else if (extension.equals(RFile.EXTENSION) || extension.equals(RFile.EXTENSION + "_tmp")) {
+ if (extension.equals(RFile.EXTENSION) || extension.equals(RFile.EXTENSION + "_tmp")) {
return new RFileOperations();
} else {
throw new IllegalArgumentException("File type " + extension + " not supported");
diff --git a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
index db82b0d..5182c61 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java
@@ -31,22 +31,24 @@
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
import org.apache.accumulo.core.file.rfile.RFile;
+import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
import org.apache.accumulo.core.spi.crypto.CryptoService;
import org.apache.accumulo.core.util.ratelimit.RateLimiter;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.FileOutputCommitter;
-import com.google.common.cache.Cache;
+import com.github.benmanes.caffeine.cache.Cache;
public abstract class FileOperations {
private static final String HADOOP_JOBHISTORY_LOCATION = "_logs"; // dir related to
// hadoop.job.history.user.location
- private static final Set<String> validExtensions =
- Set.of(Constants.MAPFILE_EXTENSION, RFile.EXTENSION);
+ private static final Set<String> validExtensions = Set.of(RFile.EXTENSION);
// Sometimes we want to know what files accumulo bulk processing creates
private static final Set<String> bulkWorkingFiles =
@@ -163,10 +165,10 @@
return new ReaderBuilder();
}
- public static class FileOptions {
+ protected static class FileOptions {
// objects used by all
public final AccumuloConfiguration tableConfiguration;
- public final String filename;
+ public final TabletFile file;
public final FileSystem fs;
public final Configuration fsConf;
public final RateLimiter rateLimiter;
@@ -185,13 +187,13 @@
public final boolean inclusive;
public final boolean dropCacheBehind;
- public FileOptions(AccumuloConfiguration tableConfiguration, String filename, FileSystem fs,
+ protected FileOptions(AccumuloConfiguration tableConfiguration, TabletFile file, FileSystem fs,
Configuration fsConf, RateLimiter rateLimiter, String compression,
FSDataOutputStream outputStream, boolean enableAccumuloStart, CacheProvider cacheProvider,
Cache<String,Long> fileLenCache, boolean seekToBeginning, CryptoService cryptoService,
Range range, Set<ByteSequence> columnFamilies, boolean inclusive, boolean dropCacheBehind) {
this.tableConfiguration = tableConfiguration;
- this.filename = filename;
+ this.file = Objects.requireNonNull(file);
this.fs = fs;
this.fsConf = fsConf;
this.rateLimiter = rateLimiter;
@@ -212,8 +214,8 @@
return tableConfiguration;
}
- public String getFilename() {
- return filename;
+ public TabletFile getFile() {
+ return file;
}
public FileSystem getFileSystem() {
@@ -274,7 +276,7 @@
*/
public static class FileHelper {
private AccumuloConfiguration tableConfiguration;
- private String filename;
+ private TabletFile file;
private FileSystem fs;
private Configuration fsConf;
private RateLimiter rateLimiter;
@@ -291,8 +293,8 @@
return this;
}
- protected FileHelper filename(String filename) {
- this.filename = Objects.requireNonNull(filename);
+ protected FileHelper file(TabletFile file) {
+ this.file = Objects.requireNonNull(file);
return this;
}
@@ -318,28 +320,27 @@
protected FileOptions toWriterBuilderOptions(String compression,
FSDataOutputStream outputStream, boolean startEnabled) {
- return new FileOptions(tableConfiguration, filename, fs, fsConf, rateLimiter, compression,
+ return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, compression,
outputStream, startEnabled, NULL_PROVIDER, null, false, cryptoService, null, null, true,
dropCacheBehind);
}
protected FileOptions toReaderBuilderOptions(CacheProvider cacheProvider,
Cache<String,Long> fileLenCache, boolean seekToBeginning) {
- return new FileOptions(tableConfiguration, filename, fs, fsConf, rateLimiter, null, null,
- false, cacheProvider == null ? NULL_PROVIDER : cacheProvider, fileLenCache,
- seekToBeginning, cryptoService, null, null, true, dropCacheBehind);
+ return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, null, null, false,
+ cacheProvider == null ? NULL_PROVIDER : cacheProvider, fileLenCache, seekToBeginning,
+ cryptoService, null, null, true, dropCacheBehind);
}
protected FileOptions toIndexReaderBuilderOptions(Cache<String,Long> fileLenCache) {
- return new FileOptions(tableConfiguration, filename, fs, fsConf, rateLimiter, null, null,
- false, NULL_PROVIDER, fileLenCache, false, cryptoService, null, null, true,
- dropCacheBehind);
+ return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, null, null, false,
+ NULL_PROVIDER, fileLenCache, false, cryptoService, null, null, true, dropCacheBehind);
}
protected FileOptions toScanReaderBuilderOptions(Range range, Set<ByteSequence> columnFamilies,
boolean inclusive) {
- return new FileOptions(tableConfiguration, filename, fs, fsConf, rateLimiter, null, null,
- false, NULL_PROVIDER, null, false, cryptoService, range, columnFamilies, inclusive,
+ return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, null, null, false,
+ NULL_PROVIDER, null, false, cryptoService, range, columnFamilies, inclusive,
dropCacheBehind);
}
@@ -357,15 +358,17 @@
private boolean enableAccumuloStart = true;
public WriterTableConfiguration forOutputStream(String extension,
- FSDataOutputStream outputStream, Configuration fsConf, CryptoService cs) {
+ FSDataOutputStream outputStream, Configuration fsConf, CryptoService cs)
+ throws IOException {
this.outputStream = outputStream;
- filename("foo" + extension).fsConf(fsConf).cryptoService(cs);
+ file(UnreferencedTabletFile.of(fsConf, new Path("foo/foo" + extension))).fsConf(fsConf)
+ .cryptoService(cs);
return this;
}
- public WriterTableConfiguration forFile(String filename, FileSystem fs, Configuration fsConf,
+ public WriterTableConfiguration forFile(TabletFile file, FileSystem fs, Configuration fsConf,
CryptoService cs) {
- filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
+ file(file).fs(fs).fsConf(fsConf).cryptoService(cs);
return this;
}
@@ -412,9 +415,9 @@
private Cache<String,Long> fileLenCache;
private boolean seekToBeginning = false;
- public ReaderTableConfiguration forFile(String filename, FileSystem fs, Configuration fsConf,
+ public ReaderTableConfiguration forFile(TabletFile file, FileSystem fs, Configuration fsConf,
CryptoService cs) {
- filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
+ file(file).fs(fs).fsConf(fsConf).cryptoService(cs);
return this;
}
@@ -480,9 +483,9 @@
private Cache<String,Long> fileLenCache = null;
- public IndexReaderTableConfiguration forFile(String filename, FileSystem fs,
+ public IndexReaderTableConfiguration forFile(TabletFile file, FileSystem fs,
Configuration fsConf, CryptoService cs) {
- filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
+ file(file).fs(fs).fsConf(fsConf).cryptoService(cs);
return this;
}
@@ -512,9 +515,9 @@
private Set<ByteSequence> columnFamilies;
private boolean inclusive;
- public ScanReaderTableConfiguration forFile(String filename, FileSystem fs,
+ public ScanReaderTableConfiguration forFile(TabletFile file, FileSystem fs,
Configuration fsConf, CryptoService cs) {
- filename(filename).fs(fs).fsConf(fsConf).cryptoService(cs);
+ file(file).fs(fs).fsConf(fsConf).cryptoService(cs);
return this;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/FilePrefix.java b/core/src/main/java/org/apache/accumulo/core/file/FilePrefix.java
new file mode 100644
index 0000000..febec86
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/file/FilePrefix.java
@@ -0,0 +1,42 @@
+/*
+ * 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.file;
+
+import java.util.stream.Stream;
+
+public enum FilePrefix {
+
+ BULK_IMPORT("I"), MINOR_COMPACTION("F"), MAJOR_COMPACTION("C"), MAJOR_COMPACTION_ALL_FILES("A");
+
+ String prefix;
+
+ FilePrefix(String prefix) {
+ this.prefix = prefix;
+ }
+
+ public static FilePrefix fromPrefix(String prefix) {
+ return Stream.of(FilePrefix.values()).filter(p -> p.prefix.equals(prefix)).findAny()
+ .orElseThrow(() -> new IllegalArgumentException("Unknown prefix type: " + prefix));
+ }
+
+ public String toPrefix() {
+ return this.prefix;
+ }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/FileSKVIterator.java b/core/src/main/java/org/apache/accumulo/core/file/FileSKVIterator.java
index 23da429..b55f1ba 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/FileSKVIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/FileSKVIterator.java
@@ -21,16 +21,16 @@
import java.io.DataInputStream;
import java.io.IOException;
-import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
import org.apache.accumulo.core.iteratorsImpl.system.InterruptibleIterator;
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
+import org.apache.hadoop.io.Text;
public interface FileSKVIterator extends InterruptibleIterator, AutoCloseable {
- Key getFirstKey() throws IOException;
+ Text getFirstRow() throws IOException;
- Key getLastKey() throws IOException;
+ Text getLastRow() throws IOException;
DataInputStream getMetaStore(String name) throws IOException, NoSuchMetaStoreException;
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/impl/BlockCacheManagerFactory.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/impl/BlockCacheManagerFactory.java
index b72979d..655b7b3 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/impl/BlockCacheManagerFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/impl/BlockCacheManagerFactory.java
@@ -31,14 +31,14 @@
/**
* Get the BlockCacheFactory specified by the property 'tserver.cache.factory.class' using the
- * AccumuloVFSClassLoader
+ * System class loader
*
* @param conf accumulo configuration
* @return block cache manager instance
- * @throws Exception error loading block cache manager implementation class
+ * @throws ReflectiveOperationException error loading block cache manager implementation class
*/
public static synchronized BlockCacheManager getInstance(AccumuloConfiguration conf)
- throws Exception {
+ throws ReflectiveOperationException {
String impl = conf.get(Property.TSERV_CACHE_MANAGER_IMPL);
Class<? extends BlockCacheManager> clazz =
ClassLoaderUtil.loadClass(impl, BlockCacheManager.class);
@@ -51,10 +51,10 @@
*
* @param conf accumulo configuration
* @return block cache manager instance
- * @throws Exception error loading block cache manager implementation class
+ * @throws ReflectiveOperationException error loading block cache manager implementation class
*/
public static synchronized BlockCacheManager getClientInstance(AccumuloConfiguration conf)
- throws Exception {
+ throws ReflectiveOperationException {
String impl = conf.get(Property.TSERV_CACHE_MANAGER_IMPL);
Class<? extends BlockCacheManager> clazz =
Class.forName(impl).asSubclass(BlockCacheManager.class);
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java
index dc85350..7a85da7 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/lru/LruBlockCache.java
@@ -154,7 +154,7 @@
try {
Thread.sleep(10);
} catch (InterruptedException ex) {
- throw new RuntimeException(ex);
+ throw new IllegalStateException(ex);
}
}
} else {
@@ -633,6 +633,11 @@
return accessCount.get();
}
+ @Override
+ public long evictionCount() {
+ return getEvictedCount();
+ }
+
public long getMissCount() {
return missCount.get();
}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java
index 78206c7..7ee9dad 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/tinylfu/TinyLfuBlockCache.java
@@ -115,6 +115,11 @@
public long requestCount() {
return stats.requestCount();
}
+
+ @Override
+ public long evictionCount() {
+ return stats.evictionCount();
+ }
};
}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
index b54b8ab..0beb67f 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java
@@ -26,7 +26,6 @@
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
@@ -48,7 +47,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.cache.Cache;
+import com.github.benmanes.caffeine.cache.Cache;
/**
* This is a wrapper class for BCFile that includes a cache for independent caches for datablocks
@@ -170,8 +169,14 @@
private long getCachedFileLen() throws IOException {
try {
- return fileLenCache.get(cacheId, lengthSupplier::get);
- } catch (ExecutionException e) {
+ return fileLenCache.get(cacheId, k -> {
+ try {
+ return lengthSupplier.get();
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ });
+ } catch (UncheckedIOException e) {
throw new IOException("Failed to get " + cacheId + " len from cache ", e);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/SeekableByteArrayInputStream.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/SeekableByteArrayInputStream.java
index c231e88..9aaa67b 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/SeekableByteArrayInputStream.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/SeekableByteArrayInputStream.java
@@ -22,8 +22,9 @@
import java.io.IOException;
import java.io.InputStream;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiFunction;
+import java.util.function.IntBinaryOperator;
/**
* This class is like byte array input stream with two differences. It supports seeking and avoids
@@ -31,24 +32,16 @@
*/
public class SeekableByteArrayInputStream extends InputStream {
- // making this volatile for the following case
- // * thread 1 creates and initializes byte array
- // * thread 2 reads from bye array
- // spotbugs complains about this because thread2 may not see any changes to the byte array after
- // thread 1 set the volatile,
- // however the expectation is that the byte array is static. In the case of it being static,
- // volatile ensures that
- // thread 2 sees all of thread 1 changes before setting the volatile.
- @SuppressFBWarnings(value = "VO_VOLATILE_REFERENCE_TO_ARRAY",
- justification = "see explanation above")
- private volatile byte[] buffer;
- private int cur;
- private int max;
+ private final byte[] buffer;
+ private final AtomicInteger cur = new AtomicInteger(0);
+ private final int max;
@Override
public int read() {
- if (cur < max) {
- return buffer[cur++] & 0xff;
+ // advance the pointer by 1 if we haven't reached the end
+ final int currentValue = cur.getAndAccumulate(1, (v, x) -> v < max ? v + x : v);
+ if (currentValue < max) {
+ return buffer[currentValue] & 0xff;
} else {
return -1;
}
@@ -68,7 +61,20 @@
return 0;
}
- int avail = max - cur;
+ // compute how much to read, based on what's left available
+ IntBinaryOperator add = (cur1, length1) -> {
+ final int available = max - cur1;
+ if (available <= 0) {
+ return cur1;
+ } else if (length1 > available) {
+ length1 = available;
+ }
+ return cur1 + length1;
+ };
+
+ final int currentValue = cur.getAndAccumulate(length, add);
+
+ final int avail = max - currentValue;
if (avail <= 0) {
return -1;
@@ -78,29 +84,29 @@
length = avail;
}
- System.arraycopy(buffer, cur, b, offset, length);
- cur += length;
+ System.arraycopy(buffer, currentValue, b, offset, length);
return length;
}
@Override
public long skip(long requestedSkip) {
- int actualSkip = max - cur;
- if (requestedSkip < actualSkip) {
- if (requestedSkip < 0) {
- actualSkip = 0;
- } else {
- actualSkip = (int) requestedSkip;
- }
- }
- cur += actualSkip;
- return actualSkip;
+ // actual skip is at least 0, but no more than what's available
+ BiFunction<Integer,Integer,Integer> skipValue =
+ (current, skip) -> Math.max(0, Math.min(max - current, skip));
+
+ // compute how much to advance, based on actual amount skipped
+ IntBinaryOperator add = (cur1, skip) -> cur1 + skipValue.apply(cur1, skip);
+
+ // advance the pointer and return the actual amount skipped
+ int currentValue = cur.getAndAccumulate((int) requestedSkip, add);
+
+ return skipValue.apply(currentValue, (int) requestedSkip);
}
@Override
public int available() {
- return max - cur;
+ return max - cur.get();
}
@Override
@@ -124,14 +130,12 @@
public SeekableByteArrayInputStream(byte[] buf) {
requireNonNull(buf, "bug argument was null");
this.buffer = buf;
- this.cur = 0;
this.max = buf.length;
}
public SeekableByteArrayInputStream(byte[] buf, int maxOffset) {
requireNonNull(buf, "bug argument was null");
this.buffer = buf;
- this.cur = 0;
this.max = maxOffset;
}
@@ -139,11 +143,11 @@
if (position < 0 || position >= max) {
throw new IllegalArgumentException("position = " + position + " maxOffset = " + max);
}
- this.cur = position;
+ this.cur.set(position);
}
public int getPosition() {
- return this.cur;
+ return this.cur.get();
}
byte[] getBuffer() {
diff --git a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java
deleted file mode 100644
index 6f4c2e2..0000000
--- a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java
+++ /dev/null
@@ -1,192 +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.file.map;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.accumulo.core.data.ByteSequence;
-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.dataImpl.KeyExtent;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iteratorsImpl.system.MapFileIterator;
-import org.apache.accumulo.core.iteratorsImpl.system.SequenceFileIterator;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.MapFile;
-
-public class MapFileOperations extends FileOperations {
-
- public static class RangeIterator implements FileSKVIterator {
-
- SortedKeyValueIterator<Key,Value> reader;
- private Range range;
- private boolean hasTop;
-
- public RangeIterator(SortedKeyValueIterator<Key,Value> reader) {
- this.reader = reader;
- }
-
- @Override
- public void close() throws IOException {
- ((FileSKVIterator) reader).close();
- }
-
- @Override
- public Key getFirstKey() throws IOException {
- return ((FileSKVIterator) reader).getFirstKey();
- }
-
- @Override
- public Key getLastKey() throws IOException {
- return ((FileSKVIterator) reader).getLastKey();
- }
-
- @Override
- public DataInputStream getMetaStore(String name) throws IOException {
- return ((FileSKVIterator) reader).getMetaStore(name);
- }
-
- @Override
- public long estimateOverlappingEntries(KeyExtent extent) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
- return new RangeIterator(reader.deepCopy(env));
- }
-
- @Override
- public Key getTopKey() {
- if (!hasTop) {
- throw new IllegalStateException();
- }
- return reader.getTopKey();
- }
-
- @Override
- public Value getTopValue() {
- if (!hasTop) {
- throw new IllegalStateException();
- }
- return reader.getTopValue();
- }
-
- @Override
- public boolean hasTop() {
- return hasTop;
- }
-
- @Override
- public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
- IteratorEnvironment env) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void next() throws IOException {
- if (!hasTop) {
- throw new IllegalStateException();
- }
- reader.next();
- hasTop = reader.hasTop() && !range.afterEndKey(reader.getTopKey());
- }
-
- @Override
- public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
- throws IOException {
- reader.seek(range, columnFamilies, inclusive);
- this.range = range;
-
- hasTop = reader.hasTop() && !range.afterEndKey(reader.getTopKey());
-
- while (hasTop() && range.beforeStartKey(getTopKey())) {
- next();
- }
- }
-
- @Override
- public void closeDeepCopies() throws IOException {
- ((FileSKVIterator) reader).closeDeepCopies();
- }
-
- @Override
- public void setInterruptFlag(AtomicBoolean flag) {
- ((FileSKVIterator) reader).setInterruptFlag(flag);
- }
-
- @Override
- public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
- return ((FileSKVIterator) reader).getSample(sampleConfig);
- }
-
- @Override
- public void setCacheProvider(CacheProvider cacheProvider) {}
- }
-
- @Override
- protected FileSKVIterator openReader(FileOptions options) throws IOException {
- FileSKVIterator iter = new RangeIterator(new MapFileIterator(options.getFileSystem(),
- options.getFilename(), options.getConfiguration()));
- if (options.isSeekToBeginning()) {
- iter.seek(new Range(new Key(), null), new ArrayList<>(), false);
- }
- return iter;
- }
-
- @Override
- protected FileSKVWriter openWriter(FileOptions options) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- protected FileSKVIterator openIndex(FileOptions options) throws IOException {
- return new SequenceFileIterator(MapFileUtil.openIndex(options.getConfiguration(),
- options.getFileSystem(), new Path(options.getFilename())), false);
- }
-
- @Override
- protected long getFileSize(FileOptions options) throws IOException {
- return options.getFileSystem()
- .getFileStatus(new Path(options.getFilename() + "/" + MapFile.DATA_FILE_NAME)).getLen();
- }
-
- @Override
- protected FileSKVIterator openScanReader(FileOptions options) throws IOException {
- MapFileIterator mfIter = new MapFileIterator(options.getFileSystem(), options.getFilename(),
- options.getConfiguration());
-
- FileSKVIterator iter = new RangeIterator(mfIter);
- iter.seek(options.getRange(), options.getColumnFamilies(), options.isRangeInclusive());
-
- return iter;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileUtil.java b/core/src/main/java/org/apache/accumulo/core/file/map/MapFileUtil.java
deleted file mode 100644
index 7fd7276..0000000
--- a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileUtil.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.file.map;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.MapFile;
-import org.apache.hadoop.io.SequenceFile;
-
-public class MapFileUtil {
- public static MapFile.Reader openMapFile(FileSystem fs, String dirName, Configuration conf)
- throws IOException {
- MapFile.Reader mfr = null;
- try {
- mfr = new MapFile.Reader(fs.makeQualified(new Path(dirName)), conf);
- return mfr;
- } catch (IOException e) {
- throw e;
- }
- }
-
- public static SequenceFile.Reader openIndex(Configuration conf, FileSystem fs, Path mapFile)
- throws IOException {
- Path indexPath = new Path(mapFile, MapFile.INDEX_FILE_NAME);
- SequenceFile.Reader index = null;
- try {
- index = new SequenceFile.Reader(conf, SequenceFile.Reader.file(fs.makeQualified(indexPath)));
- return index;
- } catch (IOException e) {
- throw e;
- }
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/CreateEmpty.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/CreateEmpty.java
deleted file mode 100644
index f7d7eb9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/CreateEmpty.java
+++ /dev/null
@@ -1,112 +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.file.rfile;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.accumulo.core.cli.Help;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.file.rfile.bcfile.Compression;
-import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
-import org.apache.accumulo.core.spi.file.rfile.compression.NoCompression;
-import org.apache.accumulo.start.spi.KeywordExecutable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.beust.jcommander.IParameterValidator;
-import com.beust.jcommander.Parameter;
-import com.beust.jcommander.ParameterException;
-import com.google.auto.service.AutoService;
-
-/**
- * Create an empty RFile for use in recovering from data loss where Accumulo still refers internally
- * to a path.
- */
-@AutoService(KeywordExecutable.class)
-public class CreateEmpty implements KeywordExecutable {
- private static final Logger log = LoggerFactory.getLogger(CreateEmpty.class);
-
- public static class NamedLikeRFile implements IParameterValidator {
- @Override
- public void validate(String name, String value) throws ParameterException {
- if (!value.endsWith(".rf")) {
- throw new ParameterException("File must end with .rf and '" + value + "' does not.");
- }
- }
- }
-
- public static class IsSupportedCompressionAlgorithm implements IParameterValidator {
- @Override
- public void validate(String name, String value) throws ParameterException {
- List<String> algorithms = Compression.getSupportedAlgorithms();
- if (!algorithms.contains(value)) {
- throw new ParameterException("Compression codec must be one of " + algorithms);
- }
- }
- }
-
- static class Opts extends Help {
- @Parameter(names = {"-c", "--codec"}, description = "the compression codec to use.",
- validateWith = IsSupportedCompressionAlgorithm.class)
- String codec = new NoCompression().getName();
- @Parameter(
- description = " <path> { <path> ... } Each path given is a URL."
- + " Relative paths are resolved according to the default filesystem defined in"
- + " your Hadoop configuration, which is usually an HDFS instance.",
- required = true, validateWith = NamedLikeRFile.class)
- List<String> files = new ArrayList<>();
- }
-
- public static void main(String[] args) throws Exception {
- new CreateEmpty().execute(args);
- }
-
- @Override
- public String keyword() {
- return "create-empty";
- }
-
- @Override
- public String description() {
- return "Creates an empty rfile";
- }
-
- @Override
- public void execute(String[] args) throws Exception {
- Configuration conf = new Configuration();
-
- Opts opts = new Opts();
- opts.parseArgs("accumulo create-empty", args);
-
- for (String arg : opts.files) {
- Path path = new Path(arg);
- log.info("Writing to file '{}'", path);
- FileSKVWriter writer = new RFileOperations().newWriterBuilder()
- .forFile(arg, path.getFileSystem(conf), conf, NoCryptoServiceFactory.NONE)
- .withTableConfiguration(DefaultConfiguration.getInstance()).withCompression(opts.codec)
- .build();
- writer.close();
- }
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/GenerateSplits.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/GenerateSplits.java
index 865210a..7ee151f 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/GenerateSplits.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/GenerateSplits.java
@@ -47,6 +47,7 @@
import org.apache.accumulo.core.file.FileSKVIterator;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
import org.apache.accumulo.core.spi.crypto.CryptoService;
import org.apache.accumulo.core.util.TextUtil;
@@ -138,18 +139,18 @@
long splitSize = opts.splitSize;
FileSystem fs = FileSystem.get(hadoopConf);
- List<Path> filePaths = new ArrayList<>();
+ List<UnreferencedTabletFile> files = new ArrayList<>();
for (String file : opts.files) {
Path path = new Path(file);
fs = PrintInfo.resolveFS(log, hadoopConf, path);
// get all the files in the directory
- filePaths.addAll(getFiles(fs, path));
+ files.addAll(getFiles(fs, path));
}
- if (filePaths.isEmpty()) {
+ if (files.isEmpty()) {
throw new IllegalArgumentException("No files were found in " + opts.files);
} else {
- log.trace("Found the following files: {}", filePaths);
+ log.trace("Found the following files: {}", files);
}
if (!encode) {
@@ -168,18 +169,17 @@
// if no size specified look at indexed keys first
if (opts.splitSize == 0) {
- splits = getIndexKeys(siteConf, hadoopConf, fs, filePaths, requestedNumSplits, encode,
- cryptoService);
+ splits =
+ getIndexKeys(siteConf, hadoopConf, fs, files, requestedNumSplits, encode, cryptoService);
// if there weren't enough splits indexed, try again with size = 0
if (splits.size() < requestedNumSplits) {
log.info("Only found {} indexed keys but need {}. Doing a full scan on files {}",
- splits.size(), requestedNumSplits, filePaths);
- splits = getSplitsFromFullScan(siteConf, hadoopConf, filePaths, fs, requestedNumSplits,
- encode, cryptoService);
+ splits.size(), requestedNumSplits, files);
+ splits = getSplitsFromFullScan(siteConf, hadoopConf, files, fs, requestedNumSplits, encode,
+ cryptoService);
}
} else {
- splits =
- getSplitsBySize(siteConf, hadoopConf, filePaths, fs, splitSize, encode, cryptoService);
+ splits = getSplitsBySize(siteConf, hadoopConf, files, fs, splitSize, encode, cryptoService);
}
TreeSet<String> desiredSplits;
@@ -205,20 +205,20 @@
}
}
- private List<Path> getFiles(FileSystem fs, Path path) throws IOException {
- List<Path> filePaths = new ArrayList<>();
+ private List<UnreferencedTabletFile> getFiles(FileSystem fs, Path path) throws IOException {
+ List<UnreferencedTabletFile> files = new ArrayList<>();
if (fs.getFileStatus(path).isDirectory()) {
var iter = fs.listFiles(path, true);
while (iter.hasNext()) {
- filePaths.addAll(getFiles(fs, iter.next().getPath()));
+ files.addAll(getFiles(fs, iter.next().getPath()));
}
} else {
if (!path.toString().endsWith(".rf")) {
throw new IllegalArgumentException("Provided file (" + path + ") does not end with '.rf'");
}
- filePaths.add(path);
+ files.add(UnreferencedTabletFile.of(fs, path));
}
- return filePaths;
+ return files;
}
private Text[] getQuantiles(SortedKeyValueIterator<Key,Value> iterator, int numSplits)
@@ -296,16 +296,15 @@
* Scan the files for indexed keys first since it is more efficient than a full file scan.
*/
private TreeSet<String> getIndexKeys(AccumuloConfiguration accumuloConf, Configuration hadoopConf,
- FileSystem fs, List<Path> files, int requestedNumSplits, boolean base64encode,
- CryptoService cs) throws IOException {
+ FileSystem fs, List<UnreferencedTabletFile> files, int requestedNumSplits,
+ boolean base64encode, CryptoService cs) throws IOException {
Text[] splitArray;
List<SortedKeyValueIterator<Key,Value>> readers = new ArrayList<>(files.size());
List<FileSKVIterator> fileReaders = new ArrayList<>(files.size());
try {
- for (Path file : files) {
+ for (UnreferencedTabletFile file : files) {
FileSKVIterator reader = FileOperations.getInstance().newIndexReaderBuilder()
- .forFile(file.toString(), fs, hadoopConf, cs).withTableConfiguration(accumuloConf)
- .build();
+ .forFile(file, fs, hadoopConf, cs).withTableConfiguration(accumuloConf).build();
readers.add(reader);
fileReaders.add(reader);
}
@@ -323,7 +322,7 @@
}
private TreeSet<String> getSplitsFromFullScan(SiteConfiguration accumuloConf,
- Configuration hadoopConf, List<Path> files, FileSystem fs, int numSplits,
+ Configuration hadoopConf, List<UnreferencedTabletFile> files, FileSystem fs, int numSplits,
boolean base64encode, CryptoService cs) throws IOException {
Text[] splitArray;
List<FileSKVIterator> fileReaders = new ArrayList<>(files.size());
@@ -331,9 +330,9 @@
SortedKeyValueIterator<Key,Value> iterator;
try {
- for (Path file : files) {
+ for (UnreferencedTabletFile file : files) {
FileSKVIterator reader = FileOperations.getInstance().newScanReaderBuilder()
- .forFile(file.toString(), fs, hadoopConf, cs).withTableConfiguration(accumuloConf)
+ .forFile(file, fs, hadoopConf, cs).withTableConfiguration(accumuloConf)
.overRange(new Range(), Set.of(), false).build();
readers.add(reader);
fileReaders.add(reader);
@@ -356,7 +355,7 @@
* Get number of splits based on requested size of split.
*/
private TreeSet<String> getSplitsBySize(AccumuloConfiguration accumuloConf,
- Configuration hadoopConf, List<Path> files, FileSystem fs, long splitSize,
+ Configuration hadoopConf, List<UnreferencedTabletFile> files, FileSystem fs, long splitSize,
boolean base64encode, CryptoService cs) throws IOException {
long currentSplitSize = 0;
long totalSize = 0;
@@ -365,9 +364,9 @@
List<SortedKeyValueIterator<Key,Value>> readers = new ArrayList<>(files.size());
SortedKeyValueIterator<Key,Value> iterator;
try {
- for (Path file : files) {
+ for (UnreferencedTabletFile file : files) {
FileSKVIterator reader = FileOperations.getInstance().newScanReaderBuilder()
- .forFile(file.toString(), fs, hadoopConf, cs).withTableConfiguration(accumuloConf)
+ .forFile(file, fs, hadoopConf, cs).withTableConfiguration(accumuloConf)
.overRange(new Range(), Set.of(), false).build();
readers.add(reader);
fileReaders.add(reader);
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiIndexIterator.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiIndexIterator.java
index d9fc362..4fb3600 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiIndexIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiIndexIterator.java
@@ -38,6 +38,7 @@
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
import org.apache.accumulo.core.iteratorsImpl.system.HeapIterator;
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
+import org.apache.hadoop.io.Text;
class MultiIndexIterator extends HeapIterator implements FileSKVIterator {
@@ -81,12 +82,12 @@
}
@Override
- public Key getFirstKey() throws IOException {
+ public Text getFirstRow() throws IOException {
throw new UnsupportedOperationException();
}
@Override
- public Key getLastKey() throws IOException {
+ public Text getLastRow() throws IOException {
throw new UnsupportedOperationException();
}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java
index 770b698..0760e0f 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/MultiLevelIndex.java
@@ -28,6 +28,7 @@
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.PrintStream;
+import java.io.UncheckedIOException;
import java.util.AbstractList;
import java.util.ArrayList;
import java.util.Collections;
@@ -190,7 +191,7 @@
sbais.seek(indexOffset + offset);
return newValue();
} catch (IOException ioe) {
- throw new RuntimeException(ioe);
+ throw new UncheckedIOException(ioe);
}
}
@@ -399,7 +400,7 @@
offsetsArray = offsets;
newFormat = false;
} else {
- throw new RuntimeException("Unexpected version " + version);
+ throw new IllegalStateException("Unexpected version " + version);
}
}
@@ -705,7 +706,7 @@
try {
return node.getPreviousNode();
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
}
@@ -713,7 +714,7 @@
try {
return node.getNextNode();
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
index 059eac9..00d01aa 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
@@ -26,6 +26,7 @@
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.PrintStream;
+import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@@ -36,6 +37,7 @@
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
+import java.util.Objects;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -55,6 +57,7 @@
import org.apache.accumulo.core.file.FileSKVWriter;
import org.apache.accumulo.core.file.NoSuchMetaStoreException;
import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
import org.apache.accumulo.core.file.rfile.BlockIndex.BlockIndexEntry;
import org.apache.accumulo.core.file.rfile.MultiLevelIndex.IndexEntry;
@@ -72,16 +75,20 @@
import org.apache.accumulo.core.iteratorsImpl.system.LocalityGroupIterator.LocalityGroup;
import org.apache.accumulo.core.iteratorsImpl.system.LocalityGroupIterator.LocalityGroupContext;
import org.apache.accumulo.core.iteratorsImpl.system.LocalityGroupIterator.LocalityGroupSeekCache;
+import org.apache.accumulo.core.metadata.TabletFile;
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
import org.apache.accumulo.core.util.LocalityGroupUtil;
import org.apache.accumulo.core.util.MutableByteSequence;
import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
public class RFile {
@@ -1034,8 +1041,8 @@
if (iiter.hasPrevious()) {
prevKey = new Key(iiter.peekPrevious().getKey()); // initially prevKey is the last key
+ // of the prev block
} else {
- // of the prev block
prevKey = new Key(); // first block in the file, so set prev key to minimal key
}
@@ -1101,16 +1108,16 @@
}
@Override
- public Key getFirstKey() {
- return firstKey;
+ public Text getFirstRow() {
+ return firstKey != null ? firstKey.getRow() : null;
}
@Override
- public Key getLastKey() {
+ public Text getLastRow() {
if (index.size() == 0) {
return null;
}
- return index.getLastKey();
+ return index.getLastKey().getRow();
}
@Override
@@ -1166,7 +1173,7 @@
}
}
- public static class Reader extends HeapIterator implements FileSKVIterator {
+ public static class Reader extends HeapIterator implements RFileSKVIterator {
private final CachableBlockFile.Reader reader;
@@ -1301,7 +1308,7 @@
@Override
public void closeDeepCopies() {
if (deepCopy) {
- throw new RuntimeException("Calling closeDeepCopies on a deep copy is not supported");
+ throw new IllegalStateException("Calling closeDeepCopies on a deep copy is not supported");
}
for (Reader deepCopy : deepCopies) {
@@ -1314,7 +1321,7 @@
@Override
public void close() throws IOException {
if (deepCopy) {
- throw new RuntimeException("Calling close on a deep copy is not supported");
+ throw new IllegalStateException("Calling close on a deep copy is not supported");
}
closeDeepCopies();
@@ -1340,47 +1347,47 @@
}
@Override
- public Key getFirstKey() throws IOException {
+ public Text getFirstRow() throws IOException {
if (currentReaders.length == 0) {
return null;
}
- Key minKey = null;
+ Text minRow = null;
for (LocalityGroupReader currentReader : currentReaders) {
- if (minKey == null) {
- minKey = currentReader.getFirstKey();
+ if (minRow == null) {
+ minRow = currentReader.getFirstRow();
} else {
- Key firstKey = currentReader.getFirstKey();
- if (firstKey != null && firstKey.compareTo(minKey) < 0) {
- minKey = firstKey;
+ Text firstRow = currentReader.getFirstRow();
+ if (firstRow != null && firstRow.compareTo(minRow) < 0) {
+ minRow = firstRow;
}
}
}
- return minKey;
+ return minRow;
}
@Override
- public Key getLastKey() throws IOException {
+ public Text getLastRow() throws IOException {
if (currentReaders.length == 0) {
return null;
}
- Key maxKey = null;
+ Text maxRow = null;
for (LocalityGroupReader currentReader : currentReaders) {
- if (maxKey == null) {
- maxKey = currentReader.getLastKey();
+ if (maxRow == null) {
+ maxRow = currentReader.getLastRow();
} else {
- Key lastKey = currentReader.getLastKey();
- if (lastKey != null && lastKey.compareTo(maxKey) > 0) {
- maxKey = lastKey;
+ Text lastRow = currentReader.getLastRow();
+ if (lastRow != null && lastRow.compareTo(maxRow) > 0) {
+ maxRow = lastRow;
}
}
}
- return maxKey;
+ return maxRow;
}
@Override
@@ -1393,7 +1400,7 @@
}
@Override
- public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+ public Reader deepCopy(IteratorEnvironment env) {
if (env != null && env.isSamplingEnabled()) {
SamplerConfiguration sc = env.getSamplerConfiguration();
if (sc == null) {
@@ -1482,6 +1489,7 @@
return (lgCache == null ? 0 : lgCache.getNumLGSeeked());
}
+ @Override
public FileSKVIterator getIndex() throws IOException {
ArrayList<Iterator<IndexEntry>> indexes = new ArrayList<>();
@@ -1494,7 +1502,7 @@
}
@Override
- public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
+ public Reader getSample(SamplerConfigurationImpl sampleConfig) {
requireNonNull(sampleConfig);
if (this.samplerConfig != null && this.samplerConfig.equals(sampleConfig)) {
@@ -1540,11 +1548,12 @@
@Override
public void setInterruptFlag(AtomicBoolean flag) {
if (deepCopy) {
- throw new RuntimeException("Calling setInterruptFlag on a deep copy is not supported");
+ throw new IllegalStateException("Calling setInterruptFlag on a deep copy is not supported");
}
if (!deepCopies.isEmpty()) {
- throw new RuntimeException("Setting interrupt flag after calling deep copy not supported");
+ throw new IllegalStateException(
+ "Setting interrupt flag after calling deep copy not supported");
}
setInterruptFlagInternal(flag);
@@ -1591,5 +1600,244 @@
return totalEntries;
}
+
+ @Override
+ public void reset() {
+ clear();
+ }
+ }
+
+ public interface RFileSKVIterator extends FileSKVIterator {
+ FileSKVIterator getIndex() throws IOException;
+
+ void reset();
+ }
+
+ static abstract class FencedFileSKVIterator implements FileSKVIterator {
+
+ private final FileSKVIterator reader;
+ protected final Range fence;
+ private final Key fencedStartKey;
+ private final Supplier<Key> fencedEndKey;
+
+ public FencedFileSKVIterator(FileSKVIterator reader, Range fence) {
+ this.reader = Objects.requireNonNull(reader);
+ this.fence = Objects.requireNonNull(fence);
+ this.fencedStartKey = fence.getStartKey();
+ this.fencedEndKey = Suppliers.memoize(() -> getEndKey(fence.getEndKey()));
+ }
+
+ @Override
+ public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
+ IteratorEnvironment env) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean hasTop() {
+ return reader.hasTop();
+ }
+
+ @Override
+ public void next() throws IOException {
+ reader.next();
+ }
+
+ @Override
+ public Key getTopKey() {
+ return reader.getTopKey();
+ }
+
+ @Override
+ public Value getTopValue() {
+ return reader.getTopValue();
+ }
+
+ @Override
+ public Text getFirstRow() throws IOException {
+ var row = reader.getFirstRow();
+ if (row != null && fence.beforeStartKey(new Key(row))) {
+ return fencedStartKey.getRow();
+ } else {
+ return row;
+ }
+ }
+
+ @Override
+ public Text getLastRow() throws IOException {
+ var row = reader.getLastRow();
+ if (row != null && fence.afterEndKey(new Key(row))) {
+ return fencedEndKey.get().getRow();
+ } else {
+ return row;
+ }
+ }
+
+ @Override
+ public boolean isRunningLowOnMemory() {
+ return reader.isRunningLowOnMemory();
+ }
+
+ @Override
+ public void setInterruptFlag(AtomicBoolean flag) {
+ reader.setInterruptFlag(flag);
+ }
+
+ @Override
+ public DataInputStream getMetaStore(String name) throws IOException {
+ return reader.getMetaStore(name);
+ }
+
+ @Override
+ public void closeDeepCopies() throws IOException {
+ reader.closeDeepCopies();
+ }
+
+ @Override
+ public void setCacheProvider(CacheProvider cacheProvider) {
+ reader.setCacheProvider(cacheProvider);
+ }
+
+ @Override
+ public void close() throws IOException {
+ reader.close();
+ }
+
+ private Key getEndKey(Key key) {
+ // If they key is infinite it will be null or if inclusive we can just use it as is
+ // as it would be the correct value for getLastKey()
+ if (fence.isInfiniteStopKey() || fence.isEndKeyInclusive()) {
+ return key;
+ }
+
+ // If exclusive we need to strip the last byte to get the last key that is part of the
+ // actual range to return
+ final byte[] ba = key.getRow().getBytes();
+ Preconditions.checkArgument(ba.length > 0 && ba[ba.length - 1] == (byte) 0x00);
+ byte[] fba = new byte[ba.length - 1];
+ System.arraycopy(ba, 0, fba, 0, ba.length - 1);
+
+ return new Key(fba);
+ }
+
+ }
+
+ static class FencedIndex extends FencedFileSKVIterator {
+ private final FileSKVIterator source;
+
+ public FencedIndex(FileSKVIterator source, Range seekFence) {
+ super(source, seekFence);
+ this.source = source;
+ }
+
+ @Override
+ public boolean hasTop() {
+ // this code filters out data because the rfile index iterators do not support seek
+
+ // If startKey is set then discard everything until we reach the start
+ // of the range
+ if (fence.getStartKey() != null) {
+
+ while (source.hasTop() && fence.beforeStartKey(source.getTopKey())) {
+ try {
+ source.next();
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+ }
+
+ // If endKey is set then ensure that the current key is not passed the end of the range
+ return source.hasTop() && !fence.afterEndKey(source.getTopKey());
+ }
+
+ @Override
+ public long estimateOverlappingEntries(KeyExtent extent) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
+ throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ static class FencedReader extends FencedFileSKVIterator implements RFileSKVIterator {
+
+ private final Reader reader;
+
+ public FencedReader(Reader reader, Range seekFence) {
+ super(reader, seekFence);
+ this.reader = reader;
+ }
+
+ @Override
+ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
+ throws IOException {
+ reader.reset();
+
+ if (fence != null) {
+ range = fence.clip(range, true);
+ if (range == null) {
+ return;
+ }
+ }
+
+ reader.seek(range, columnFamilies, inclusive);
+ }
+
+ @Override
+ public FencedReader deepCopy(IteratorEnvironment env) {
+ return new FencedReader(reader.deepCopy(env), fence);
+ }
+
+ @Override
+ public FileSKVIterator getIndex() throws IOException {
+ return new FencedIndex(reader.getIndex(), fence);
+ }
+
+ @Override
+ public long estimateOverlappingEntries(KeyExtent c) throws IOException {
+ KeyExtent overlapping = c.clip(fence, true);
+ if (overlapping == null) {
+ return 0;
+ }
+ return reader.estimateOverlappingEntries(overlapping);
+ }
+
+ @Override
+ public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
+ final Reader sample = reader.getSample(sampleConfig);
+ return sample != null ? new FencedReader(sample, fence) : null;
+ }
+
+ @Override
+ public void reset() {
+ reader.reset();
+ }
+ }
+
+ public static RFileSKVIterator getReader(final CachableBuilder cb, final TabletFile dataFile)
+ throws IOException {
+ final RFile.Reader reader = new RFile.Reader(Objects.requireNonNull(cb));
+ return dataFile.hasRange() ? new FencedReader(reader, dataFile.getRange()) : reader;
+ }
+
+ public static RFileSKVIterator getReader(final CachableBuilder cb, Range range)
+ throws IOException {
+ final RFile.Reader reader = new RFile.Reader(Objects.requireNonNull(cb));
+ return !range.isInfiniteStartKey() || !range.isInfiniteStopKey()
+ ? new FencedReader(reader, range) : reader;
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
index 609c195..cf3e9f6 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
@@ -33,14 +33,15 @@
import org.apache.accumulo.core.file.FileSKVIterator;
import org.apache.accumulo.core.file.FileSKVWriter;
import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
+import org.apache.accumulo.core.file.rfile.RFile.RFileSKVIterator;
import org.apache.accumulo.core.file.rfile.bcfile.BCFile;
+import org.apache.accumulo.core.metadata.TabletFile;
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
import org.apache.accumulo.core.sample.impl.SamplerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -53,18 +54,18 @@
private static final Collection<ByteSequence> EMPTY_CF_SET = Collections.emptySet();
- private static RFile.Reader getReader(FileOptions options) throws IOException {
+ private static RFileSKVIterator getReader(FileOptions options) throws IOException {
CachableBuilder cb = new CachableBuilder()
- .fsPath(options.getFileSystem(), new Path(options.getFilename()), options.dropCacheBehind)
+ .fsPath(options.getFileSystem(), options.getFile().getPath(), options.dropCacheBehind)
.conf(options.getConfiguration()).fileLen(options.getFileLenCache())
.cacheProvider(options.cacheProvider).readLimiter(options.getRateLimiter())
.cryptoService(options.getCryptoService());
- return new RFile.Reader(cb);
+ return RFile.getReader(cb, options.getFile());
}
@Override
protected long getFileSize(FileOptions options) throws IOException {
- return options.getFileSystem().getFileStatus(new Path(options.getFilename())).getLen();
+ return options.getFileSystem().getFileStatus(options.getFile().getPath()).getLen();
}
@Override
@@ -74,7 +75,7 @@
@Override
protected FileSKVIterator openReader(FileOptions options) throws IOException {
- RFile.Reader reader = getReader(options);
+ FileSKVIterator reader = getReader(options);
if (options.isSeekToBeginning()) {
reader.seek(new Range((Key) null, null), EMPTY_CF_SET, false);
@@ -85,7 +86,7 @@
@Override
protected FileSKVIterator openScanReader(FileOptions options) throws IOException {
- RFile.Reader reader = getReader(options);
+ FileSKVIterator reader = getReader(options);
reader.seek(options.getRange(), options.getColumnFamilies(), options.isRangeInclusive());
return reader;
}
@@ -133,25 +134,25 @@
}
int bufferSize = conf.getInt("io.file.buffer.size", 4096);
- String file = options.getFilename();
+ TabletFile file = options.getFile();
FileSystem fs = options.getFileSystem();
if (options.dropCacheBehind) {
EnumSet<CreateFlag> set = EnumSet.of(CreateFlag.SYNC_BLOCK, CreateFlag.CREATE);
- outputStream = fs.create(new Path(file), FsPermission.getDefault(), set, bufferSize,
+ outputStream = fs.create(file.getPath(), FsPermission.getDefault(), set, bufferSize,
(short) rep, block, null);
try {
// Tell the DataNode that the file does not need to be cached in the OS page cache
outputStream.setDropBehind(Boolean.TRUE);
- LOG.trace("Called setDropBehind(TRUE) for stream writing file {}", options.filename);
+ LOG.trace("Called setDropBehind(TRUE) for stream writing file {}", options.file);
} catch (UnsupportedOperationException e) {
- LOG.debug("setDropBehind not enabled for file: {}", options.filename);
+ LOG.debug("setDropBehind not enabled for file: {}", options.file);
} catch (IOException e) {
- LOG.debug("IOException setting drop behind for file: {}, msg: {}", options.filename,
+ LOG.debug("IOException setting drop behind for file: {}, msg: {}", options.file,
e.getMessage());
}
} else {
- outputStream = fs.create(new Path(file), false, bufferSize, (short) rep, block);
+ outputStream = fs.create(file.getPath(), false, bufferSize, (short) rep, block);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompressionAlgorithm.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompressionAlgorithm.java
index a82dd8a..f76983c 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompressionAlgorithm.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompressionAlgorithm.java
@@ -25,7 +25,6 @@
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Map.Entry;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.accumulo.core.spi.file.rfile.compression.CompressionAlgorithmConfiguration;
@@ -41,9 +40,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
import com.google.common.collect.Maps;
/**
@@ -113,12 +111,8 @@
* Guava cache to have a limited factory pattern defined in the Algorithm enum.
*/
private static LoadingCache<Entry<CompressionAlgorithm,Integer>,CompressionCodec> codecCache =
- CacheBuilder.newBuilder().maximumSize(25).build(new CacheLoader<>() {
- @Override
- public CompressionCodec load(Entry<CompressionAlgorithm,Integer> key) {
- return key.getKey().createNewCodec(key.getValue());
- }
- });
+ Caffeine.newBuilder().maximumSize(25)
+ .build(key -> key.getKey().createNewCodec(key.getValue()));
// Data input buffer size to absorb small reads from application.
protected static final int DATA_IBUF_SIZE = 1024;
@@ -170,11 +164,7 @@
// If the default buffer size is not being used, pull from the loading cache.
if (bufferSize != defaultBufferSize) {
Entry<CompressionAlgorithm,Integer> sizeOpt = Maps.immutableEntry(algorithm, bufferSize);
- try {
- codec = codecCache.get(sizeOpt);
- } catch (ExecutionException e) {
- throw new IOException(e);
- }
+ codec = codecCache.get(sizeOpt);
}
CompressionInputStream cis = codec.createInputStream(stream, decompressor);
return new BufferedInputStream(cis, DATA_IBUF_SIZE);
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintBCInfo.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintBCInfo.java
index fd8356a..55d4338 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintBCInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintBCInfo.java
@@ -47,8 +47,7 @@
CryptoService cryptoService = NoCryptoServiceFactory.NONE;
public void printMetaBlockInfo() throws IOException {
- FSDataInputStream fsin = fs.open(path);
- try (BCFile.Reader bcfr =
+ try (FSDataInputStream fsin = fs.open(path); BCFile.Reader bcfr =
new BCFile.Reader(fsin, fs.getFileStatus(path).getLen(), conf, cryptoService)) {
Set<Entry<String,MetaIndexEntry>> es = bcfr.metaIndex.index.entrySet();
@@ -67,6 +66,17 @@
}
}
+ public String getCompressionType() throws IOException {
+ try (FSDataInputStream fsin = fs.open(path); BCFile.Reader bcfr =
+ new BCFile.Reader(fsin, fs.getFileStatus(path).getLen(), conf, cryptoService)) {
+
+ Set<Entry<String,MetaIndexEntry>> es = bcfr.metaIndex.index.entrySet();
+
+ return es.stream().filter(entry -> entry.getKey().equals("RFile.index")).findFirst()
+ .map(entry -> entry.getValue().getCompressionAlgorithm().getName()).orElse(null);
+ }
+ }
+
static class Opts extends ConfigOpts {
@Parameter(description = " <file>")
String file;
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java
index d3fbfac..3f96630 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/Utils.java
@@ -142,7 +142,7 @@
out.writeLong(n);
return;
default:
- throw new RuntimeException("Internal error");
+ throw new IllegalStateException("Internal error");
}
}
@@ -157,7 +157,7 @@
public static int readVInt(DataInput in) throws IOException {
long ret = readVLong(in);
if ((ret > Integer.MAX_VALUE) || (ret < Integer.MIN_VALUE)) {
- throw new RuntimeException("Number too large to be represented as Integer");
+ throw new IllegalStateException("Number too large to be represented as Integer");
}
return (int) ret;
}
@@ -218,7 +218,7 @@
throw new IOException("Corrupted VLong encoding");
}
default:
- throw new RuntimeException("Internal error");
+ throw new IllegalStateException("Internal error");
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/gc/Reference.java b/core/src/main/java/org/apache/accumulo/core/gc/Reference.java
index 4c67bfd..a6ee689 100644
--- a/core/src/main/java/org/apache/accumulo/core/gc/Reference.java
+++ b/core/src/main/java/org/apache/accumulo/core/gc/Reference.java
@@ -42,13 +42,14 @@
TableId getTableId();
/**
- * Get the exact string stored in the metadata table for this file or directory. A file will be
- * read from the Tablet "file" column family:
+ * Get the path stored in the metadata table for this file or directory. The path will be read
+ * from the Tablet "file" column family:
* {@link org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily}
* A directory will be read from the "srv:dir" column family:
* {@link org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily}
* A scan will be read from the Tablet "scan" column family:
* {@link org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily}
*/
- String getMetadataEntry();
+ String getMetadataPath();
+
}
diff --git a/core/src/main/java/org/apache/accumulo/core/gc/ReferenceDirectory.java b/core/src/main/java/org/apache/accumulo/core/gc/ReferenceDirectory.java
index 5491020..a3bff2b 100644
--- a/core/src/main/java/org/apache/accumulo/core/gc/ReferenceDirectory.java
+++ b/core/src/main/java/org/apache/accumulo/core/gc/ReferenceDirectory.java
@@ -46,11 +46,11 @@
* A Tablet directory should have a metadata entry equal to the dirName.
*/
@Override
- public String getMetadataEntry() {
- if (!tabletDir.equals(metadataEntry)) {
+ public String getMetadataPath() {
+ if (!tabletDir.equals(metadataPath)) {
throw new IllegalStateException(
- "Tablet dir " + tabletDir + " is not equal to metadataEntry: " + metadataEntry);
+ "Tablet dir " + tabletDir + " is not equal to metadataPath: " + metadataPath);
}
- return metadataEntry;
+ return metadataPath;
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/gc/ReferenceFile.java b/core/src/main/java/org/apache/accumulo/core/gc/ReferenceFile.java
index b9eece9..3044ec2 100644
--- a/core/src/main/java/org/apache/accumulo/core/gc/ReferenceFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/gc/ReferenceFile.java
@@ -21,6 +21,9 @@
import java.util.Objects;
import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.hadoop.fs.Path;
/**
* A GC reference used for streaming and delete markers. This type is a file. Subclass is a
@@ -31,21 +34,33 @@
public final TableId tableId; // 2a
public final boolean isScan;
- // the exact string that is stored in the metadata
- protected final String metadataEntry;
+ // the exact path from the file reference string that is stored in the metadata
+ protected final String metadataPath;
- protected ReferenceFile(TableId tableId, String metadataEntry, boolean isScan) {
+ protected ReferenceFile(TableId tableId, String metadataPath, boolean isScan) {
this.tableId = Objects.requireNonNull(tableId);
- this.metadataEntry = Objects.requireNonNull(metadataEntry);
+ this.metadataPath = Objects.requireNonNull(metadataPath);
this.isScan = isScan;
}
- public static ReferenceFile forFile(TableId tableId, String metadataEntry) {
- return new ReferenceFile(tableId, metadataEntry, false);
+ public static ReferenceFile forFile(TableId tableId, StoredTabletFile tabletFile) {
+ return new ReferenceFile(tableId, tabletFile.getMetadataPath(), false);
}
- public static ReferenceFile forScan(TableId tableId, String metadataEntry) {
- return new ReferenceFile(tableId, metadataEntry, true);
+ public static ReferenceFile forFile(TableId tableId, Path metadataPathPath) {
+ return new ReferenceFile(tableId, metadataPathPath.toString(), false);
+ }
+
+ public static ReferenceFile forScan(TableId tableId, ScanServerRefTabletFile tabletFile) {
+ return new ReferenceFile(tableId, tabletFile.getNormalizedPathStr(), true);
+ }
+
+ public static ReferenceFile forScan(TableId tableId, StoredTabletFile tabletFile) {
+ return new ReferenceFile(tableId, tabletFile.getMetadataPath(), true);
+ }
+
+ public static ReferenceFile forScan(TableId tableId, Path metadataPathPath) {
+ return new ReferenceFile(tableId, metadataPathPath.toString(), true);
}
@Override
@@ -64,8 +79,8 @@
}
@Override
- public String getMetadataEntry() {
- return metadataEntry;
+ public String getMetadataPath() {
+ return metadataPath;
}
@Override
@@ -73,7 +88,7 @@
if (equals(that)) {
return 0;
} else {
- return this.metadataEntry.compareTo(that.metadataEntry);
+ return this.metadataPath.compareTo(that.metadataPath);
}
}
@@ -89,17 +104,17 @@
return false;
}
ReferenceFile other = (ReferenceFile) obj;
- return metadataEntry.equals(other.metadataEntry);
+ return metadataPath.equals(other.metadataPath);
}
@Override
public int hashCode() {
- return this.metadataEntry.hashCode();
+ return this.metadataPath.hashCode();
}
@Override
public String toString() {
- return "Reference [id=" + tableId + ", ref=" + metadataEntry + "]";
+ return "Reference [id=" + tableId + ", ref=" + metadataPath + "]";
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java
index 40922bd..b87f309 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/Combiner.java
@@ -27,7 +27,6 @@
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
-import java.util.concurrent.ExecutionException;
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.IteratorSetting.Column;
@@ -43,10 +42,10 @@
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.annotations.VisibleForTesting;
import com.google.common.base.Splitter;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
import com.google.common.collect.Lists;
/**
@@ -187,12 +186,11 @@
@VisibleForTesting
static final Cache<String,Boolean> loggedMsgCache =
- CacheBuilder.newBuilder().expireAfterWrite(1, HOURS).maximumSize(10000).build();
+ Caffeine.newBuilder().expireAfterWrite(1, HOURS).maximumSize(10000).build();
private void sawDelete() {
- if (isMajorCompaction && !reduceOnFullCompactionOnly) {
- try {
- loggedMsgCache.get(this.getClass().getName(), () -> {
+ if (isMajorCompaction && !reduceOnFullCompactionOnly
+ && loggedMsgCache.get(this.getClass().getName(), k -> {
sawDeleteLog.error(
"Combiner of type {} saw a delete during a"
+ " partial compaction. This could cause undesired results. See"
@@ -200,10 +198,11 @@
Combiner.this.getClass().getSimpleName());
// the value is not used and does not matter
return Boolean.TRUE;
- });
- } catch (ExecutionException e) {
- throw new RuntimeException(e);
- }
+ })) {
+ // do nothing;
+ // this is a workaround to ignore the return value of the cache, since we're relying only on
+ // the side-effect of logging when the cache entry expires;
+ // if the cached value is present, it's value is always true
}
}
@@ -316,8 +315,8 @@
Combiner newInstance;
try {
newInstance = this.getClass().getDeclaredConstructor().newInstance();
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (ReflectiveOperationException e) {
+ throw new IllegalStateException(e);
}
newInstance.setSource(getSource().deepCopy(env));
newInstance.combiners = combiners;
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java b/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
index 9c12dcc..8492fb8 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/Filter.java
@@ -19,6 +19,7 @@
package org.apache.accumulo.core.iterators;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
@@ -45,8 +46,8 @@
Filter newInstance;
try {
newInstance = this.getClass().getDeclaredConstructor().newInstance();
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (ReflectiveOperationException e) {
+ throw new IllegalStateException(e);
}
newInstance.setSource(getSource().deepCopy(env));
newInstance.negate = negate;
@@ -79,7 +80,7 @@
try {
source.next();
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java
index 372a0e4..0e42acb 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java
@@ -18,40 +18,16 @@
*/
package org.apache.accumulo.core.iterators;
-import java.io.IOException;
-
import org.apache.accumulo.core.client.PluginEnvironment;
import org.apache.accumulo.core.client.SampleNotPresentException;
import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-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.iterators.IteratorUtil.IteratorScope;
import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.spi.common.ServiceEnvironment;
public interface IteratorEnvironment {
/**
- * @deprecated since 2.0.0. This is a legacy method used for internal backwards compatibility.
- */
- @Deprecated(since = "2.0.0")
- default SortedKeyValueIterator<Key,Value> reserveMapFileReader(String mapFileName)
- throws IOException {
- throw new UnsupportedOperationException();
- }
-
- /**
- * @deprecated since 2.0.0. This method was using an unstable non public type. Use
- * {@link #getPluginEnv()}
- */
- @Deprecated(since = "2.0.0")
- default AccumuloConfiguration getConfig() {
- throw new UnsupportedOperationException();
- }
-
- /**
* Return the executed scope of the Iterator. Value will be one of the following:
* {@link IteratorScope#scan}, {@link IteratorScope#minc}, {@link IteratorScope#majc}
*/
@@ -68,14 +44,6 @@
}
/**
- * @deprecated since 2.0.0. This was an experimental feature and was never tested or documented.
- */
- @Deprecated(since = "2.0.0")
- default void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {
- throw new UnsupportedOperationException();
- }
-
- /**
* Return the Scan Authorizations used in this Iterator. Will throw UnsupportedOperationException
* if {@link #getIteratorScope()} != {@link IteratorScope#scan}.
*/
@@ -153,30 +121,13 @@
* obtain a table configuration, use the following methods:
*
* <pre>
- * iterEnv.getServiceEnv().getConfiguration(env.getTableId())
- * </pre>
- *
- * @since 2.0.0
- * @deprecated since 2.1.0. This method was using a non public API type. Use
- * {@link #getPluginEnv()} instead because it has better stability guarantees.
- */
- @Deprecated(since = "2.1.0")
- default ServiceEnvironment getServiceEnv() {
- throw new UnsupportedOperationException();
- }
-
- /**
- * Returns an object containing information about the server where this iterator was run. To
- * obtain a table configuration, use the following methods:
- *
- * <pre>
* iterEnv.getPluginEnv().getConfiguration(env.getTableId())
* </pre>
*
* @since 2.1.0
*/
default PluginEnvironment getPluginEnv() {
- return getServiceEnv();
+ throw new UnsupportedOperationException();
}
/**
@@ -187,4 +138,14 @@
default TableId getTableId() {
throw new UnsupportedOperationException();
}
+
+ /**
+ * Return whether or not the server is running low on memory
+ *
+ * @return true if server is running low on memory
+ * @since 3.0.0
+ */
+ default boolean isRunningLowOnMemory() {
+ throw new UnsupportedOperationException();
+ }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java
index be9bfe0..30efd5f 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java
@@ -149,4 +149,17 @@
* @exception UnsupportedOperationException if not supported.
*/
SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
+
+ /**
+ * Returns true when running in a server process and the GarbageCollectionLogger determines that
+ * the server is running low on memory. This is useful for iterators that aggregate KV pairs or
+ * perform long running operations that create a lot of garbage. Server side iterators can
+ * override this method and return the value of IteratorEnvironment.isRunningLowOnMemory.
+ *
+ * @return true if running in server process and server is running low on memory
+ * @since 3.0.0
+ */
+ default boolean isRunningLowOnMemory() {
+ return false;
+ }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java
index eb84bdc..f7edbb4 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java
@@ -43,6 +43,7 @@
public abstract class WrappingIterator implements SortedKeyValueIterator<Key,Value> {
private SortedKeyValueIterator<Key,Value> source = null;
+ private IteratorEnvironment env = null;
boolean seenSeek = false;
protected void setSource(SortedKeyValueIterator<Key,Value> source) {
@@ -89,7 +90,7 @@
public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
IteratorEnvironment env) throws IOException {
this.setSource(source);
-
+ this.env = env;
}
@Override
@@ -107,4 +108,12 @@
seenSeek = true;
}
+ @Override
+ public boolean isRunningLowOnMemory() {
+ if (env == null) {
+ return SortedKeyValueIterator.super.isRunningLowOnMemory();
+ }
+ return env.isRunningLowOnMemory();
+ }
+
}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowEncodingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowEncodingIterator.java
index 27c269a..79f81d8 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowEncodingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowEncodingIterator.java
@@ -90,8 +90,8 @@
RowEncodingIterator newInstance;
try {
newInstance = this.getClass().getDeclaredConstructor().newInstance();
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (ReflectiveOperationException e) {
+ throw new IllegalStateException(e);
}
newInstance.sourceIter = sourceIter.deepCopy(env);
newInstance.maxBufferSize = maxBufferSize;
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
index 54d8b09..ac93ab1 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
@@ -19,6 +19,7 @@
package org.apache.accumulo.core.iterators.user;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.Collection;
import java.util.Map;
@@ -158,8 +159,10 @@
try {
newInstance = getClass().getDeclaredConstructor().newInstance();
newInstance.init(getSource().deepCopy(env), options, env);
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (ReflectiveOperationException e) {
+ throw new IllegalStateException(e);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
}
newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
return newInstance;
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/SeekingFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/SeekingFilter.java
index d6094a9..d3bcee8 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/SeekingFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/SeekingFilter.java
@@ -152,8 +152,8 @@
SeekingFilter newInstance;
try {
newInstance = this.getClass().getDeclaredConstructor().newInstance();
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (ReflectiveOperationException e) {
+ throw new IllegalStateException(e);
}
newInstance.setSource(getSource().deepCopy(env));
newInstance.negate = negate;
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java
index 4329c6d..492ef6d 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java
@@ -30,6 +30,9 @@
import java.util.Map.Entry;
import java.util.NoSuchElementException;
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.AccessExpression;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
import org.apache.accumulo.core.data.ByteSequence;
@@ -43,10 +46,6 @@
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
import org.apache.accumulo.core.iterators.WrappingIterator;
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.util.BadArgumentException;
import org.apache.accumulo.core.util.Pair;
import org.apache.commons.collections4.map.LRUMap;
import org.apache.hadoop.io.Text;
@@ -103,7 +102,7 @@
protected Collection<ByteSequence> seekColumnFamilies;
protected boolean seekColumnFamiliesInclusive;
- private VisibilityEvaluator ve = null;
+ private AccessEvaluator ve = null;
private LRUMap<ByteSequence,Boolean> visibleCache = null;
private LRUMap<ByteSequence,Boolean> parsedVisibilitiesCache = null;
private long maxBufferSize;
@@ -118,7 +117,7 @@
if (scanning) {
String auths = options.get(AUTH_OPT);
if (auths != null && !auths.isEmpty()) {
- ve = new VisibilityEvaluator(new Authorizations(auths.getBytes(UTF_8)));
+ ve = AccessEvaluator.of(new Authorizations(auths.getBytes(UTF_8)).toAccessAuthorizations());
visibleCache = new LRUMap<>(100);
}
}
@@ -176,8 +175,8 @@
try {
copy = getClass().getDeclaredConstructor().newInstance();
- } catch (Exception e) {
- throw new RuntimeException(e);
+ } catch (ReflectiveOperationException e) {
+ throw new IllegalStateException(e);
}
copy.setSource(getSource().deepCopy(env));
@@ -409,13 +408,12 @@
// Ensure that the visibility (which could have been transformed) parses. Must always do this
// check, even if visibility is not evaluated.
ByteSequence visibility = key.getColumnVisibilityData();
- ColumnVisibility colVis = null;
Boolean parsed = parsedVisibilitiesCache.get(visibility);
if (parsed == null) {
try {
- colVis = new ColumnVisibility(visibility.toArray());
+ AccessExpression.validate(visibility.toArray());
parsedVisibilitiesCache.put(visibility, Boolean.TRUE);
- } catch (BadArgumentException e) {
+ } catch (InvalidAccessExpressionException e) {
log.error("Parse error after transformation : {}", visibility);
parsedVisibilitiesCache.put(visibility, Boolean.FALSE);
if (scanning) {
@@ -441,12 +439,9 @@
visible = visibleCache.get(visibility);
if (visible == null) {
try {
- if (colVis == null) {
- colVis = new ColumnVisibility(visibility.toArray());
- }
- visible = ve.evaluate(colVis);
+ visible = ve.canAccess(visibility.toArray());
visibleCache.put(visibility, visible);
- } catch (VisibilityParseException | BadArgumentException e) {
+ } catch (InvalidAccessExpressionException e) {
log.error("Parse Error", e);
visible = Boolean.FALSE;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java
index c91009b..3bbb77f 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/VisibilityFilter.java
@@ -23,6 +23,9 @@
import java.io.IOException;
import java.util.Map;
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.AccessExpression;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
@@ -32,10 +35,6 @@
import org.apache.accumulo.core.iterators.OptionDescriber;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
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.util.BadArgumentException;
import org.apache.commons.collections4.map.LRUMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -45,7 +44,7 @@
*/
public class VisibilityFilter extends Filter implements OptionDescriber {
- protected VisibilityEvaluator ve;
+ private AccessEvaluator accessEvaluator;
protected Map<ByteSequence,Boolean> cache;
private static final Logger log = LoggerFactory.getLogger(VisibilityFilter.class);
@@ -66,7 +65,8 @@
String auths = options.get(AUTHS);
Authorizations authObj = auths == null || auths.isEmpty() ? new Authorizations()
: new Authorizations(auths.getBytes(UTF_8));
- this.ve = new VisibilityEvaluator(authObj);
+
+ this.accessEvaluator = AccessEvaluator.of(authObj.toAccessAuthorizations());
}
this.cache = new LRUMap<>(1000);
}
@@ -80,10 +80,10 @@
return b;
}
try {
- new ColumnVisibility(testVis.toArray());
+ AccessExpression.validate(testVis.toArray());
cache.put(testVis, true);
return true;
- } catch (BadArgumentException e) {
+ } catch (InvalidAccessExpressionException e) {
cache.put(testVis, false);
return false;
}
@@ -98,10 +98,10 @@
}
try {
- boolean bb = ve.evaluate(new ColumnVisibility(testVis.toArray()));
+ boolean bb = accessEvaluator.canAccess(testVis.toArray());
cache.put(testVis, bb);
return bb;
- } catch (VisibilityParseException | BadArgumentException e) {
+ } catch (InvalidAccessExpressionException e) {
log.error("Parse Error", e);
return false;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java
index 93c098d..7149995 100644
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/IteratorConfigUtil.java
@@ -199,8 +199,9 @@
final boolean useClassLoader = iteratorBuilder.useAccumuloClassLoader;
Map<String,Class<SortedKeyValueIterator<Key,Value>>> classCache = new HashMap<>();
- for (IterInfo iterInfo : iteratorBuilder.iters) {
- try {
+ try {
+ for (IterInfo iterInfo : iteratorBuilder.iters) {
+
Class<SortedKeyValueIterator<Key,Value>> clazz = null;
log.trace("Attempting to load iterator class {}", iterInfo.className);
if (iteratorBuilder.useClassCache) {
@@ -224,12 +225,10 @@
skvi.init(prev, options, iteratorBuilder.iteratorEnvironment);
prev = skvi;
-
- } catch (ReflectiveOperationException e) {
- log.error("Failed to load iterator {}, for table {}, from context {}", iterInfo.className,
- iteratorBuilder.iteratorEnvironment.getTableId(), iteratorBuilder.context, e);
- throw new RuntimeException(e);
}
+ } catch (ReflectiveOperationException e) {
+ log.error(e.toString());
+ throw new IllegalStateException(e);
}
return prev;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/MapFileIterator.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/MapFileIterator.java
deleted file mode 100644
index 3f84a3a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/MapFileIterator.java
+++ /dev/null
@@ -1,128 +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.iteratorsImpl.system;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.accumulo.core.data.ByteSequence;
-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.dataImpl.KeyExtent;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.file.blockfile.impl.CacheProvider;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-
-public class MapFileIterator implements FileSKVIterator {
-
- private static final String MSG = "Map files are not supported";
-
- public MapFileIterator(FileSystem fs, String dir, Configuration conf) {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public void setInterruptFlag(AtomicBoolean flag) {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
- IteratorEnvironment env) {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public boolean hasTop() {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public void next() {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public Key getTopKey() {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public Value getTopValue() {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public Key getFirstKey() {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public Key getLastKey() {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public DataInputStream getMetaStore(String name) {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public long estimateOverlappingEntries(KeyExtent extent) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void closeDeepCopies() {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public void close() {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
- throw new UnsupportedOperationException(MSG);
- }
-
- @Override
- public void setCacheProvider(CacheProvider cacheProvider) {
- throw new UnsupportedOperationException(MSG);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SequenceFileIterator.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SequenceFileIterator.java
index 474255a..71cbb76 100644
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SequenceFileIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SequenceFileIterator.java
@@ -36,6 +36,7 @@
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.Text;
public class SequenceFileIterator implements FileSKVIterator {
@@ -117,12 +118,12 @@
}
@Override
- public Key getFirstKey() throws IOException {
+ public Text getFirstRow() throws IOException {
throw new UnsupportedOperationException("getFirstKey() not supported");
}
@Override
- public Key getLastKey() throws IOException {
+ public Text getLastRow() throws IOException {
throw new UnsupportedOperationException("getLastKey() not supported");
}
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java
index bac1d31..cfad4ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java
@@ -32,10 +32,6 @@
import org.apache.accumulo.core.security.Authorizations;
import org.apache.accumulo.core.tabletserver.thrift.IteratorConfig;
import org.apache.accumulo.core.tabletserver.thrift.TIteratorSetting;
-import org.apache.thrift.TDeserializer;
-import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
-import org.apache.thrift.protocol.TBinaryProtocol;
/**
* System utility class. Not for client use.
@@ -62,39 +58,6 @@
return new IteratorConfig(tisList);
}
- public static List<IteratorSetting> toIteratorSettings(IteratorConfig ic) {
- List<IteratorSetting> ret = new ArrayList<>();
- for (TIteratorSetting tIteratorSetting : ic.getIterators()) {
- ret.add(toIteratorSetting(tIteratorSetting));
- }
-
- return ret;
- }
-
- public static byte[] encodeIteratorSettings(IteratorConfig iterators) {
- try {
- TSerializer tser = new TSerializer(new TBinaryProtocol.Factory());
- return tser.serialize(iterators);
- } catch (TException e) {
- throw new RuntimeException(e);
- }
- }
-
- public static byte[] encodeIteratorSettings(List<IteratorSetting> iterators) {
- return encodeIteratorSettings(toIteratorConfig(iterators));
- }
-
- public static List<IteratorSetting> decodeIteratorSettings(byte[] enc) {
- IteratorConfig ic = new IteratorConfig();
- try {
- TDeserializer tdser = new TDeserializer(new TBinaryProtocol.Factory());
- tdser.deserialize(ic, enc);
- } catch (TException e) {
- throw new RuntimeException(e);
- }
- return toIteratorSettings(ic);
- }
-
public static SortedKeyValueIterator<Key,Value> setupSystemScanIterators(
SortedKeyValueIterator<Key,Value> source, Set<Column> cols, Authorizations auths,
byte[] defaultVisibility, AccumuloConfiguration conf) throws IOException {
diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/VisibilityFilter.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/VisibilityFilter.java
index 0a0fd5f..ddb60ca 100644
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/VisibilityFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/VisibilityFilter.java
@@ -18,6 +18,8 @@
*/
package org.apache.accumulo.core.iteratorsImpl.system;
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
import org.apache.accumulo.core.data.ArrayByteSequence;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
@@ -26,10 +28,6 @@
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
import org.apache.accumulo.core.iterators.SynchronizedServerFilter;
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.util.BadArgumentException;
import org.apache.commons.collections4.map.LRUMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -43,7 +41,7 @@
* class.
*/
public class VisibilityFilter extends SynchronizedServerFilter {
- protected VisibilityEvaluator ve;
+ protected AccessEvaluator ve;
protected ByteSequence defaultVisibility;
protected LRUMap<ByteSequence,Boolean> cache;
protected Authorizations authorizations;
@@ -53,7 +51,7 @@
private VisibilityFilter(SortedKeyValueIterator<Key,Value> iterator,
Authorizations authorizations, byte[] defaultVisibility) {
super(iterator);
- this.ve = new VisibilityEvaluator(authorizations);
+ this.ve = AccessEvaluator.of(authorizations.toAccessAuthorizations());
this.authorizations = authorizations;
this.defaultVisibility = new ArrayByteSequence(defaultVisibility);
this.cache = new LRUMap<>(1000);
@@ -80,14 +78,11 @@
}
try {
- boolean bb = ve.evaluate(new ColumnVisibility(testVis.toArray()));
+ boolean bb = ve.canAccess(testVis.toArray());
cache.put(testVis, bb);
return bb;
- } catch (VisibilityParseException e) {
- log.error("VisibilityParseException with visibility of Key: {}", k, e);
- return false;
- } catch (BadArgumentException e) {
- log.error("BadArgumentException with visibility of Key: {}", k, e);
+ } catch (InvalidAccessExpressionException e) {
+ log.error("IllegalAccessExpressionException with visibility of Key: {}", k, e);
return false;
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLock.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java
similarity index 91%
rename from core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLock.java
rename to core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java
index 63807d0..4338805 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLock.java
+++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java
@@ -16,16 +16,20 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.accumulo.core.fate.zookeeper;
+package org.apache.accumulo.core.lock;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Objects.requireNonNull;
import java.util.ArrayList;
import java.util.List;
+import java.util.Optional;
import java.util.UUID;
+import org.apache.accumulo.core.fate.zookeeper.ZooCache;
import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.LockID;
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
import org.apache.zookeeper.CreateMode;
@@ -114,9 +118,9 @@
zooKeeper.exists(path.toString(), this);
watchingParent = true;
this.vmLockPrefix = new Prefix(ZLOCK_PREFIX + uuid.toString() + "#");
- } catch (Exception ex) {
+ } catch (KeeperException | InterruptedException ex) {
LOG.error("Error setting initial watch", ex);
- throw new RuntimeException(ex);
+ throw new IllegalStateException(ex);
}
}
@@ -151,12 +155,12 @@
}
- public synchronized boolean tryLock(LockWatcher lw, byte[] data)
+ public synchronized boolean tryLock(LockWatcher lw, ServiceLockData lockData)
throws KeeperException, InterruptedException {
LockWatcherWrapper lww = new LockWatcherWrapper(lw);
- lock(lww, data);
+ lock(lww, lockData);
if (lww.acquiredLock) {
return true;
@@ -284,7 +288,7 @@
if (!children.contains(createdEphemeralNode)) {
LOG.error("Expected ephemeral node {} to be in the list of children {}", createdEphemeralNode,
children);
- throw new RuntimeException(
+ throw new IllegalStateException(
"Lock attempt ephemeral node no longer exist " + createdEphemeralNode);
}
@@ -387,7 +391,7 @@
localLw.lostLock(reason);
}
- public synchronized void lock(final AccumuloLockWatcher lw, byte[] data) {
+ public synchronized void lock(final AccumuloLockWatcher lw, ServiceLockData lockData) {
if (lockWatcher != null || lockNodeName != null || createdNodeName != null) {
throw new IllegalStateException();
@@ -401,9 +405,9 @@
// except that instead of the ephemeral lock node being of the form guid-lock- use lock-guid-.
// Another deviation from the recipe is that we cleanup any extraneous ephemeral nodes that
// were created.
- final String createPath =
- zooKeeper.create(lockPathPrefix, data, ZooUtil.PUBLIC, CreateMode.EPHEMERAL_SEQUENTIAL);
- LOG.debug("[{}] Ephemeral node {} created", vmLockPrefix, createPath);
+ final String createPath = zooKeeper.create(lockPathPrefix, lockData.serialize(),
+ ZooUtil.PUBLIC, CreateMode.EPHEMERAL_SEQUENTIAL);
+ LOG.debug("[{}] Ephemeral node {} created with data: {}", vmLockPrefix, createPath, lockData);
// It's possible that the call above was retried several times and multiple ephemeral nodes
// were created but the client missed the response for some reason. Find the ephemeral nodes
@@ -412,7 +416,8 @@
if (!children.contains(createPath.substring(path.toString().length() + 1))) {
LOG.error("Expected ephemeral node {} to be in the list of children {}", createPath,
children);
- throw new RuntimeException("Lock attempt ephemeral node no longer exist " + createPath);
+ throw new IllegalStateException(
+ "Lock attempt ephemeral node no longer exist " + createPath);
}
String lowestSequentialPath = null;
@@ -446,9 +451,6 @@
}
}
}
- if (lowestSequentialPath == null) {
- throw new IllegalStateException("Could not find lowest sequential path under " + path);
- }
final String pathForWatcher = lowestSequentialPath;
// Set a watcher on the lowest sequential node that we created, this handles the case
@@ -598,9 +600,12 @@
return lockNodeName != null;
}
- public synchronized void replaceLockData(byte[] b) throws KeeperException, InterruptedException {
+ public synchronized void replaceLockData(ServiceLockData lockData)
+ throws KeeperException, InterruptedException {
if (getLockPath() != null) {
- zooKeeper.setData(getLockPath(), b, -1);
+ zooKeeper.setData(getLockPath(), lockData.serialize(), -1);
+ LOG.debug("[{}] Lock data replaced at path {} with data: {}", vmLockPrefix, getLockPath(),
+ lockData);
}
}
@@ -652,36 +657,44 @@
return zc.get(lid.path + "/" + lid.node, stat) != null && stat.getEphemeralOwner() == lid.eid;
}
- public static byte[] getLockData(ZooKeeper zk, ServiceLockPath path)
+ public static Optional<ServiceLockData> getLockData(ZooKeeper zk, ServiceLockPath path)
throws KeeperException, InterruptedException {
List<String> children = validateAndSort(path, zk.getChildren(path.toString(), null));
if (children.isEmpty()) {
- return null;
+ return Optional.empty();
}
String lockNode = children.get(0);
- return zk.getData(path + "/" + lockNode, false, null);
+ byte[] data = zk.getData(path + "/" + lockNode, false, null);
+ if (data == null) {
+ data = new byte[0];
+ }
+ return ServiceLockData.parse(data);
}
- public static byte[] getLockData(org.apache.accumulo.core.fate.zookeeper.ZooCache zc,
- ServiceLockPath path, ZcStat stat) {
+ public static Optional<ServiceLockData> getLockData(
+ org.apache.accumulo.core.fate.zookeeper.ZooCache zc, ServiceLockPath path, ZcStat stat) {
List<String> children = validateAndSort(path, zc.getChildren(path.toString()));
if (children.isEmpty()) {
- return null;
+ return Optional.empty();
}
String lockNode = children.get(0);
if (!lockNode.startsWith(ZLOCK_PREFIX)) {
- throw new RuntimeException("Node " + lockNode + " at " + path + " is not a lock node");
+ throw new IllegalStateException("Node " + lockNode + " at " + path + " is not a lock node");
}
- return zc.get(path + "/" + lockNode, stat);
+ byte[] data = zc.get(path + "/" + lockNode, stat);
+ if (data == null) {
+ data = new byte[0];
+ }
+ return ServiceLockData.parse(data);
}
public static long getSessionId(ZooCache zc, ServiceLockPath path) {
@@ -727,7 +740,7 @@
String lockNode = children.get(0);
if (!lockNode.startsWith(ZLOCK_PREFIX)) {
- throw new RuntimeException("Node " + lockNode + " at " + path + " is not a lock node");
+ throw new IllegalStateException("Node " + lockNode + " at " + path + " is not a lock node");
}
String pathToDelete = path + "/" + lockNode;
@@ -748,7 +761,7 @@
String lockNode = children.get(0);
if (!lockNode.startsWith(ZLOCK_PREFIX)) {
- throw new RuntimeException("Node " + lockNode + " at " + path + " is not a lock node");
+ throw new IllegalStateException("Node " + lockNode + " at " + path + " is not a lock node");
}
byte[] data = zk.getData(path + "/" + lockNode);
diff --git a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java
new file mode 100644
index 0000000..c550d64
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLockData.java
@@ -0,0 +1,221 @@
+/*
+ * 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.lock;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Objects.requireNonNull;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.accumulo.core.util.AddressUtil;
+
+import com.google.common.net.HostAndPort;
+
+public class ServiceLockData implements Comparable<ServiceLockData> {
+
+ /**
+ * Thrift Service list
+ */
+ public static enum ThriftService {
+ CLIENT,
+ COORDINATOR,
+ COMPACTOR,
+ FATE,
+ GC,
+ MANAGER,
+ NONE,
+ TABLET_INGEST,
+ TABLET_MANAGEMENT,
+ TABLET_SCAN,
+ TSERV
+ }
+
+ /**
+ * An object that describes a process, the group assigned to that process, the Thrift service and
+ * the address to use to communicate with that service.
+ */
+ public static class ServiceDescriptor {
+
+ /**
+ * The group name that will be used when one is not specified.
+ */
+ public static final String DEFAULT_GROUP_NAME = "default";
+
+ private final UUID uuid;
+ private final ThriftService service;
+ private final String address;
+ private final String group;
+
+ public ServiceDescriptor(UUID uuid, ThriftService service, String address) {
+ this(uuid, service, address, DEFAULT_GROUP_NAME);
+ }
+
+ public ServiceDescriptor(UUID uuid, ThriftService service, String address, String group) {
+ this.uuid = requireNonNull(uuid);
+ this.service = requireNonNull(service);
+ this.address = requireNonNull(address);
+ this.group = requireNonNull(group);
+ }
+
+ public UUID getUUID() {
+ return uuid;
+ }
+
+ public ThriftService getService() {
+ return service;
+ }
+
+ public String getAddress() {
+ return address;
+ }
+
+ public String getGroup() {
+ return group;
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (obj == null) {
+ return false;
+ }
+ if (getClass() != obj.getClass()) {
+ return false;
+ }
+ ServiceDescriptor other = (ServiceDescriptor) obj;
+ return toString().equals(other.toString());
+ }
+
+ @Override
+ public String toString() {
+ return GSON.get().toJson(this);
+ }
+
+ }
+
+ /**
+ * A set of ServiceDescriptor's
+ */
+ public static class ServiceDescriptors {
+ private final Set<ServiceDescriptor> descriptors;
+
+ public ServiceDescriptors() {
+ descriptors = new HashSet<>();
+ }
+
+ public ServiceDescriptors(HashSet<ServiceDescriptor> descriptors) {
+ this.descriptors = descriptors;
+ }
+
+ public void addService(ServiceDescriptor sd) {
+ this.descriptors.add(sd);
+ }
+
+ public Set<ServiceDescriptor> getServices() {
+ return descriptors;
+ }
+ }
+
+ private EnumMap<ThriftService,ServiceDescriptor> services;
+
+ public ServiceLockData(ServiceDescriptors sds) {
+ this.services = new EnumMap<>(ThriftService.class);
+ sds.getServices().forEach(sd -> this.services.put(sd.getService(), sd));
+ }
+
+ public ServiceLockData(UUID uuid, String address, ThriftService service, String group) {
+ this(new ServiceDescriptors(new HashSet<>(
+ Collections.singleton(new ServiceDescriptor(uuid, service, address, group)))));
+ }
+
+ public ServiceLockData(UUID uuid, String address, ThriftService service) {
+ this(new ServiceDescriptors(
+ new HashSet<>(Collections.singleton(new ServiceDescriptor(uuid, service, address)))));
+ }
+
+ public String getAddressString(ThriftService service) {
+ ServiceDescriptor sd = services.get(service);
+ return sd == null ? null : sd.getAddress();
+ }
+
+ public HostAndPort getAddress(ThriftService service) {
+ String s = getAddressString(service);
+ return s == null ? null : AddressUtil.parseAddress(s);
+ }
+
+ public String getGroup(ThriftService service) {
+ ServiceDescriptor sd = services.get(service);
+ return sd == null ? null : sd.getGroup();
+ }
+
+ public UUID getServerUUID(ThriftService service) {
+ ServiceDescriptor sd = services.get(service);
+ return sd == null ? null : sd.getUUID();
+ }
+
+ public byte[] serialize() {
+ ServiceDescriptors sd = new ServiceDescriptors();
+ services.values().forEach(s -> sd.addService(s));
+ return GSON.get().toJson(sd).getBytes(UTF_8);
+ }
+
+ @Override
+ public String toString() {
+ return new String(serialize(), UTF_8);
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ return o instanceof ServiceLockData ? Objects.equals(toString(), o.toString()) : false;
+ }
+
+ @Override
+ public int compareTo(ServiceLockData other) {
+ return toString().compareTo(other.toString());
+ }
+
+ public static Optional<ServiceLockData> parse(byte[] lockData) {
+ if (lockData == null) {
+ return Optional.empty();
+ }
+ String data = new String(lockData, UTF_8);
+ return data.isBlank() ? Optional.empty()
+ : Optional.of(new ServiceLockData(GSON.get().fromJson(data, ServiceDescriptors.class)));
+ }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java
index ccad01a..fa7ed86 100644
--- a/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java
+++ b/core/src/main/java/org/apache/accumulo/core/logging/FateLogger.java
@@ -21,9 +21,9 @@
import static org.apache.accumulo.core.fate.FateTxId.formatTid;
import java.io.Serializable;
+import java.time.Duration;
import java.util.EnumSet;
import java.util.List;
-import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import org.apache.accumulo.core.fate.Fate;
@@ -62,8 +62,8 @@
}
@Override
- public void unreserve(long tid, long deferTime, TimeUnit deferTimeUnit) {
- store.unreserve(tid, deferTime, deferTimeUnit);
+ public void unreserve(long tid, Duration deferTime) {
+ store.unreserve(tid, deferTime);
}
@Override
diff --git a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
index c91eb9f..2209e41 100644
--- a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
+++ b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
@@ -23,23 +23,26 @@
import java.util.Collection;
import java.util.List;
import java.util.Optional;
-import java.util.concurrent.TimeUnit;
+import java.util.UUID;
import org.apache.accumulo.core.client.admin.CompactionConfig;
import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.CompactableFileImpl;
import org.apache.accumulo.core.metadata.StoredTabletFile;
import org.apache.accumulo.core.metadata.TServerInstance;
import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
import org.apache.accumulo.core.spi.compaction.CompactionJob;
import org.apache.accumulo.core.spi.compaction.CompactionKind;
import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.time.SteadyTime;
import org.apache.commons.io.FileUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.Collections2;
+import com.google.common.net.HostAndPort;
/**
* This class contains source for logs messages about a tablets internal state, like its location,
@@ -72,10 +75,9 @@
locLog.debug("Loading {} on {}", extent, server);
}
- public static void suspended(KeyExtent extent, HostAndPort server, long time, TimeUnit timeUnit,
+ public static void suspended(KeyExtent extent, HostAndPort server, SteadyTime time,
int numWalogs) {
- locLog.debug("Suspended {} to {} at {} ms with {} walogs", extent, server,
- timeUnit.toMillis(time), numWalogs);
+ locLog.debug("Suspended {} to {} at {} ms with {} walogs", extent, server, time, numWalogs);
}
public static void unsuspended(KeyExtent extent) {
@@ -117,32 +119,45 @@
* Lazily converts TableFile to file names. The lazy part is really important because when it is
* not called with log.isDebugEnabled().
*/
- private static Collection<String> asFileNames(Collection<CompactableFile> files) {
- return Collections2.transform(files, CompactableFile::getFileName);
+ private static Collection<String> asMinimalString(Collection<CompactableFile> files) {
+ return Collections2.transform(files,
+ cf -> CompactableFileImpl.toStoredTabletFile(cf).toMinimalString());
}
public static void selected(KeyExtent extent, CompactionKind kind,
- Collection<? extends TabletFile> inputs) {
+ Collection<StoredTabletFile> inputs) {
fileLog.trace("{} changed compaction selection set for {} new set {}", extent, kind,
- Collections2.transform(inputs, TabletFile::getFileName));
+ Collections2.transform(inputs, StoredTabletFile::toMinimalString));
}
public static void compacting(KeyExtent extent, CompactionJob job, CompactionConfig config) {
if (fileLog.isDebugEnabled()) {
if (config == null) {
fileLog.debug("Compacting {} on {} for {} from {} size {}", extent, job.getExecutor(),
- job.getKind(), asFileNames(job.getFiles()), getSize(job.getFiles()));
+ job.getKind(), asMinimalString(job.getFiles()), getSize(job.getFiles()));
} else {
fileLog.debug("Compacting {} on {} for {} from {} size {} config {}", extent,
- job.getExecutor(), job.getKind(), asFileNames(job.getFiles()), getSize(job.getFiles()),
- config);
+ job.getExecutor(), job.getKind(), asMinimalString(job.getFiles()),
+ getSize(job.getFiles()), config);
}
}
}
- public static void compacted(KeyExtent extent, CompactionJob job, TabletFile output) {
+ public static void compacted(KeyExtent extent, CompactionJob job, StoredTabletFile output) {
fileLog.debug("Compacted {} for {} created {} from {}", extent, job.getKind(), output,
- asFileNames(job.getFiles()));
+ asMinimalString(job.getFiles()));
+ }
+
+ public static void compactionFailed(KeyExtent extent, CompactionJob job,
+ CompactionConfig config) {
+ fileLog.debug("Failed to compact: extent: {}, input files: {}, iterators: {}", extent,
+ asMinimalString(job.getFiles()), config.getIterators());
+ }
+
+ public static void externalCompactionFailed(KeyExtent extent, ExternalCompactionId id,
+ CompactionJob job, CompactionConfig config) {
+ fileLog.debug("Failed to compact: id: {}, extent: {}, input files: {}, iterators: {}", id,
+ extent, asMinimalString(job.getFiles()), config.getIterators());
}
public static void flushed(KeyExtent extent, Optional<StoredTabletFile> newDatafile) {
@@ -159,7 +174,7 @@
public static void recovering(KeyExtent extent, List<LogEntry> logEntries) {
if (recoveryLog.isDebugEnabled()) {
- List<String> logIds = logEntries.stream().map(LogEntry::getUniqueID).collect(toList());
+ List<UUID> logIds = logEntries.stream().map(LogEntry::getUniqueID).collect(toList());
recoveryLog.debug("For {} recovering data from walogs: {}", extent, logIds);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java
index ca7b6e7..14ccaa5 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java
@@ -27,7 +27,7 @@
import org.apache.accumulo.core.data.TabletId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.dataImpl.TabletIdImpl;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
import org.apache.accumulo.core.metadata.TServerInstance;
import org.apache.accumulo.core.spi.balancer.TabletBalancer;
import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java
index a0c30d4..a7cc522 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java
@@ -27,7 +27,7 @@
import org.apache.accumulo.core.data.TabletId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.dataImpl.TabletIdImpl;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
import org.apache.accumulo.core.metadata.TServerInstance;
import org.apache.accumulo.core.spi.balancer.TabletBalancer;
import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java
index d1a762d..2d6bf37 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java
@@ -23,7 +23,7 @@
import java.util.HashMap;
import java.util.Map;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java
index 348152c..449b91d 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java
@@ -18,7 +18,7 @@
*/
package org.apache.accumulo.core.manager.balancer;
-import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
public class TableStatisticsImpl implements TableStatistics {
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java
index 9eff55a..2fefb46 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java
@@ -22,7 +22,8 @@
import org.apache.accumulo.core.metadata.TServerInstance;
import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
-import org.apache.accumulo.core.util.HostAndPort;
+
+import com.google.common.net.HostAndPort;
/**
* @since 2.1.0
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java
index 4795489..5d5977b 100644
--- a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java
@@ -46,6 +46,7 @@
}
@Override
+ @Deprecated
public long getSplitCreationTime() {
return thriftStats.getSplitCreationTime();
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java
new file mode 100644
index 0000000..aff3ee2
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java
@@ -0,0 +1,74 @@
+/*
+ * 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.metadata;
+
+import static org.apache.accumulo.core.util.RowRangeUtil.requireKeyExtentDataRange;
+import static org.apache.accumulo.core.util.RowRangeUtil.stripZeroTail;
+
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.Range;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * A base class used to represent file references that are handled by code that processes tablet
+ * files.
+ *
+ * @since 3.0.0
+ */
+public abstract class AbstractTabletFile<T extends AbstractTabletFile<T>>
+ implements TabletFile, Comparable<T> {
+
+ protected final Path path;
+ protected final Range range;
+
+ protected AbstractTabletFile(Path path, Range range) {
+ this.path = Objects.requireNonNull(path);
+ this.range = requireKeyExtentDataRange(range);
+ }
+
+ @Override
+ public Path getPath() {
+ return path;
+ }
+
+ @Override
+ public Range getRange() {
+ return range;
+ }
+
+ @Override
+ public boolean hasRange() {
+ return !range.isInfiniteStartKey() || !range.isInfiniteStopKey();
+ }
+
+ @Override
+ public String toMinimalString() {
+ if (hasRange()) {
+ String startRow = range.isInfiniteStartKey() ? "-inf"
+ : stripZeroTail(range.getStartKey().getRowData()).toString();
+ String endRow = range.isInfiniteStopKey() ? "+inf"
+ : stripZeroTail(range.getEndKey().getRowData()).toString();
+ return getFileName() + " (" + startRow + "," + endRow + "]";
+ } else {
+ return getFileName();
+ }
+ }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/AccumuloTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/AccumuloTable.java
new file mode 100644
index 0000000..568959d
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/AccumuloTable.java
@@ -0,0 +1,57 @@
+/*
+ * 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.metadata;
+
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.clientImpl.Namespace;
+import org.apache.accumulo.core.data.TableId;
+
+/**
+ * Defines the name and id of all tables in the accumulo table namespace.
+ */
+public enum AccumuloTable {
+
+ ROOT("root", "+r"), METADATA("metadata", "!0"), SCAN_REF("scanref", "+scanref");
+
+ private final String name;
+ private final TableId tableId;
+
+ public String tableName() {
+ return name;
+ }
+
+ public TableId tableId() {
+ return tableId;
+ }
+
+ AccumuloTable(String name, String id) {
+ this.name = Namespace.ACCUMULO.name() + "." + name;
+ this.tableId = TableId.of(id);
+ }
+
+ private static final Set<TableId> ALL_IDS =
+ Arrays.stream(values()).map(AccumuloTable::tableId).collect(Collectors.toUnmodifiableSet());
+
+ public static Set<TableId> allTableIds() {
+ return ALL_IDS;
+ }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/CompactableFileImpl.java b/core/src/main/java/org/apache/accumulo/core/metadata/CompactableFileImpl.java
index e8cf109..b25e83d 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/CompactableFileImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/CompactableFileImpl.java
@@ -22,6 +22,7 @@
import java.util.Objects;
import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
+import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.metadata.schema.DataFileValue;
public class CompactableFileImpl implements CompactableFile {
@@ -30,7 +31,12 @@
private final DataFileValue dataFileValue;
public CompactableFileImpl(URI uri, long size, long entries) {
- this.storedTabletFile = new StoredTabletFile(uri.toString());
+ this.storedTabletFile = StoredTabletFile.of(uri);
+ this.dataFileValue = new DataFileValue(size, entries);
+ }
+
+ public CompactableFileImpl(URI uri, Range range, long size, long entries) {
+ this.storedTabletFile = StoredTabletFile.of(uri, range);
this.dataFileValue = new DataFileValue(size, entries);
}
@@ -45,6 +51,11 @@
}
@Override
+ public Range getRange() {
+ return storedTabletFile.getRange();
+ }
+
+ @Override
public String getFileName() {
return storedTabletFile.getFileName();
}
@@ -89,6 +100,6 @@
@Override
public String toString() {
- return "[" + storedTabletFile.getFileName() + ", " + dataFileValue + "]";
+ return "[" + storedTabletFile.toMinimalString() + ", " + dataFileValue + "]";
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
index 9ac2658..80126d1 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
@@ -21,6 +21,7 @@
import static java.util.concurrent.TimeUnit.SECONDS;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@@ -88,8 +89,8 @@
if (log.isTraceEnabled()) {
log.trace("tid={} Looking up in {} row={} extent={} tserver={}",
- Thread.currentThread().getId(), src.tablet_extent.tableId(), TextUtil.truncate(row),
- src.tablet_extent, src.tablet_location);
+ Thread.currentThread().getId(), src.getExtent().tableId(), TextUtil.truncate(row),
+ src.getExtent(), src.getTserverLocation());
timer = new OpTimer().start();
}
@@ -105,8 +106,8 @@
List<IterInfo> serverSideIteratorList = new ArrayList<>();
serverSideIteratorList.add(new IterInfo(10000, WholeRowIterator.class.getName(), "WRI"));
Map<String,Map<String,String>> serverSideIteratorOptions = Collections.emptyMap();
- boolean more = ThriftScanner.getBatchFromServer(context, range, src.tablet_extent,
- src.tablet_location, encodedResults, locCols, serverSideIteratorList,
+ boolean more = ThriftScanner.getBatchFromServer(context, range, src.getExtent(),
+ src.getTserverLocation(), encodedResults, locCols, serverSideIteratorList,
serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, 0L, null);
decodeRows(encodedResults, results);
@@ -115,7 +116,7 @@
range = new Range(results.lastKey().followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME),
true, new Key(stopRow).followingKey(PartialKey.ROW), false);
encodedResults.clear();
- ThriftScanner.getBatchFromServer(context, range, src.tablet_extent, src.tablet_location,
+ ThriftScanner.getBatchFromServer(context, range, src.getExtent(), src.getTserverLocation(),
encodedResults, locCols, serverSideIteratorList, serverSideIteratorOptions,
Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, 0L, null);
@@ -125,7 +126,7 @@
if (timer != null) {
timer.stop();
log.trace("tid={} Got {} results from {} in {}", Thread.currentThread().getId(),
- results.size(), src.tablet_extent, String.format("%.3f secs", timer.scale(SECONDS)));
+ results.size(), src.getExtent(), String.format("%.3f secs", timer.scale(SECONDS)));
}
// if (log.isTraceEnabled()) log.trace("results "+results);
@@ -134,15 +135,15 @@
} catch (AccumuloServerException ase) {
if (log.isTraceEnabled()) {
- log.trace("{} lookup failed, {} server side exception", src.tablet_extent.tableId(),
- src.tablet_location);
+ log.trace("{} lookup failed, {} server side exception", src.getExtent().tableId(),
+ src.getTserverLocation());
}
throw ase;
} catch (AccumuloException e) {
if (log.isTraceEnabled()) {
- log.trace("{} lookup failed", src.tablet_extent.tableId(), e);
+ log.trace("{} lookup failed", src.getExtent().tableId(), e);
}
- parent.invalidateCache(context, src.tablet_location);
+ parent.invalidateCache(context, src.getTserverLocation());
}
return null;
@@ -180,7 +181,7 @@
try {
results.putAll(WholeRowIterator.decodeRow(entry.getKey(), entry.getValue()));
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
}
};
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
index a4f6d2a..fc4e23f 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
@@ -42,9 +42,9 @@
public static MetadataServicer forTableId(ClientContext context, TableId tableId) {
checkArgument(tableId != null, "tableId is null");
- if (RootTable.ID.equals(tableId)) {
+ if (AccumuloTable.ROOT.tableId().equals(tableId)) {
return new ServicerForRootTable(context);
- } else if (MetadataTable.ID.equals(tableId)) {
+ } else if (AccumuloTable.METADATA.tableId().equals(tableId)) {
return new ServicerForMetadataTable(context);
} else {
return new ServicerForUserTables(context, tableId);
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
deleted file mode 100644
index b2a9a5d..0000000
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
+++ /dev/null
@@ -1,27 +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.metadata;
-
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.data.TableId;
-
-public class MetadataTable {
- public static final TableId ID = TableId.of("!0");
- public static final String NAME = Namespace.ACCUMULO.name() + ".metadata";
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ReferencedTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/ReferencedTabletFile.java
new file mode 100644
index 0000000..4e219d8
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ReferencedTabletFile.java
@@ -0,0 +1,238 @@
+/*
+ * 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.metadata;
+
+import static org.apache.accumulo.core.Constants.HDFS_TABLES_DIR;
+
+import java.net.URI;
+import java.util.Comparator;
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Object representing a tablet file that may exist in the metadata table. This class is used for
+ * reading and opening tablet files. It is also used when inserting new tablet files. When a new
+ * file is inserted, the {@link #insert()} method is called and returns a {@link StoredTabletFile}
+ * For situations where a tablet file needs to be updated or deleted in the metadata, a
+ * {@link StoredTabletFile} is required.
+ * <p>
+ * As of 2.1, Tablet file paths should now be only absolute URIs with the removal of relative paths
+ * in Upgrader9to10.upgradeRelativePaths()
+ */
+public class ReferencedTabletFile extends AbstractTabletFile<ReferencedTabletFile> {
+
+ public static class FileParts {
+
+ // parts of an absolute URI, like "hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf"
+ // volume: hdfs://1.2.3.4/accumulo
+ // tableId: 2a
+ // tabletDir: t-0003
+ // fileName: C0004.rf
+ // normalizedPath: hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf
+ private final String volume;
+ private final TableId tableId;
+ private final String tabletDir;
+ private final String fileName;
+ private final String normalizedPath;
+
+ public FileParts(String volume, TableId tableId, String tabletDir, String fileName,
+ String normalizedPath) {
+ this.volume = volume;
+ this.tableId = tableId;
+ this.tabletDir = tabletDir;
+ this.fileName = fileName;
+ this.normalizedPath = normalizedPath;
+ }
+
+ public String getVolume() {
+ return volume;
+ }
+
+ public TableId getTableId() {
+ return tableId;
+ }
+
+ public String getTabletDir() {
+ return tabletDir;
+ }
+
+ public String getFileName() {
+ return fileName;
+ }
+
+ public String getNormalizedPath() {
+ return normalizedPath;
+ }
+
+ }
+
+ private static String constructErrorMsg(Path filePath) {
+ return "Missing or invalid part of tablet file metadata entry: " + filePath;
+ }
+
+ public static FileParts parsePath(Path filePath) {
+ // File name construct: <volume>/<tablePath>/<tableId>/<tablet>/<file>
+ // Example: hdfs://namenode:9020/accumulo/tables/1/default_tablet/F00001.rf
+ final URI uri = filePath.toUri();
+
+ // validate that this is a fully qualified uri
+ Preconditions.checkArgument(uri.getScheme() != null, constructErrorMsg(filePath));
+
+ final String path = uri.getPath(); // ex: /accumulo/tables/1/default_tablet/F00001.rf
+ final String[] parts = path.split("/");
+ final int numParts = parts.length; // should contain tables, 1, default_tablet, F00001.rf
+
+ if (numParts < 4) {
+ throw new IllegalArgumentException(constructErrorMsg(filePath));
+ }
+
+ final String fileName = parts[numParts - 1];
+ final String tabletDirectory = parts[numParts - 2];
+ final TableId tableId = TableId.of(parts[numParts - 3]);
+ final String tablesPath = parts[numParts - 4];
+
+ // determine where file path starts, the rest is the volume
+ final String computedFilePath =
+ HDFS_TABLES_DIR + "/" + tableId.canonical() + "/" + tabletDirectory + "/" + fileName;
+ final String uriString = uri.toString();
+ int idx = uriString.lastIndexOf(computedFilePath);
+
+ if (idx == -1) {
+ throw new IllegalArgumentException(constructErrorMsg(filePath));
+ }
+
+ // The volume is the beginning portion of the uri up to the start
+ // of the file path.
+ final String volume = uriString.substring(0, idx);
+
+ if (StringUtils.isBlank(fileName) || StringUtils.isBlank(tabletDirectory)
+ || StringUtils.isBlank(tablesPath) || StringUtils.isBlank(volume)) {
+ throw new IllegalArgumentException(constructErrorMsg(filePath));
+ }
+ ValidationUtil.validateFileName(fileName);
+ Preconditions.checkArgument(tablesPath.equals(HDFS_TABLES_DIR_NAME),
+ "tables directory name is not " + HDFS_TABLES_DIR_NAME + ", is " + tablesPath);
+
+ final String normalizedPath = volume + computedFilePath;
+
+ if (!normalizedPath.equals(uriString)) {
+ throw new RuntimeException("Error parsing file path, " + normalizedPath + " != " + uriString);
+ }
+
+ return new FileParts(volume, tableId, tabletDirectory, fileName, normalizedPath);
+
+ }
+
+ private final FileParts parts;
+
+ private static final Logger log = LoggerFactory.getLogger(ReferencedTabletFile.class);
+ private static final String HDFS_TABLES_DIR_NAME = HDFS_TABLES_DIR.substring(1);
+
+ private static final Comparator<ReferencedTabletFile> comparator =
+ Comparator.comparing(ReferencedTabletFile::getNormalizedPathStr)
+ .thenComparing(ReferencedTabletFile::getRange);
+
+ public ReferencedTabletFile(Path metaPath) {
+ this(metaPath, new Range());
+ }
+
+ /**
+ * Construct new tablet file using a Path. Used in the case where we had to use Path object to
+ * qualify an absolute path or create a new file.
+ */
+ public ReferencedTabletFile(Path metaPath, Range range) {
+ super(Objects.requireNonNull(metaPath), range);
+ log.trace("Parsing TabletFile from {}", metaPath);
+ parts = parsePath(metaPath);
+ }
+
+ public String getVolume() {
+ return parts.getVolume();
+ }
+
+ public TableId getTableId() {
+ return parts.getTableId();
+ }
+
+ public String getTabletDir() {
+ return parts.getTabletDir();
+ }
+
+ @Override
+ public String getFileName() {
+ return parts.getFileName();
+ }
+
+ /**
+ * Return a string for opening and reading the tablet file. Doesn't have to be exact string in
+ * metadata.
+ */
+ public String getNormalizedPathStr() {
+ return parts.getNormalizedPath();
+ }
+
+ /**
+ * New file was written to metadata so return a StoredTabletFile
+ */
+ public StoredTabletFile insert() {
+ return StoredTabletFile.of(getPath(), getRange());
+ }
+
+ @Override
+ public int compareTo(ReferencedTabletFile o) {
+ return comparator.compare(this, o);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof ReferencedTabletFile) {
+ ReferencedTabletFile that = (ReferencedTabletFile) obj;
+ return parts.getNormalizedPath().equals(that.parts.getNormalizedPath())
+ && range.equals(that.range);
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(parts.getNormalizedPath(), range);
+ }
+
+ @Override
+ public String toString() {
+ return parts.getNormalizedPath();
+ }
+
+ public static ReferencedTabletFile of(final Path path) {
+ return new ReferencedTabletFile(path);
+ }
+
+ public static ReferencedTabletFile of(final Path path, Range range) {
+ return new ReferencedTabletFile(path, range);
+ }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
index 7c00cc0..f7c5b0e 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
@@ -18,17 +18,11 @@
*/
package org.apache.accumulo.core.metadata;
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
public class RootTable {
- public static final TableId ID = TableId.of("+r");
-
- public static final String NAME = Namespace.ACCUMULO.name() + ".root";
-
/**
* DFS location relative to the Accumulo directory
*/
@@ -44,8 +38,8 @@
*/
public static final String ZROOT_TABLET_GC_CANDIDATES = ZROOT_TABLET + "/gc_candidates";
- public static final KeyExtent EXTENT = new KeyExtent(ID, null, null);
- public static final KeyExtent OLD_EXTENT =
- new KeyExtent(MetadataTable.ID, TabletsSection.encodeRow(MetadataTable.ID, null), null);
+ public static final KeyExtent EXTENT = new KeyExtent(AccumuloTable.ROOT.tableId(), null, null);
+ public static final KeyExtent OLD_EXTENT = new KeyExtent(AccumuloTable.METADATA.tableId(),
+ TabletsSection.encodeRow(AccumuloTable.METADATA.tableId(), null), null);
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefStore.java b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefStore.java
new file mode 100644
index 0000000..461427f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefStore.java
@@ -0,0 +1,64 @@
+/*
+ * 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.metadata;
+
+import java.util.Collection;
+import java.util.UUID;
+import java.util.stream.Stream;
+
+public interface ScanServerRefStore {
+
+ /**
+ * Insert ScanServer references to Tablet files
+ *
+ * @param scanRefs set of scan server ref table file objects
+ */
+ default void put(Collection<ScanServerRefTabletFile> scanRefs) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Get ScanServer references to Tablet files
+ *
+ * @return stream of scan server references
+ */
+ default Stream<ScanServerRefTabletFile> list() {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Delete the set of scan server references
+ *
+ * @param refsToDelete set of scan server references to delete
+ */
+ default void delete(Collection<ScanServerRefTabletFile> refsToDelete) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Delete scan server references for this server
+ *
+ * @param serverAddress address of server, cannot be null
+ * @param serverSessionId server session id, cannot be null
+ */
+ default void delete(String serverAddress, UUID serverSessionId) {
+ throw new UnsupportedOperationException();
+ }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
index f3365c7..50076ea 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
@@ -22,101 +22,38 @@
import java.util.Objects;
import java.util.UUID;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ScanServerFileReferenceSection;
-import org.apache.accumulo.core.util.UuidUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
-import com.google.common.base.Preconditions;
+public class ScanServerRefTabletFile extends ReferencedTabletFile {
-public class ScanServerRefTabletFile extends TabletFile {
-
- @SuppressWarnings("deprecation")
- private static final String OLD_PREFIX =
- org.apache.accumulo.core.metadata.schema.MetadataSchema.OldScanServerFileReferenceSection
- .getRowPrefix();
- private final String prefix;
private final Value NULL_VALUE = new Value(new byte[0]);
- private final Text serverAddress;
- private final Text uuid;
+ private final Text colf;
+ private final Text colq;
- public ScanServerRefTabletFile(UUID serverLockUUID, String serverAddress, String file) {
+ public ScanServerRefTabletFile(String file, String serverAddress, UUID serverLockUUID) {
super(new Path(URI.create(file)));
- // For new data, always use the current prefix
- prefix = ScanServerFileReferenceSection.getRowPrefix();
- this.serverAddress = new Text(serverAddress);
- uuid = new Text(serverLockUUID.toString());
+ this.colf = new Text(serverAddress);
+ this.colq = new Text(serverLockUUID.toString());
}
- public ScanServerRefTabletFile(Key k) {
- super(new Path(URI.create(extractFile(k))));
- serverAddress = k.getColumnFamily();
- if (isOldPrefix(k)) {
- prefix = OLD_PREFIX;
- uuid = new Text(k.getColumnQualifier().toString());
- } else {
- prefix = ScanServerFileReferenceSection.getRowPrefix();
- var row = k.getRow().toString();
- Preconditions.checkArgument(row.startsWith(prefix), "Unexpected row prefix %s ", row);
- var uuidStr = row.substring(prefix.length());
- Preconditions.checkArgument(UuidUtil.isUUID(uuidStr, 0), "Row suffix is not uuid %s", row);
- uuid = new Text(uuidStr);
- }
+ public ScanServerRefTabletFile(String file, Text colf, Text colq) {
+ super(new Path(URI.create(file)));
+ this.colf = colf;
+ this.colq = colq;
}
- public Mutation putMutation() {
- // Only write scan refs in the new format
- Mutation mutation = new Mutation(prefix + uuid.toString());
- mutation.put(serverAddress, getFilePath(), getValue());
- return mutation;
+ public String getRow() {
+ return this.getNormalizedPathStr();
}
- public Mutation putDeleteMutation() {
- Mutation mutation;
- if (Objects.equals(prefix, OLD_PREFIX)) {
- mutation = new Mutation(prefix + this.getPath().toString());
- mutation.putDelete(serverAddress, uuid);
- } else {
- mutation = new Mutation(prefix + uuid.toString());
- mutation.putDelete(serverAddress, getFilePath());
- }
- return mutation;
+ public Text getServerAddress() {
+ return this.colf;
}
- private static String extractFile(Key k) {
- if (isOldPrefix(k)) {
- return k.getRow().toString().substring(OLD_PREFIX.length());
- } else {
- return k.getColumnQualifier().toString();
- }
- }
-
- /**
- * Returns the correctly formatted range for a unique uuid
- *
- * @param uuid ServerLockUUID of a Scan Server
- * @return Range for a single scan server
- */
- public static Range getRange(UUID uuid) {
- Objects.requireNonNull(uuid);
- return new Range(MetadataSchema.ScanServerFileReferenceSection.getRowPrefix() + uuid);
- }
-
- private static boolean isOldPrefix(Key k) {
- return k.getRow().toString().startsWith(OLD_PREFIX);
- }
-
- public UUID getServerLockUUID() {
- return UUID.fromString(uuid.toString());
- }
-
- public Text getFilePath() {
- return new Text(this.getPath().toString());
+ public Text getServerLockUUID() {
+ return this.colq;
}
public Value getValue() {
@@ -127,8 +64,8 @@
public int hashCode() {
final int prime = 31;
int result = super.hashCode();
- result = prime * result + ((serverAddress == null) ? 0 : serverAddress.hashCode());
- result = prime * result + ((uuid == null) ? 0 : uuid.hashCode());
+ result = prime * result + ((colf == null) ? 0 : colf.hashCode());
+ result = prime * result + ((colq == null) ? 0 : colq.hashCode());
return result;
}
@@ -144,13 +81,13 @@
return false;
}
ScanServerRefTabletFile other = (ScanServerRefTabletFile) obj;
- return Objects.equals(serverAddress, other.serverAddress) && Objects.equals(uuid, other.uuid);
+ return Objects.equals(colf, other.colf) && Objects.equals(colq, other.colq);
}
@Override
public String toString() {
- return "ScanServerRefTabletFile [file=" + this.getPath().toString() + ", server address="
- + serverAddress + ", server lock uuid=" + uuid + "]";
+ return "ScanServerRefTabletFile [file=" + this.getRow() + ", server address=" + colf
+ + ", server lock uuid=" + colq + "]";
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java
index 948fcc1..a4b4150 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForMetadataTable.java
@@ -27,7 +27,7 @@
class ServicerForMetadataTable extends TableMetadataServicer {
public ServicerForMetadataTable(ClientContext context) {
- super(context, RootTable.NAME, MetadataTable.ID);
+ super(context, AccumuloTable.ROOT.tableName(), AccumuloTable.METADATA.tableId());
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
index aa85fc9..94e8644 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
@@ -38,7 +38,7 @@
@Override
public TableId getServicedTableId() {
- return RootTable.ID;
+ return AccumuloTable.ROOT.tableId();
}
@Override
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
index eeb7f7a..369bcc4 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
@@ -28,7 +28,7 @@
class ServicerForUserTables extends TableMetadataServicer {
public ServicerForUserTables(ClientContext context, TableId tableId) {
- super(context, MetadataTable.NAME, tableId);
+ super(context, AccumuloTable.METADATA.tableName(), tableId);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java
index 24da313..0f66759 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java
@@ -18,10 +18,27 @@
*/
package org.apache.accumulo.core.metadata;
-import java.net.URI;
+import static org.apache.accumulo.core.util.RowRangeUtil.requireKeyExtentDataRange;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.URI;
+import java.util.Comparator;
+import java.util.Objects;
+
+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.util.json.ByteArrayToBase64TypeAdapter;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Text;
+import org.checkerframework.checker.nullness.qual.NonNull;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
/**
* Object representing a tablet file entry stored in the metadata table. Keeps a string of the exact
@@ -34,16 +51,31 @@
* As of 2.1, Tablet file paths should now be only absolute URIs with the removal of relative paths
* in Upgrader9to10.upgradeRelativePaths()
*/
-public class StoredTabletFile extends TabletFile {
+public class StoredTabletFile extends AbstractTabletFile<StoredTabletFile> {
private final String metadataEntry;
+ private final ReferencedTabletFile referencedTabletFile;
+ private final String metadataEntryPath;
+
+ private static final Comparator<StoredTabletFile> comparator = Comparator
+ .comparing(StoredTabletFile::getMetadataPath).thenComparing(StoredTabletFile::getRange);
/**
* Construct a tablet file using the string read from the metadata. Preserve the exact string so
* the entry can be deleted.
*/
public StoredTabletFile(String metadataEntry) {
- super(new Path(URI.create(metadataEntry)));
- this.metadataEntry = metadataEntry;
+ this(metadataEntry, deserialize(metadataEntry));
+ }
+
+ private StoredTabletFile(TabletFileCq fileCq) {
+ this(serialize(fileCq), fileCq);
+ }
+
+ private StoredTabletFile(String metadataEntry, TabletFileCq fileCq) {
+ super(Objects.requireNonNull(fileCq).path, fileCq.range);
+ this.metadataEntry = Objects.requireNonNull(metadataEntry);
+ this.metadataEntryPath = fileCq.path.toString();
+ this.referencedTabletFile = ReferencedTabletFile.of(getPath(), fileCq.range);
}
/**
@@ -51,15 +83,217 @@
* and deleting metadata entries. If the exact string is not used, erroneous entries can pollute
* the metadata table.
*/
- public String getMetaUpdateDelete() {
+ public String getMetadata() {
return metadataEntry;
}
/**
- * Return a new Text object of {@link #getMetaUpdateDelete()}
+ * Returns just the Path portion of the metadata, not the full Json.
*/
- public Text getMetaUpdateDeleteText() {
- return new Text(getMetaUpdateDelete());
+ public String getMetadataPath() {
+ return metadataEntryPath;
}
+ /**
+ * Return a new Text object of {@link #getMetadata()}
+ */
+ public Text getMetadataText() {
+ return new Text(getMetadata());
+ }
+
+ public ReferencedTabletFile getTabletFile() {
+ return referencedTabletFile;
+ }
+
+ public TableId getTableId() {
+ return referencedTabletFile.getTableId();
+ }
+
+ @Override
+ public String getFileName() {
+ return referencedTabletFile.getFileName();
+ }
+
+ public String getNormalizedPathStr() {
+ return referencedTabletFile.getNormalizedPathStr();
+ }
+
+ @Override
+ public int compareTo(StoredTabletFile o) {
+ if (equals(o)) {
+ return 0;
+ } else {
+ return comparator.compare(this, o);
+ }
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ StoredTabletFile that = (StoredTabletFile) o;
+ return Objects.equals(metadataEntry, that.metadataEntry);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(metadataEntry);
+ }
+
+ @Override
+ public String toString() {
+ return metadataEntry;
+ }
+
+ /**
+ * Validates that the provided metadata string for the StoredTabletFile is valid.
+ */
+ public static void validate(String metadataEntry) {
+ final TabletFileCq tabletFileCq = deserialize(metadataEntry);
+ // Validate the path
+ ReferencedTabletFile.parsePath(deserialize(metadataEntry).path);
+ // Validate the range
+ requireKeyExtentDataRange(tabletFileCq.range);
+ }
+
+ public static StoredTabletFile of(final Text metadataEntry) {
+ return new StoredTabletFile(Objects.requireNonNull(metadataEntry).toString());
+ }
+
+ public static StoredTabletFile of(final String metadataEntry) {
+ return new StoredTabletFile(metadataEntry);
+ }
+
+ public static StoredTabletFile of(final URI path, Range range) {
+ return of(new Path(Objects.requireNonNull(path)), range);
+ }
+
+ public static StoredTabletFile of(final Path path, Range range) {
+ return new StoredTabletFile(new TabletFileCq(Objects.requireNonNull(path), range));
+ }
+
+ public static StoredTabletFile of(final URI path) {
+ return of(path, new Range());
+ }
+
+ public static StoredTabletFile of(final Path path) {
+ return of(path, new Range());
+ }
+
+ private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
+
+ private static TabletFileCq deserialize(String json) {
+ final TabletFileCqMetadataGson metadata =
+ gson.fromJson(Objects.requireNonNull(json), TabletFileCqMetadataGson.class);
+
+ // Check each field and provide better error messages if null as all fields should be set
+ Objects.requireNonNull(metadata.path, "Serialized StoredTabletFile path must not be null");
+ Objects.requireNonNull(metadata.startRow,
+ "Serialized StoredTabletFile range startRow must not be null");
+ Objects.requireNonNull(metadata.endRow,
+ "Serialized StoredTabletFile range endRow must not be null");
+
+ // Recreate the exact Range that was originally stored in Metadata. Stored ranges are originally
+ // constructed with inclusive/exclusive for the start and end key inclusivity settings.
+ // (Except for Ranges with no start/endkey as then the inclusivity flags do not matter)
+ // The ranges must match the format of KeyExtent.toDataRange()
+ //
+ // With this particular constructor, when setting the startRowInclusive to true and
+ // endRowInclusive to false, both the start and end row values will be taken as is
+ // and not modified and will recreate the original Range.
+ //
+ // This constructor will always set the resulting inclusivity of the Range to be true for the
+ // start row and false for end row regardless of what the startRowInclusive and endRowInclusive
+ // flags are set to.
+ return new TabletFileCq(new Path(URI.create(metadata.path)),
+ new Range(decodeRow(metadata.startRow), true, decodeRow(metadata.endRow), false));
+ }
+
+ public static String serialize(String path) {
+ return serialize(path, new Range());
+ }
+
+ public static String serialize(String path, Range range) {
+ requireKeyExtentDataRange(range);
+ final TabletFileCqMetadataGson metadata = new TabletFileCqMetadataGson();
+ metadata.path = Objects.requireNonNull(path);
+ metadata.startRow = encodeRow(range.getStartKey());
+ metadata.endRow = encodeRow(range.getEndKey());
+
+ return gson.toJson(metadata);
+ }
+
+ private static String serialize(TabletFileCq tabletFileCq) {
+ return serialize(Objects.requireNonNull(tabletFileCq).path.toString(), tabletFileCq.range);
+ }
+
+ /**
+ * Helper methods to encode and decode rows in a range to/from byte arrays. Null rows will just be
+ * returned as an empty byte array
+ **/
+
+ private static byte[] encodeRow(final Key key) {
+ final Text row = key != null ? key.getRow() : null;
+ if (row != null) {
+ try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(baos)) {
+ row.write(dos);
+ dos.close();
+ return baos.toByteArray();
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+ // Empty byte array means null row
+ return new byte[0];
+ }
+
+ private static Text decodeRow(byte[] serialized) {
+ // Empty byte array means null row
+ if (serialized.length == 0) {
+ return null;
+ }
+
+ try (DataInputBuffer buffer = new DataInputBuffer()) {
+ final Text row = new Text();
+ buffer.reset(serialized, serialized.length);
+ row.readFields(buffer);
+ return row;
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
+ /**
+ * Quick validation to see if value has been converted by checking if the candidate looks like
+ * json by checking the candidate starts with "{" and ends with "}".
+ *
+ * @param candidate a possible file: reference.
+ * @return false if a likely a json object, true if not a likely json object
+ */
+ @VisibleForTesting
+ public static boolean fileNeedsConversion(@NonNull final String candidate) {
+ String trimmed = candidate.trim();
+ return !trimmed.startsWith("{") || !trimmed.endsWith("}");
+ }
+
+ private static class TabletFileCq {
+ public final Path path;
+ public final Range range;
+
+ public TabletFileCq(Path path, Range range) {
+ this.path = Objects.requireNonNull(path);
+ this.range = Objects.requireNonNull(range);
+ }
+ }
+
+ private static class TabletFileCqMetadataGson {
+ private String path;
+ private byte[] startRow;
+ private byte[] endRow;
+ }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java b/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java
index 8b307ca..e481369 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/SuspendingTServer.java
@@ -19,30 +19,34 @@
package org.apache.accumulo.core.metadata;
import java.util.Objects;
+import java.util.concurrent.TimeUnit;
import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.time.SteadyTime;
+
+import com.google.common.net.HostAndPort;
/**
* For a suspended tablet, the time of suspension and the server it was suspended from.
*/
public class SuspendingTServer {
public final HostAndPort server;
- public final long suspensionTime;
+ public final SteadyTime suspensionTime;
- SuspendingTServer(HostAndPort server, long suspensionTime) {
+ SuspendingTServer(HostAndPort server, SteadyTime suspensionTime) {
this.server = Objects.requireNonNull(server);
- this.suspensionTime = suspensionTime;
+ this.suspensionTime = Objects.requireNonNull(suspensionTime);
}
public static SuspendingTServer fromValue(Value value) {
String valStr = value.toString();
String[] parts = valStr.split("[|]", 2);
- return new SuspendingTServer(HostAndPort.fromString(parts[0]), Long.parseLong(parts[1]));
+ return new SuspendingTServer(HostAndPort.fromString(parts[0]),
+ SteadyTime.from(Long.parseLong(parts[1]), TimeUnit.MILLISECONDS));
}
- public static Value toValue(TServerInstance tServer, long suspensionTime) {
- return new Value(tServer.getHostPort() + "|" + suspensionTime);
+ public static Value toValue(TServerInstance tServer, SteadyTime suspensionTime) {
+ return new Value(tServer.getHostPort() + "|" + suspensionTime.getMillis());
}
@Override
@@ -51,7 +55,7 @@
return false;
}
SuspendingTServer rhs = (SuspendingTServer) rhsObject;
- return server.equals(rhs.server) && suspensionTime == rhs.suspensionTime;
+ return server.equals(rhs.server) && suspensionTime.equals(rhs.suspensionTime);
}
@Override
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java b/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java
index 2bf8ca4..9e26102 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/TServerInstance.java
@@ -22,9 +22,10 @@
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.hadoop.io.Text;
+import com.google.common.net.HostAndPort;
+
/**
* A tablet is assigned to a tablet server at the given address as long as it is alive and well.
* When the tablet server is restarted, the instance information it advertises will change.
@@ -61,11 +62,11 @@
}
public TServerInstance(String address, long session) {
- this(AddressUtil.parseAddress(address, false), Long.toHexString(session));
+ this(AddressUtil.parseAddress(address), Long.toHexString(session));
}
public TServerInstance(Value address, Text session) {
- this(AddressUtil.parseAddress(new String(address.get(), UTF_8), false), session.toString());
+ this(AddressUtil.parseAddress(new String(address.get(), UTF_8)), session.toString());
}
@Override
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java
index f216e24..8b90bd7 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java
@@ -18,224 +18,43 @@
*/
package org.apache.accumulo.core.metadata;
-import static org.apache.accumulo.core.Constants.HDFS_TABLES_DIR;
-
-import java.net.URI;
-import java.util.Objects;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.commons.lang3.StringUtils;
+import org.apache.accumulo.core.data.Range;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Preconditions;
/**
- * Object representing a tablet file that may exist in the metadata table. This class is used for
- * reading and opening tablet files. It is also used when inserting new tablet files. When a new
- * file is inserted, the {@link #insert()} method is called and returns a {@link StoredTabletFile}
- * For situations where a tablet file needs to be updated or deleted in the metadata, a
- * {@link StoredTabletFile} is required.
- * <p>
- * As of 2.1, Tablet file paths should now be only absolute URIs with the removal of relative paths
- * in Upgrader9to10.upgradeRelativePaths()
+ * An interface that represents different types of file references that are handled by code that
+ * processes tablet files.
*/
-public class TabletFile implements Comparable<TabletFile> {
-
- public static class FileParts {
-
- // parts of an absolute URI, like "hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf"
- // volume: hdfs://1.2.3.4/accumulo
- // tableId: 2a
- // tabletDir: t-0003
- // fileName: C0004.rf
- // normalizedPath: hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf
- private final String volume;
- private final TableId tableId;
- private final String tabletDir;
- private final String fileName;
- private final String normalizedPath;
-
- public FileParts(String volume, TableId tableId, String tabletDir, String fileName,
- String normalizedPath) {
- this.volume = volume;
- this.tableId = tableId;
- this.tabletDir = tabletDir;
- this.fileName = fileName;
- this.normalizedPath = normalizedPath;
- }
-
- public String getVolume() {
- return volume;
- }
-
- public TableId getTableId() {
- return tableId;
- }
-
- public String getTabletDir() {
- return tabletDir;
- }
-
- public String getFileName() {
- return fileName;
- }
-
- public String getNormalizedPath() {
- return normalizedPath;
- }
-
- }
-
- private static String constructErrorMsg(Path filePath) {
- return "Missing or invalid part of tablet file metadata entry: " + filePath;
- }
-
- public static FileParts parsePath(Path filePath) {
- // File name construct: <volume>/<tablePath>/<tableId>/<tablet>/<file>
- // Example: hdfs://namenode:9020/accumulo/tables/1/default_tablet/F00001.rf
- final URI uri = filePath.toUri();
-
- // validate that this is a fully qualified uri
- Preconditions.checkArgument(uri.getScheme() != null, constructErrorMsg(filePath));
-
- final String path = uri.getPath(); // ex: /accumulo/tables/1/default_tablet/F00001.rf
- final String[] parts = path.split("/");
- final int numParts = parts.length; // should contain tables, 1, default_tablet, F00001.rf
-
- if (numParts < 4) {
- throw new IllegalArgumentException(constructErrorMsg(filePath));
- }
-
- final String fileName = parts[numParts - 1];
- final String tabletDirectory = parts[numParts - 2];
- final TableId tableId = TableId.of(parts[numParts - 3]);
- final String tablesPath = parts[numParts - 4];
-
- // determine where file path starts, the rest is the volume
- final String computedFilePath =
- HDFS_TABLES_DIR + "/" + tableId.canonical() + "/" + tabletDirectory + "/" + fileName;
- final String uriString = uri.toString();
- int idx = uriString.lastIndexOf(computedFilePath);
-
- if (idx == -1) {
- throw new IllegalArgumentException(constructErrorMsg(filePath));
- }
-
- // The volume is the beginning portion of the uri up to the start
- // of the file path.
- final String volume = uriString.substring(0, idx);
-
- if (StringUtils.isBlank(fileName) || StringUtils.isBlank(tabletDirectory)
- || StringUtils.isBlank(tablesPath) || StringUtils.isBlank(volume)) {
- throw new IllegalArgumentException(constructErrorMsg(filePath));
- }
- ValidationUtil.validateFileName(fileName);
- Preconditions.checkArgument(tablesPath.equals(HDFS_TABLES_DIR_NAME),
- "tables directory name is not " + HDFS_TABLES_DIR_NAME + ", is " + tablesPath);
-
- final String normalizedPath = volume + computedFilePath;
-
- if (!normalizedPath.equals(uriString)) {
- throw new RuntimeException("Error parsing file path, " + normalizedPath + " != " + uriString);
- }
-
- return new FileParts(volume, tableId, tabletDirectory, fileName, normalizedPath);
-
- }
-
- protected final Path metaPath;
- private final FileParts parts;
-
- private static final Logger log = LoggerFactory.getLogger(TabletFile.class);
- private static final String HDFS_TABLES_DIR_NAME = HDFS_TABLES_DIR.substring(1);
+public interface TabletFile {
/**
- * Construct new tablet file using a Path. Used in the case where we had to use Path object to
- * qualify an absolute path or create a new file.
+ * Returns the fileName of the TabletFile. The value return is the name itself and not the entire
+ * path. For example, if the full path for a TabletFile is
+ * 'hdfs://nn1/accumulo/tables/5a/t-0001/F0002.rf', this method returns 'F0002.rf'.
*/
- public TabletFile(Path metaPath) {
- this.metaPath = Objects.requireNonNull(metaPath);
- log.trace("Parsing TabletFile from {}", metaPath);
- parts = parsePath(metaPath);
- }
-
- public String getVolume() {
- return parts.getVolume();
- }
-
- public TableId getTableId() {
- return parts.getTableId();
- }
-
- public String getTabletDir() {
- return parts.getTabletDir();
- }
-
- public String getFileName() {
- return parts.getFileName();
- }
+ String getFileName();
/**
- * Return a string for opening and reading the tablet file. Doesn't have to be exact string in
- * metadata.
+ * Returns the full path for the TabletFile on the file system. The path may be normalized
+ * depending on the specific implementation. For example, a path in hdfs would be returned as
+ * 'hdfs://nn1/accumulo/tables/5a/t-0001/F0002.rf'
*/
- public String getPathStr() {
- return parts.getNormalizedPath();
- }
+ Path getPath();
/**
- * Return a string for inserting a new tablet file.
+ * @return The range of the TabletFile
+ *
*/
- public String getMetaInsert() {
- return parts.getNormalizedPath();
- }
+ Range getRange();
/**
- * Return a new Text object of {@link #getMetaInsert()}
+ * @return True if this file is fenced by a range
+ *
*/
- public Text getMetaInsertText() {
- return new Text(getMetaInsert());
- }
+ boolean hasRange();
/**
- * New file was written to metadata so return a StoredTabletFile
+ * @return a string with the filename and row range if there is one.
*/
- public StoredTabletFile insert() {
- return new StoredTabletFile(parts.getNormalizedPath());
- }
-
- public Path getPath() {
- return metaPath;
- }
-
- @Override
- public int compareTo(TabletFile o) {
- if (equals(o)) {
- return 0;
- } else {
- return parts.getNormalizedPath().compareTo(o.parts.getNormalizedPath());
- }
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj instanceof TabletFile) {
- TabletFile that = (TabletFile) obj;
- return parts.getNormalizedPath().equals(that.parts.getNormalizedPath());
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return parts.getNormalizedPath().hashCode();
- }
-
- @Override
- public String toString() {
- return parts.getNormalizedPath();
- }
+ String toMinimalString();
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TabletLocationState.java b/core/src/main/java/org/apache/accumulo/core/metadata/TabletLocationState.java
index 0a70594..4c2421b 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/TabletLocationState.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/TabletLocationState.java
@@ -28,6 +28,7 @@
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.metadata.schema.TabletMetadata;
import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
import org.apache.accumulo.core.util.TextUtil;
import org.apache.hadoop.io.Text;
@@ -57,8 +58,7 @@
}
public TabletLocationState(KeyExtent extent, Location future, Location current, Location last,
- SuspendingTServer suspend, Collection<Collection<String>> walogs, boolean chopped)
- throws BadLocationStateException {
+ SuspendingTServer suspend, Collection<LogEntry> walogs) throws BadLocationStateException {
this.extent = Objects.requireNonNull(extent);
this.future = validateLocation(future, TabletMetadata.LocationType.FUTURE);
this.current = validateLocation(current, TabletMetadata.LocationType.CURRENT);
@@ -68,7 +68,6 @@
walogs = Collections.emptyList();
}
this.walogs = walogs;
- this.chopped = chopped;
if (hasCurrent() && hasFuture()) {
throw new BadLocationStateException(
extent + " is both assigned and hosted, which should never happen: " + this,
@@ -81,8 +80,7 @@
public final Location current;
public final Location last;
public final SuspendingTServer suspend;
- public final Collection<Collection<String>> walogs;
- public final boolean chopped;
+ public final Collection<LogEntry> walogs;
public TServerInstance getCurrentServer() {
return serverInstance(current);
@@ -151,7 +149,7 @@
@Override
public String toString() {
- return extent + "@(" + future + "," + current + "," + last + ")" + (chopped ? " chopped" : "");
+ return extent + "@(" + future + "," + current + "," + last + ")";
}
private static Location validateLocation(final Location location,
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/UnreferencedTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/UnreferencedTabletFile.java
new file mode 100644
index 0000000..3d5c1d4
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/UnreferencedTabletFile.java
@@ -0,0 +1,109 @@
+/*
+ * 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.metadata;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.Range;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * A file that is not intended to be added to a tablet as a reference, within the scope of the code
+ * using this class, but needs to be passed to code that processes tablet files. These files could
+ * be temp files or files directly created by a user for bulk import. The file may ultimately be
+ * added to a tablet later as a new file reference, but within a different scope (process, thread,
+ * code block, method, etc.) that uses a different class to represent the file in that scope.
+ *
+ * Unlike {@link ReferencedTabletFile}, this class does not perform any validation or normalization
+ * on the provided path.
+ *
+ * @since 3.0.0
+ */
+public class UnreferencedTabletFile extends AbstractTabletFile<UnreferencedTabletFile> {
+
+ private final String fileName; // C0004.rf
+
+ public UnreferencedTabletFile(FileSystem fs, Path path) {
+ this(fs, path, new Range());
+ }
+
+ public UnreferencedTabletFile(FileSystem fs, Path path, Range range) {
+ super(Objects.requireNonNull(fs).makeQualified(Objects.requireNonNull(path)), range);
+ this.fileName = path.getName();
+ ValidationUtil.validateFileName(fileName);
+ }
+
+ @Override
+ public String getFileName() {
+ return fileName;
+ }
+
+ @Override
+ public int compareTo(UnreferencedTabletFile o) {
+ if (equals(o)) {
+ return 0;
+ } else {
+ return path.compareTo(o.path);
+ }
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof UnreferencedTabletFile) {
+ UnreferencedTabletFile that = (UnreferencedTabletFile) obj;
+ return path.equals(that.path);
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return path.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return path.toString();
+ }
+
+ public static UnreferencedTabletFile of(FileSystem fs, File file) {
+ return new UnreferencedTabletFile(fs, new Path(Objects.requireNonNull(file).toString()));
+ }
+
+ public static UnreferencedTabletFile ofRanged(FileSystem fs, File file, Range range) {
+ return new UnreferencedTabletFile(fs, new Path(Objects.requireNonNull(file).toString()), range);
+ }
+
+ public static UnreferencedTabletFile of(FileSystem fs, Path path) {
+ return new UnreferencedTabletFile(fs, path);
+ }
+
+ public static UnreferencedTabletFile ofRanged(FileSystem fs, Path path, Range range) {
+ return new UnreferencedTabletFile(fs, path, range);
+ }
+
+ public static UnreferencedTabletFile of(Configuration conf, Path path) throws IOException {
+ return new UnreferencedTabletFile(Objects.requireNonNull(path).getFileSystem(conf), path);
+ }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java b/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java
index db60a89..26d49e5 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java
@@ -43,7 +43,7 @@
}
public static ReferenceFile validate(ReferenceFile reference) {
- validate(new Path(reference.getMetadataEntry()));
+ validate(new Path(reference.getMetadataPath()));
return reference;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
index f7232d7..b212356 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
@@ -20,24 +20,23 @@
import java.util.Collection;
import java.util.Iterator;
-import java.util.UUID;
import java.util.stream.Stream;
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
import org.apache.accumulo.core.gc.GcCandidate;
import org.apache.accumulo.core.gc.ReferenceFile;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
+import org.apache.accumulo.core.metadata.ScanServerRefStore;
import org.apache.accumulo.core.metadata.StoredTabletFile;
import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.metadata.TabletFile;
import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.core.util.time.SteadyTime;
import org.apache.hadoop.io.Text;
/**
@@ -74,8 +73,8 @@
*/
public enum DataLevel {
ROOT(null, null),
- METADATA(RootTable.NAME, RootTable.ID),
- USER(MetadataTable.NAME, MetadataTable.ID);
+ METADATA(AccumuloTable.ROOT.tableName(), AccumuloTable.ROOT.tableId()),
+ USER(AccumuloTable.METADATA.tableName(), AccumuloTable.METADATA.tableId());
private final String table;
private final TableId id;
@@ -106,9 +105,9 @@
}
public static DataLevel of(TableId tableId) {
- if (tableId.equals(RootTable.ID)) {
+ if (tableId.equals(AccumuloTable.ROOT.tableId())) {
return DataLevel.ROOT;
- } else if (tableId.equals(MetadataTable.ID)) {
+ } else if (tableId.equals(AccumuloTable.METADATA.tableId())) {
return DataLevel.METADATA;
} else {
return DataLevel.USER;
@@ -241,7 +240,7 @@
/**
* Return an encoded delete marker Mutation to delete the specified TabletFile path. A
* ReferenceFile is used for the parameter because the Garbage Collector is optimized to store a
- * directory for Tablet File. Otherwise, a {@link TabletFile} object could be used. The
+ * directory for Tablet File. Otherwise, a {@link ReferencedTabletFile} object could be used. The
* tabletFilePathToRemove is validated and normalized before creating the mutation.
*
* @param tabletFilePathToRemove String full path of the TabletFile
@@ -268,13 +267,13 @@
interface TabletMutator {
TabletMutator putPrevEndRow(Text per);
- TabletMutator putFile(TabletFile path, DataFileValue dfv);
+ TabletMutator putFile(ReferencedTabletFile path, DataFileValue dfv);
TabletMutator putFile(StoredTabletFile path, DataFileValue dfv);
TabletMutator deleteFile(StoredTabletFile path);
- TabletMutator putScan(TabletFile path);
+ TabletMutator putScan(StoredTabletFile path);
TabletMutator deleteScan(StoredTabletFile path);
@@ -292,19 +291,15 @@
TabletMutator putWal(LogEntry logEntry);
- TabletMutator deleteWal(String wal);
-
- TabletMutator deleteWal(LogEntry logEntry);
+ TabletMutator deleteWal(LogEntry wal);
TabletMutator putTime(MetadataTime time);
- TabletMutator putBulkFile(TabletFile bulkref, long tid);
+ TabletMutator putBulkFile(ReferencedTabletFile bulkref, long tid);
- TabletMutator deleteBulkFile(TabletFile bulkref);
+ TabletMutator deleteBulkFile(StoredTabletFile bulkref);
- TabletMutator putChopped();
-
- TabletMutator putSuspension(TServerInstance tserver, long suspensionTime);
+ TabletMutator putSuspension(TServerInstance tserver, SteadyTime suspensionTime);
TabletMutator deleteSuspension();
@@ -329,43 +324,6 @@
}
/**
- * Insert ScanServer references to Tablet files
- *
- * @param scanRefs set of scan server ref table file objects
- */
- default void putScanServerFileReferences(Collection<ScanServerRefTabletFile> scanRefs) {
- throw new UnsupportedOperationException();
- }
-
- /**
- * Get ScanServer references to Tablet files
- *
- * @return stream of scan server references
- */
- default Stream<ScanServerRefTabletFile> getScanServerFileReferences() {
- throw new UnsupportedOperationException();
- }
-
- /**
- * Delete the set of scan server references
- *
- * @param refsToDelete set of scan server references to delete
- */
- default void deleteScanServerFileReferences(Collection<ScanServerRefTabletFile> refsToDelete) {
- throw new UnsupportedOperationException();
- }
-
- /**
- * Delete scan server references for this server
- *
- * @param serverAddress address of server, cannot be null
- * @param serverSessionId server session id, cannot be null
- */
- default void deleteScanServerFileReferences(String serverAddress, UUID serverSessionId) {
- throw new UnsupportedOperationException();
- }
-
- /**
* Create a Bulk Load In Progress flag in the metadata table
*
* @param path The bulk directory filepath
@@ -384,6 +342,10 @@
throw new UnsupportedOperationException();
}
+ default ScanServerRefStore scanServerRefs() {
+ throw new UnsupportedOperationException();
+ }
+
/**
* Remove all the Bulk Load transaction ids from a given table's metadata
*
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/DataFileValue.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/DataFileValue.java
index 67c5e15..fe770d9 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/DataFileValue.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/DataFileValue.java
@@ -21,6 +21,8 @@
import static java.nio.charset.StandardCharsets.UTF_8;
import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iteratorsImpl.system.InterruptibleIterator;
+import org.apache.accumulo.core.iteratorsImpl.system.TimeSettingIterator;
public class DataFileValue {
private long size;
@@ -114,4 +116,23 @@
}
this.time = time;
}
+
+ /**
+ * @return true if {@link #wrapFileIterator} would wrap a given iterator, false otherwise.
+ */
+ public boolean willWrapIterator() {
+ return isTimeSet();
+ }
+
+ /**
+ * Use per file information from the metadata table to wrap the raw iterator over a file with
+ * iterators that may take action based on data set in the metadata table.
+ */
+ public InterruptibleIterator wrapFileIterator(InterruptibleIterator iter) {
+ if (isTimeSet()) {
+ return new TimeSettingIterator(iter, getTime());
+ } else {
+ return iter;
+ }
+ }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java
index 9212ccd..421a707 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java
@@ -18,6 +18,8 @@
*/
package org.apache.accumulo.core.metadata.schema;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
+
import java.util.Base64;
import org.apache.accumulo.core.data.TableId;
@@ -26,12 +28,9 @@
import org.apache.hadoop.io.Text;
import com.google.common.base.Preconditions;
-import com.google.gson.Gson;
public class ExternalCompactionFinalState {
- private static final Gson GSON = new Gson();
-
public enum FinalState {
FINISHED, FAILED
}
@@ -123,11 +122,11 @@
jd.fileSize = fileSize;
jd.entries = fileEntries;
jd.extent = new Extent(extent);
- return GSON.toJson(jd);
+ return GSON.get().toJson(jd);
}
public static ExternalCompactionFinalState fromJson(ExternalCompactionId ecid, String json) {
- JsonData jd = GSON.fromJson(json, JsonData.class);
+ JsonData jd = GSON.get().fromJson(json, JsonData.class);
return new ExternalCompactionFinalState(ecid, jd.extent.toKeyExtent(),
FinalState.valueOf(jd.state), jd.fileSize, jd.entries);
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java
index 71837f0..4c7e429 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java
@@ -20,27 +20,24 @@
import static java.util.stream.Collectors.toList;
import static java.util.stream.Collectors.toSet;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
import java.util.List;
import java.util.Objects;
import java.util.Set;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
import org.apache.accumulo.core.spi.compaction.CompactionKind;
import org.apache.accumulo.core.util.compaction.CompactionExecutorIdImpl;
-import org.apache.hadoop.fs.Path;
-
-import com.google.gson.Gson;
+import org.apache.accumulo.core.util.compaction.DeprecatedCompactionKind;
public class ExternalCompactionMetadata {
- private static final Gson GSON = new Gson();
-
private final Set<StoredTabletFile> jobFiles;
private final Set<StoredTabletFile> nextFiles;
- private final TabletFile compactTmpName;
+ private final ReferencedTabletFile compactTmpName;
private final String compactorId;
private final CompactionKind kind;
private final short priority;
@@ -50,11 +47,11 @@
private final Long compactionId;
public ExternalCompactionMetadata(Set<StoredTabletFile> jobFiles, Set<StoredTabletFile> nextFiles,
- TabletFile compactTmpName, String compactorId, CompactionKind kind, short priority,
+ ReferencedTabletFile compactTmpName, String compactorId, CompactionKind kind, short priority,
CompactionExecutorId ceid, boolean propagateDeletes, boolean initiallySelectedAll,
Long compactionId) {
if (!initiallySelectedAll && !propagateDeletes
- && (kind == CompactionKind.SELECTOR || kind == CompactionKind.USER)) {
+ && (kind == DeprecatedCompactionKind.SELECTOR || kind == CompactionKind.USER)) {
throw new IllegalArgumentException(
"When user or selector compactions do not propagate deletes, it's expected that all "
+ "files were selected initially.");
@@ -79,7 +76,7 @@
return nextFiles;
}
- public TabletFile getCompactTmpName() {
+ public ReferencedTabletFile getCompactTmpName() {
return compactTmpName;
}
@@ -129,10 +126,9 @@
public String toJson() {
GSonData jData = new GSonData();
- jData.inputs = jobFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
- jData.nextFiles =
- nextFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
- jData.tmp = compactTmpName.getMetaInsert();
+ jData.inputs = jobFiles.stream().map(StoredTabletFile::getMetadata).collect(toList());
+ jData.nextFiles = nextFiles.stream().map(StoredTabletFile::getMetadata).collect(toList());
+ jData.tmp = compactTmpName.insert().getMetadata();
jData.compactor = compactorId;
jData.kind = kind.name();
jData.executorId = ((CompactionExecutorIdImpl) ceid).getExternalName();
@@ -140,18 +136,19 @@
jData.propDels = propagateDeletes;
jData.selectedAll = initiallySelectedAll;
jData.compactionId = compactionId;
- return GSON.toJson(jData);
+ return GSON.get().toJson(jData);
}
public static ExternalCompactionMetadata fromJson(String json) {
- GSonData jData = GSON.fromJson(json, GSonData.class);
+ GSonData jData = GSON.get().fromJson(json, GSonData.class);
return new ExternalCompactionMetadata(
jData.inputs.stream().map(StoredTabletFile::new).collect(toSet()),
jData.nextFiles.stream().map(StoredTabletFile::new).collect(toSet()),
- new TabletFile(new Path(jData.tmp)), jData.compactor, CompactionKind.valueOf(jData.kind),
- jData.priority, CompactionExecutorIdImpl.externalId(jData.executorId), jData.propDels,
- jData.selectedAll, jData.compactionId);
+ StoredTabletFile.of(jData.tmp).getTabletFile(), jData.compactor,
+ CompactionKind.valueOf(jData.kind), jData.priority,
+ CompactionExecutorIdImpl.externalId(jData.executorId), jData.propDels, jData.selectedAll,
+ jData.compactionId);
}
@Override
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/LinkingIterator.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/LinkingIterator.java
index 5da06b8..7c8bf7b 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/LinkingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/LinkingIterator.java
@@ -18,8 +18,8 @@
*/
package org.apache.accumulo.core.metadata.schema;
+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.Iterator;
import java.util.function.Function;
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
index 5bb4e3a..e4356c5 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
@@ -18,14 +18,11 @@
*/
package org.apache.accumulo.core.metadata.schema;
-import static com.google.common.base.Preconditions.checkArgument;
import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
import java.util.regex.Pattern;
import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.data.ArrayByteSequence;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.PartialKey;
@@ -329,15 +326,27 @@
* data for the current tablet, so that they are safe to merge
*/
public static class ChoppedColumnFamily {
- public static final String STR_NAME = "chopped";
- public static final Text NAME = new Text(STR_NAME);
- public static final ColumnFQ CHOPPED_COLUMN = new ColumnFQ(NAME, new Text(STR_NAME));
+ // kept to support upgrades to 3.1; name is used for both col fam and col qual
+ @Deprecated(since = "3.1.0")
+ public static final Text NAME = new Text("chopped");
}
public static class ExternalCompactionColumnFamily {
public static final String STR_NAME = "ecomp";
public static final Text NAME = new Text(STR_NAME);
}
+
+ /**
+ * Column family for indicating that the files in a tablet contain fenced files that have been
+ * merged from other tablets during a merge operation. This is used to support resuming a failed
+ * merge operation.
+ */
+ public static class MergedColumnFamily {
+ public static final String STR_NAME = "merged";
+ public static final Text NAME = new Text(STR_NAME);
+ public static final ColumnFQ MERGED_COLUMN = new ColumnFQ(NAME, new Text(STR_NAME));
+ public static final Value MERGED_VALUE = new Value("merged");
+ }
}
/**
@@ -422,58 +431,6 @@
}
- /**
- * Holds references to files that need replication
- *
- * <pre>
- * <code>
- * ~replhdfs://localhost:8020/accumulo/wal/tserver+port/WAL stat:local_table_id [] -> protobuf
- * </code>
- * </pre>
- */
- public static class ReplicationSection {
- public static final Text COLF = new Text("stat");
- private static final ArrayByteSequence COLF_BYTE_SEQ = new ArrayByteSequence(COLF.toString());
- private static final Section section =
- new Section(RESERVED_PREFIX + "repl", true, RESERVED_PREFIX + "repm", false);
-
- public static Range getRange() {
- return section.getRange();
- }
-
- public static String getRowPrefix() {
- return section.getRowPrefix();
- }
-
- /**
- * Extract the table ID from the colfam
- *
- * @param k Key to extract from
- */
- public static TableId getTableId(Key k) {
- requireNonNull(k);
- return TableId.of(k.getColumnQualifier().toString());
- }
-
- /**
- * Extract the file name from the row suffix into the given {@link Text}
- *
- * @param k Key to extract from
- * @param buff Text to place file name into
- */
- public static void getFile(Key k, Text buff) {
- requireNonNull(k);
- requireNonNull(buff);
- checkArgument(COLF_BYTE_SEQ.equals(k.getColumnFamilyData()),
- "Given metadata replication status key with incorrect colfam");
-
- k.getRow(buff);
-
- buff.set(buff.getBytes(), section.getRowPrefix().length(),
- buff.getLength() - section.getRowPrefix().length());
- }
- }
-
public static class ExternalCompactionSection {
private static final Section section =
new Section(RESERVED_PREFIX + "ecomp", true, RESERVED_PREFIX + "ecomq", false);
@@ -486,31 +443,4 @@
return section.getRowPrefix();
}
}
-
- public static class ScanServerFileReferenceSection {
- private static final Section section =
- new Section(RESERVED_PREFIX + "scanfileref", true, RESERVED_PREFIX + "scanfilereg", false);
-
- public static Range getRange() {
- return section.getRange();
- }
-
- public static String getRowPrefix() {
- return section.getRowPrefix();
- }
- }
-
- @Deprecated(since = "2.1")
- public static class OldScanServerFileReferenceSection {
- private static final Section section =
- new Section(RESERVED_PREFIX + "sserv", true, RESERVED_PREFIX + "sserx", false);
-
- public static Range getRange() {
- return section.getRange();
- }
-
- public static String getRowPrefix() {
- return section.getRowPrefix();
- }
- }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java
index 3f01eed..cc16bb6 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java
@@ -20,6 +20,7 @@
import static com.google.common.base.Preconditions.checkArgument;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
import java.nio.ByteBuffer;
import java.nio.charset.CharacterCodingException;
@@ -38,10 +39,6 @@
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.gson.Gson;
/**
* This class is used to serialize and deserialize root tablet metadata using GSon. The only data
@@ -51,7 +48,6 @@
*/
public class RootTabletMetadata {
- private static final Logger log = LoggerFactory.getLogger(RootTabletMetadata.class);
private static final CharsetDecoder UTF8_error_detecting_decoder = UTF_8.newDecoder();
private static final Predicate<Entry<String,TreeMap<String,String>>> isLocationCF = e -> {
String fam = e.getKey();
@@ -59,7 +55,6 @@
|| fam.equals(FutureLocationColumnFamily.STR_NAME);
};
- // JSON Mapping Version 1. Released with Accumulo version 2.1.0
private static final int VERSION = 1;
// This class is used to serialize and deserialize root tablet metadata using GSon. Any changes to
@@ -74,10 +69,20 @@
*/
private final TreeMap<String,TreeMap<String,String>> columnValues;
- public Data(int version, TreeMap<String,TreeMap<String,String>> columnValues) {
+ private Data(int version, TreeMap<String,TreeMap<String,String>> columnValues) {
this.version = version;
this.columnValues = columnValues;
}
+
+ public int getVersion() {
+ return version;
+ }
+
+ public static boolean needsUpgrade(final String json) {
+ var rootData = GSON.get().fromJson(json, Data.class);
+ int currVersion = rootData.getVersion();
+ return currVersion < VERSION;
+ }
}
/**
@@ -93,12 +98,14 @@
}
}
- private final Gson gson = new Gson();
private final Data data;
public RootTabletMetadata(String json) {
- log.trace("Creating root tablet metadata from stored JSON: {}", json);
- this.data = gson.fromJson(json, Data.class);
+ this(GSON.get().fromJson(json, Data.class));
+ }
+
+ private RootTabletMetadata(final Data data) {
+ this.data = data;
checkArgument(data.version == VERSION, "Invalid Root Table Metadata JSON version %s",
data.version);
data.columnValues.forEach((fam, qualVals) -> {
@@ -108,7 +115,7 @@
}
public RootTabletMetadata() {
- this.data = new Data(VERSION, new TreeMap<>());
+ data = new Data(VERSION, new TreeMap<>());
}
/**
@@ -145,26 +152,33 @@
}
}
- /**
- * Convert this class to a {@link TabletMetadata}
- */
- public TabletMetadata toTabletMetadata() {
+ public Stream<SimpleImmutableEntry<Key,Value>> toKeyValues() {
String row = RootTable.EXTENT.toMetaRow().toString();
- // use a stream so we don't have to re-sort in a new TreeMap<Key,Value> structure
- Stream<SimpleImmutableEntry<Key,Value>> entries = data.columnValues.entrySet().stream()
+ return data.columnValues.entrySet().stream()
.flatMap(famToQualVal -> famToQualVal.getValue().entrySet().stream()
.map(qualVal -> new SimpleImmutableEntry<>(
new Key(row, famToQualVal.getKey(), qualVal.getKey(), 1),
new Value(qualVal.getValue()))));
- return TabletMetadata.convertRow(entries.iterator(),
+ }
+
+ /**
+ * Convert this class to a {@link TabletMetadata}
+ */
+ public TabletMetadata toTabletMetadata() {
+ // use a stream so we don't have to re-sort in a new TreeMap<Key,Value> structure
+ return TabletMetadata.convertRow(toKeyValues().iterator(),
EnumSet.allOf(TabletMetadata.ColumnType.class), false);
}
+ public static boolean needsUpgrade(final String json) {
+ return Data.needsUpgrade(json);
+ }
+
/**
* @return a JSON representation of the root tablet's data.
*/
public String toJson() {
- return gson.toJson(data);
+ return GSON.get().toJson(data);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
index c1e8161..33d307c 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
@@ -18,7 +18,6 @@
*/
package org.apache.accumulo.core.metadata.schema;
-import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.COMPACT_QUAL;
import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_QUAL;
import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.FLUSH_QUAL;
@@ -38,7 +37,6 @@
import java.util.Optional;
import java.util.OptionalLong;
import java.util.Set;
-import java.util.SortedMap;
import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.clientImpl.ClientContext;
@@ -47,16 +45,15 @@
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.fate.zookeeper.ServiceLock;
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
import org.apache.accumulo.core.metadata.StoredTabletFile;
import org.apache.accumulo.core.metadata.SuspendingTServer;
import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.metadata.TabletFile;
import org.apache.accumulo.core.metadata.TabletLocationState;
import org.apache.accumulo.core.metadata.TabletState;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
@@ -64,50 +61,81 @@
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.MergedColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.core.util.ServerServices;
import org.apache.hadoop.io.Text;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.ImmutableSortedMap.Builder;
+import com.google.common.net.HostAndPort;
public class TabletMetadata {
private static final Logger log = LoggerFactory.getLogger(TabletMetadata.class);
- private TableId tableId;
- private Text prevEndRow;
- private boolean sawPrevEndRow = false;
- private Text oldPrevEndRow;
- private boolean sawOldPrevEndRow = false;
- private Text endRow;
- private Location location;
- private Map<StoredTabletFile,DataFileValue> files;
- private List<StoredTabletFile> scans;
- private Map<TabletFile,Long> loadedFiles;
- private EnumSet<ColumnType> fetchedCols;
- private KeyExtent extent;
- private Location last;
- private SuspendingTServer suspend;
- private String dirName;
- private MetadataTime time;
- private String cloned;
- private SortedMap<Key,Value> keyValues;
- private OptionalLong flush = OptionalLong.empty();
- private List<LogEntry> logs;
- private OptionalLong compact = OptionalLong.empty();
- private Double splitRatio = null;
- private Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions;
- private boolean chopped = false;
+ private final TableId tableId;
+ private final Text prevEndRow;
+ private final boolean sawPrevEndRow;
+ private final Text oldPrevEndRow;
+ private final boolean sawOldPrevEndRow;
+ private final Text endRow;
+ private final Location location;
+ private final Map<StoredTabletFile,DataFileValue> files;
+ private final List<StoredTabletFile> scans;
+ private final Map<StoredTabletFile,Long> loadedFiles;
+ private final EnumSet<ColumnType> fetchedCols;
+ private final Supplier<KeyExtent> extent;
+ private final Location last;
+ private final SuspendingTServer suspend;
+ private final String dirName;
+ private final MetadataTime time;
+ private final String cloned;
+ private final List<Entry<Key,Value>> keyValues;
+ private final OptionalLong flush;
+ private final List<LogEntry> logs;
+ private final OptionalLong compact;
+ private final Double splitRatio;
+ private final Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions;
+ private final boolean merged;
+
+ private TabletMetadata(Builder tmBuilder) {
+ this.tableId = tmBuilder.tableId;
+ this.prevEndRow = tmBuilder.prevEndRow;
+ this.sawPrevEndRow = tmBuilder.sawPrevEndRow;
+ this.oldPrevEndRow = tmBuilder.oldPrevEndRow;
+ this.sawOldPrevEndRow = tmBuilder.sawOldPrevEndRow;
+ this.endRow = tmBuilder.endRow;
+ this.location = tmBuilder.location;
+ this.files = Objects.requireNonNull(tmBuilder.files.build());
+ this.scans = Objects.requireNonNull(tmBuilder.scans.build());
+ this.loadedFiles = tmBuilder.loadedFiles.build();
+ this.fetchedCols = Objects.requireNonNull(tmBuilder.fetchedCols);
+ this.last = tmBuilder.last;
+ this.suspend = tmBuilder.suspend;
+ this.dirName = tmBuilder.dirName;
+ this.time = tmBuilder.time;
+ this.cloned = tmBuilder.cloned;
+ this.keyValues =
+ Optional.ofNullable(tmBuilder.keyValues).map(ImmutableList.Builder::build).orElse(null);
+ this.flush = tmBuilder.flush;
+ this.logs = Objects.requireNonNull(tmBuilder.logs.build());
+ this.compact = Objects.requireNonNull(tmBuilder.compact);
+ this.splitRatio = tmBuilder.splitRatio;
+ this.extCompactions = Objects.requireNonNull(tmBuilder.extCompactions.build());
+ this.merged = tmBuilder.merged;
+ this.extent =
+ Suppliers.memoize(() -> new KeyExtent(getTableId(), getEndRow(), getPrevEndRow()));
+ }
public enum LocationType {
CURRENT, FUTURE, LAST
@@ -129,8 +157,8 @@
COMPACT_ID,
SPLIT_RATIO,
SUSPEND,
- CHOPPED,
- ECOMP
+ ECOMP,
+ MERGED
}
public static class Location {
@@ -233,10 +261,7 @@
}
public KeyExtent getExtent() {
- if (extent == null) {
- extent = new KeyExtent(getTableId(), getEndRow(), getPrevEndRow());
- }
- return extent;
+ return extent.get();
}
private void ensureFetched(ColumnType col) {
@@ -285,7 +310,7 @@
return location != null && location.getType() == LocationType.CURRENT;
}
- public Map<TabletFile,Long> getLoaded() {
+ public Map<StoredTabletFile,Long> getLoaded() {
ensureFetched(ColumnType.LOADED);
return loadedFiles;
}
@@ -350,12 +375,12 @@
return splitRatio;
}
- public boolean hasChopped() {
- ensureFetched(ColumnType.CHOPPED);
- return chopped;
+ public boolean hasMerged() {
+ ensureFetched(ColumnType.MERGED);
+ return merged;
}
- public SortedMap<Key,Value> getKeyValues() {
+ public List<Entry<Key,Value>> getKeyValues() {
Preconditions.checkState(keyValues != null, "Requested key values when it was not saved");
return keyValues;
}
@@ -375,7 +400,7 @@
// only care about the state so don't need walogs and chopped params
// Use getExtent() when passing the extent as the private reference may not have been
// initialized yet. This will also ensure PREV_ROW was fetched
- var tls = new TabletLocationState(getExtent(), future, current, last, suspend, null, false);
+ var tls = new TabletLocationState(getExtent(), future, current, last, suspend, null);
return tls.getState(liveTServers);
} catch (TabletLocationState.BadLocationStateException blse) {
throw new IllegalArgumentException("Error creating TabletLocationState", blse);
@@ -392,16 +417,7 @@
EnumSet<ColumnType> fetchedColumns, boolean buildKeyValueMap) {
Objects.requireNonNull(rowIter);
- TabletMetadata te = new TabletMetadata();
- final ImmutableSortedMap.Builder<Key,Value> kvBuilder =
- buildKeyValueMap ? ImmutableSortedMap.naturalOrder() : null;
-
- final var filesBuilder = ImmutableMap.<StoredTabletFile,DataFileValue>builder();
- final var scansBuilder = ImmutableList.<StoredTabletFile>builder();
- final var logsBuilder = ImmutableList.<LogEntry>builder();
- final var extCompBuilder =
- ImmutableMap.<ExternalCompactionId,ExternalCompactionMetadata>builder();
- final var loadedFilesBuilder = ImmutableMap.<TabletFile,Long>builder();
+ final var tmBuilder = new Builder();
ByteSequence row = null;
while (rowIter.hasNext()) {
@@ -412,14 +428,14 @@
final String qual = key.getColumnQualifierData().toString();
if (buildKeyValueMap) {
- kvBuilder.put(key, kv.getValue());
+ tmBuilder.keyValue(kv);
}
if (row == null) {
row = key.getRowData();
KeyExtent ke = KeyExtent.fromMetaRow(key.getRow());
- te.endRow = ke.endRow();
- te.tableId = ke.tableId();
+ tmBuilder.endRow(ke.endRow());
+ tmBuilder.table(ke.tableId());
} else if (!row.equals(key.getRowData())) {
throw new IllegalArgumentException(
"Input contains more than one row : " + row + " " + key.getRowData());
@@ -429,15 +445,15 @@
case TabletColumnFamily.STR_NAME:
switch (qual) {
case PREV_ROW_QUAL:
- te.prevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
- te.sawPrevEndRow = true;
+ tmBuilder.prevEndRow(TabletColumnFamily.decodePrevEndRow(kv.getValue()));
+ tmBuilder.sawPrevEndRow(true);
break;
case OLD_PREV_ROW_QUAL:
- te.oldPrevEndRow = TabletColumnFamily.decodePrevEndRow(kv.getValue());
- te.sawOldPrevEndRow = true;
+ tmBuilder.oldPrevEndRow(TabletColumnFamily.decodePrevEndRow(kv.getValue()));
+ tmBuilder.sawOldPrevEndRow(true);
break;
case SPLIT_RATIO_QUAL:
- te.splitRatio = Double.parseDouble(val);
+ tmBuilder.splitRatio(Double.parseDouble(val));
break;
}
break;
@@ -446,88 +462,70 @@
case DIRECTORY_QUAL:
Preconditions.checkArgument(ServerColumnFamily.isValidDirCol(val),
"Saw invalid dir name %s %s", key, val);
- te.dirName = val;
+ tmBuilder.dirName(val);
break;
case TIME_QUAL:
- te.time = MetadataTime.parse(val);
+ tmBuilder.time(MetadataTime.parse(val));
break;
case FLUSH_QUAL:
- te.flush = OptionalLong.of(Long.parseLong(val));
+ tmBuilder.flush(Long.parseLong(val));
break;
case COMPACT_QUAL:
- te.compact = OptionalLong.of(Long.parseLong(val));
+ tmBuilder.compact(Long.parseLong(val));
break;
}
break;
case DataFileColumnFamily.STR_NAME:
- filesBuilder.put(new StoredTabletFile(qual), new DataFileValue(val));
+ tmBuilder.file(new StoredTabletFile(qual), new DataFileValue(val));
break;
case BulkFileColumnFamily.STR_NAME:
- loadedFilesBuilder.put(new StoredTabletFile(qual),
+ tmBuilder.loadedFile(new StoredTabletFile(qual),
BulkFileColumnFamily.getBulkLoadTid(val));
break;
case CurrentLocationColumnFamily.STR_NAME:
- te.setLocationOnce(val, qual, LocationType.CURRENT);
+ tmBuilder.location(val, qual, LocationType.CURRENT);
break;
case FutureLocationColumnFamily.STR_NAME:
- te.setLocationOnce(val, qual, LocationType.FUTURE);
+ tmBuilder.location(val, qual, LocationType.FUTURE);
break;
case LastLocationColumnFamily.STR_NAME:
- te.last = Location.last(val, qual);
+ tmBuilder.last(Location.last(val, qual));
break;
case SuspendLocationColumn.STR_NAME:
- te.suspend = SuspendingTServer.fromValue(kv.getValue());
+ tmBuilder.suspend(SuspendingTServer.fromValue(kv.getValue()));
break;
case ScanFileColumnFamily.STR_NAME:
- scansBuilder.add(new StoredTabletFile(qual));
+ tmBuilder.scan(new StoredTabletFile(qual));
break;
case ClonedColumnFamily.STR_NAME:
- te.cloned = val;
+ tmBuilder.cloned(val);
break;
case LogColumnFamily.STR_NAME:
- logsBuilder.add(LogEntry.fromMetaWalEntry(kv));
+ tmBuilder.log(LogEntry.fromMetaWalEntry(kv));
break;
case ExternalCompactionColumnFamily.STR_NAME:
- extCompBuilder.put(ExternalCompactionId.of(qual),
+ tmBuilder.extCompaction(ExternalCompactionId.of(qual),
ExternalCompactionMetadata.fromJson(val));
break;
- case ChoppedColumnFamily.STR_NAME:
- te.chopped = true;
+ case MergedColumnFamily.STR_NAME:
+ tmBuilder.merged(true);
break;
default:
throw new IllegalStateException("Unexpected family " + fam);
}
}
- te.files = filesBuilder.build();
- te.loadedFiles = loadedFilesBuilder.build();
- te.fetchedCols = fetchedColumns;
- te.scans = scansBuilder.build();
- te.logs = logsBuilder.build();
- te.extCompactions = extCompBuilder.build();
- if (buildKeyValueMap) {
- te.keyValues = kvBuilder.build();
- }
- return te;
- }
-
- private void setLocationOnce(String val, String qual, LocationType lt) {
- if (location != null) {
- throw new IllegalStateException("Attempted to set second location for tableId: " + tableId
- + " endrow: " + endRow + " -- " + location + " " + qual + " " + val);
- }
- location = new Location(val, qual, lt);
+ return tmBuilder.build(fetchedColumns);
}
@VisibleForTesting
static TabletMetadata create(String id, String prevEndRow, String endRow) {
- TabletMetadata te = new TabletMetadata();
- te.tableId = TableId.of(id);
- te.sawPrevEndRow = true;
- te.prevEndRow = prevEndRow == null ? null : new Text(prevEndRow);
- te.endRow = endRow == null ? null : new Text(endRow);
- te.fetchedCols = EnumSet.of(ColumnType.PREV_ROW);
- return te;
+ final var tmBuilder = new Builder();
+ tmBuilder.table(TableId.of(id));
+ tmBuilder.sawPrevEndRow(true);
+ tmBuilder.prevEndRow(prevEndRow == null ? null : new Text(prevEndRow));
+ tmBuilder.endRow(endRow == null ? null : new Text(endRow));
+ return tmBuilder.build(EnumSet.of(ColumnType.PREV_ROW));
}
/**
@@ -553,17 +551,139 @@
*/
private static Optional<TServerInstance> checkServer(ClientContext context, String path,
String zPath) {
- Optional<TServerInstance> server = Optional.empty();
final var lockPath = ServiceLock.path(path + "/" + zPath);
ZooCache.ZcStat stat = new ZooCache.ZcStat();
- byte[] lockData = ServiceLock.getLockData(context.getZooCache(), lockPath, stat);
-
log.trace("Checking server at ZK path = " + lockPath);
- if (lockData != null) {
- ServerServices services = new ServerServices(new String(lockData, UTF_8));
- HostAndPort client = services.getAddress(ServerServices.Service.TSERV_CLIENT);
- server = Optional.of(new TServerInstance(client, stat.getEphemeralOwner()));
+ return ServiceLock.getLockData(context.getZooCache(), lockPath, stat)
+ .map(sld -> sld.getAddress(ServiceLockData.ThriftService.TSERV))
+ .map(address -> new TServerInstance(address, stat.getEphemeralOwner()));
+ }
+
+ static class Builder {
+ private TableId tableId;
+ private Text prevEndRow;
+ private boolean sawPrevEndRow;
+ private Text oldPrevEndRow;
+ private boolean sawOldPrevEndRow;
+ private Text endRow;
+ private Location location;
+ private final ImmutableMap.Builder<StoredTabletFile,DataFileValue> files =
+ ImmutableMap.builder();
+ private final ImmutableList.Builder<StoredTabletFile> scans = ImmutableList.builder();
+ private final ImmutableMap.Builder<StoredTabletFile,Long> loadedFiles = ImmutableMap.builder();
+ private EnumSet<ColumnType> fetchedCols;
+ private Location last;
+ private SuspendingTServer suspend;
+ private String dirName;
+ private MetadataTime time;
+ private String cloned;
+ private ImmutableList.Builder<Entry<Key,Value>> keyValues;
+ private OptionalLong flush = OptionalLong.empty();
+ private final ImmutableList.Builder<LogEntry> logs = ImmutableList.builder();
+ private OptionalLong compact = OptionalLong.empty();
+ private Double splitRatio = null;
+ private final ImmutableMap.Builder<ExternalCompactionId,
+ ExternalCompactionMetadata> extCompactions = ImmutableMap.builder();
+ private boolean merged;
+
+ void table(TableId tableId) {
+ this.tableId = tableId;
}
- return server;
+
+ void endRow(Text endRow) {
+ this.endRow = endRow;
+ }
+
+ void prevEndRow(Text prevEndRow) {
+ this.prevEndRow = prevEndRow;
+ }
+
+ void sawPrevEndRow(boolean sawPrevEndRow) {
+ this.sawPrevEndRow = sawPrevEndRow;
+ }
+
+ void oldPrevEndRow(Text oldPrevEndRow) {
+ this.oldPrevEndRow = oldPrevEndRow;
+ }
+
+ void sawOldPrevEndRow(boolean sawOldPrevEndRow) {
+ this.sawOldPrevEndRow = sawOldPrevEndRow;
+ }
+
+ void splitRatio(Double splitRatio) {
+ this.splitRatio = splitRatio;
+ }
+
+ void dirName(String dirName) {
+ this.dirName = dirName;
+ }
+
+ void time(MetadataTime time) {
+ this.time = time;
+ }
+
+ void flush(long flush) {
+ this.flush = OptionalLong.of(flush);
+ }
+
+ void compact(long compact) {
+ this.compact = OptionalLong.of(compact);
+ }
+
+ void file(StoredTabletFile stf, DataFileValue dfv) {
+ this.files.put(stf, dfv);
+ }
+
+ void loadedFile(StoredTabletFile stf, Long tid) {
+ this.loadedFiles.put(stf, tid);
+ }
+
+ void location(String val, String qual, LocationType lt) {
+ if (location != null) {
+ throw new IllegalStateException("Attempted to set second location for tableId: " + tableId
+ + " endrow: " + endRow + " -- " + location + " " + qual + " " + val);
+ }
+ this.location = new Location(val, qual, lt);
+ }
+
+ void last(Location last) {
+ this.last = last;
+ }
+
+ void suspend(SuspendingTServer suspend) {
+ this.suspend = suspend;
+ }
+
+ void scan(StoredTabletFile stf) {
+ this.scans.add(stf);
+ }
+
+ void cloned(String cloned) {
+ this.cloned = cloned;
+ }
+
+ void log(LogEntry log) {
+ this.logs.add(log);
+ }
+
+ void extCompaction(ExternalCompactionId id, ExternalCompactionMetadata metadata) {
+ this.extCompactions.put(id, metadata);
+ }
+
+ void merged(boolean merged) {
+ this.merged = merged;
+ }
+
+ void keyValue(Entry<Key,Value> kv) {
+ if (this.keyValues == null) {
+ this.keyValues = ImmutableList.builder();
+ }
+ this.keyValues.add(kv);
+ }
+
+ TabletMetadata build(EnumSet<ColumnType> fetchedCols) {
+ this.fetchedCols = fetchedCols;
+ return new TabletMetadata(this);
+ }
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
index 6ce050c..95045a2 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
@@ -34,13 +34,17 @@
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Objects;
+import java.util.Optional;
import java.util.Set;
+import java.util.function.Consumer;
import java.util.function.Function;
+import java.util.stream.Collectors;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
@@ -59,7 +63,7 @@
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
import org.apache.accumulo.core.fate.zookeeper.ZooReader;
import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
import org.apache.accumulo.core.metadata.RootTable;
import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
import org.apache.accumulo.core.metadata.schema.Ample.ReadConsistency;
@@ -72,6 +76,7 @@
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.MergedColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
@@ -94,6 +99,8 @@
private final List<Text> families = new ArrayList<>();
private final List<ColumnFQ> qualifiers = new ArrayList<>();
private Set<KeyExtent> extentsToFetch = null;
+ private boolean fetchTablets = false;
+ private Optional<Consumer<KeyExtent>> notFoundHandler;
private Ample.DataLevel level;
private String table;
private Range range;
@@ -104,7 +111,6 @@
private TableId tableId;
private ReadConsistency readConsistency = ReadConsistency.IMMEDIATE;
private final AccumuloClient _client;
- private Collection<KeyExtent> extents = null;
Builder(AccumuloClient client) {
this._client = client;
@@ -112,7 +118,7 @@
@Override
public TabletsMetadata build() {
- if (extents != null) {
+ if (fetchTablets) {
// setting multiple extents with forTablets(extents) is mutually exclusive with these
// single-tablet options
checkState(range == null && table == null && level == null && !checkConsistency);
@@ -120,7 +126,8 @@
}
checkState((level == null) != (table == null),
- "scanTable() cannot be used in conjunction with forLevel(), forTable() or forTablet()");
+ "scanTable() cannot be used in conjunction with forLevel(), forTable() or forTablet() %s %s",
+ level, table);
if (level == DataLevel.ROOT) {
ClientContext ctx = ((ClientContext) _client);
return new TabletsMetadata(getRootMetadata(ctx, readConsistency));
@@ -132,7 +139,7 @@
private TabletsMetadata buildExtents(AccumuloClient client) {
Map<DataLevel,List<KeyExtent>> groupedExtents =
- extents.stream().collect(groupingBy(ke -> DataLevel.of(ke.tableId())));
+ extentsToFetch.stream().collect(groupingBy(ke -> DataLevel.of(ke.tableId())));
List<Iterable<TabletMetadata>> iterables = new ArrayList<>();
@@ -175,19 +182,39 @@
closables.add(scanner);
} catch (TableNotFoundException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
}
- return new TabletsMetadata(() -> {
+ if (notFoundHandler.isPresent()) {
+ HashSet<KeyExtent> extentsNotSeen = new HashSet<>(extentsToFetch);
+
+ var tablets = iterables.stream().flatMap(i -> StreamSupport.stream(i.spliterator(), false))
+ .filter(tabletMetadata -> extentsNotSeen.remove(tabletMetadata.getExtent()))
+ .collect(Collectors.toList());
+
+ extentsNotSeen.forEach(notFoundHandler.orElseThrow());
+
for (AutoCloseable closable : closables) {
- closable.close();
+ try {
+ closable.close();
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
}
- }, () -> iterables.stream().flatMap(i -> StreamSupport.stream(i.spliterator(), false))
- .filter(tabletMetadata -> extentsToFetch.contains(tabletMetadata.getExtent()))
- .iterator());
+
+ return new TabletsMetadata(() -> {}, tablets);
+ } else {
+ return new TabletsMetadata(() -> {
+ for (AutoCloseable closable : closables) {
+ closable.close();
+ }
+ }, () -> iterables.stream().flatMap(i -> StreamSupport.stream(i.spliterator(), false))
+ .filter(tabletMetadata -> extentsToFetch.contains(tabletMetadata.getExtent()))
+ .iterator());
+ }
}
@@ -239,7 +266,7 @@
return new TabletsMetadata(scanner, tmi);
}
} catch (TableNotFoundException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
@@ -253,7 +280,7 @@
@Override
public Options checkConsistency() {
- checkState(extents == null, "Unable to check consistency of non-contiguous tablets");
+ checkState(!fetchTablets, "Unable to check consistency of non-contiguous tablets");
this.checkConsistency = true;
return this;
}
@@ -310,6 +337,9 @@
case ECOMP:
families.add(ExternalCompactionColumnFamily.NAME);
break;
+ case MERGED:
+ families.add(MergedColumnFamily.NAME);
+ break;
default:
throw new IllegalArgumentException("Unknown col type " + colToFetch);
@@ -356,10 +386,12 @@
}
@Override
- public Options forTablets(Collection<KeyExtent> extents) {
+ public Options forTablets(Collection<KeyExtent> extents,
+ Optional<Consumer<KeyExtent>> notFoundHandler) {
this.level = null;
- this.extents = extents;
this.extentsToFetch = Set.copyOf(extents);
+ this.notFoundHandler = Objects.requireNonNull(notFoundHandler);
+ this.fetchTablets = true;
return this;
}
@@ -412,6 +444,8 @@
/**
* Checks that the metadata table forms a linked list and automatically backs up until it does.
+ * May cause {@link TabletDeletedException} to be thrown while reading tablets metadata in the
+ * case where a table is deleted or merge runs concurrently with scan.
*/
Options checkConsistency();
@@ -449,8 +483,15 @@
/**
* Get the tablet metadata for the given extents. This will only return tablets where the end
* row and prev end row exactly match the given extents.
+ *
+ * @param notFoundConsumer if a consumer is present, the extents that do not exists in the
+ * metadata store are passed to the consumer. If the missing extents are not needed, then
+ * pass Optional.empty() and it will be more efficient. Computing the missing extents
+ * requires buffering all tablet metadata in memory before returning anything, when
+ * Optional.empty() is passed this buffering is not done.
*/
- Options forTablets(Collection<KeyExtent> extents);
+ Options forTablets(Collection<KeyExtent> extents,
+ Optional<Consumer<KeyExtent>> notFoundConsumer);
/**
* This method automatically determines where the metadata for the passed in table ID resides.
@@ -465,7 +506,7 @@
* {@link TabletsSection#getRange()}
*/
default RangeOptions scanMetadataTable() {
- return scanTable(MetadataTable.NAME);
+ return scanTable(AccumuloTable.METADATA.tableName());
}
/**
@@ -557,7 +598,7 @@
byte[] bytes = zooReader.getData(path);
return new RootTabletMetadata(new String(bytes, UTF_8)).toTabletMetadata();
} catch (InterruptedException | KeeperException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
default:
throw new IllegalArgumentException("Unknown consistency level " + readConsistency);
@@ -592,7 +633,7 @@
// avoid wrapping runtime w/ runtime
throw e;
} catch (Exception e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java
index 4277e11..b6ef72b 100644
--- a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsInfo.java
@@ -24,7 +24,7 @@
import java.util.List;
import java.util.Objects;
-import org.apache.accumulo.core.util.HostAndPort;
+import com.google.common.net.HostAndPort;
import io.micrometer.core.instrument.MeterRegistry;
import io.micrometer.core.instrument.Tag;
diff --git a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java
index 8cf2ffc..ee6eb6e 100644
--- a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java
@@ -55,6 +55,13 @@
* <tr>
* <td>N/A</td>
* <td>N/A</td>
+ * <td>{@value #METRICS_LOW_MEMORY}</td>
+ * <td>Guage</td>
+ * <td>reports 1 when process memory usage is above threshold, 0 when memory is okay</td>
+ * </tr>
+ * <tr>
+ * <td>N/A</td>
+ * <td>N/A</td>
* <td>{@value #METRICS_COMPACTOR_MAJC_STUCK}</td>
* <td>LongTaskTimer</td>
* <td></td>
@@ -480,6 +487,13 @@
* <td>The compaction service information is in a tag:
* id={i|e}_{compactionServiceName}_{executor_name}</td>
* </tr>
+ * <tr>
+ * <td></td>
+ * <td></td>
+ * <td>{@link #METRICS_MAJC_PAUSED}</td>
+ * <td>Counter</td>
+ * <td></td>
+ * </tr>
* <!-- minor compactions -->
* <tr>
* <td>Queue</td>
@@ -495,40 +509,11 @@
* <td>Timer</td>
* <td></td>
* </tr>
- * <!-- replication -->
* <tr>
- * <td>ReplicationQueue</td>
- * <td>Stat</td>
- * <td>{@value #METRICS_REPLICATION_QUEUE}</td>
- * <td>Timer</td>
* <td></td>
- * </tr>
- * <tr>
- * <td>ReplicationQueue10m</td>
- * <td>Quantiles</td>
- * <td>N/A</td>
- * <td>N/A</td>
* <td></td>
- * </tr>
- * <tr>
- * <td>filesPendingReplication</td>
- * <td>Stat</td>
- * <td>{@value #METRICS_REPLICATION_PENDING_FILES}</td>
- * <td>Gauge</td>
- * <td></td>
- * </tr>
- * <tr>
- * <td>maxReplicationThreads</td>
- * <td>Stat</td>
- * <td>{@value #METRICS_REPLICATION_THREADS}</td>
- * <td>Gauge</td>
- * <td></td>
- * </tr>
- * <tr>
- * <td>numPeers</td>
- * <td>Stat</td>
- * <td>{@value #METRICS_REPLICATION_PEERS}</td>
- * <td>Gauge</td>
+ * <td>{@value #METRICS_MINC_PAUSED}</td>
+ * <td>Counter</td>
* <td></td>
* </tr>
* <!-- Updates (ingest) -->
@@ -604,6 +589,7 @@
Logger LOG = LoggerFactory.getLogger(MetricsProducer.class);
+ String METRICS_LOW_MEMORY = "accumulo.detected.low.memory";
String METRICS_SERVER_IDLE = "accumulo.server.idle";
String METRICS_COMPACTOR_PREFIX = "accumulo.compactor.";
@@ -637,16 +623,12 @@
String METRICS_MAJC_PREFIX = "accumulo.tserver.compactions.majc.";
String METRICS_MAJC_QUEUED = METRICS_MAJC_PREFIX + "queued";
String METRICS_MAJC_RUNNING = METRICS_MAJC_PREFIX + "running";
+ String METRICS_MAJC_PAUSED = METRICS_MAJC_PREFIX + "paused";
String METRICS_MINC_PREFIX = "accumulo.tserver.compactions.minc.";
String METRICS_MINC_QUEUED = METRICS_MINC_PREFIX + "queued";
String METRICS_MINC_RUNNING = METRICS_MINC_PREFIX + "running";
-
- String METRICS_REPLICATION_PREFIX = "accumulo.replication.";
- String METRICS_REPLICATION_QUEUE = METRICS_REPLICATION_PREFIX + "queue";
- String METRICS_REPLICATION_PENDING_FILES = METRICS_REPLICATION_PREFIX + "files.pending";
- String METRICS_REPLICATION_PEERS = METRICS_REPLICATION_PREFIX + "peers";
- String METRICS_REPLICATION_THREADS = METRICS_REPLICATION_PREFIX + "threads";
+ String METRICS_MINC_PAUSED = METRICS_MINC_PREFIX + "paused";
String METRICS_SCAN_PREFIX = "accumulo.scan.";
String METRICS_SCAN_TIMES = METRICS_SCAN_PREFIX + "times";
@@ -666,6 +648,8 @@
String METRICS_SCAN_QUERY_SCAN_RESULTS = METRICS_SCAN_PREFIX + "query.results";
String METRICS_SCAN_QUERY_SCAN_RESULTS_BYTES = METRICS_SCAN_PREFIX + "query.results.bytes";
String METRICS_SCAN_SCANNED_ENTRIES = METRICS_SCAN_PREFIX + "query.scanned.entries";
+ String METRICS_SCAN_PAUSED_FOR_MEM = METRICS_SCAN_PREFIX + "paused.for.memory";
+ String METRICS_SCAN_RETURN_FOR_MEM = METRICS_SCAN_PREFIX + "return.early.for.memory";
String METRICS_SCAN_TABLET_METADATA_CACHE = METRICS_SCAN_PREFIX + "tablet.metadata.cache";
@@ -702,11 +686,15 @@
String METRICS_BLOCKCACHE_PREFIX = "accumulo.blockcache.";
String METRICS_BLOCKCACHE_INDEX_HITCOUNT = METRICS_BLOCKCACHE_PREFIX + "index.hitcount";
String METRICS_BLOCKCACHE_INDEX_REQUESTCOUNT = METRICS_BLOCKCACHE_PREFIX + "index.requestcount";
+ String METRICS_BLOCKCACHE_INDEX_EVICTIONCOUNT = METRICS_BLOCKCACHE_PREFIX + "index.evictioncount";
String METRICS_BLOCKCACHE_DATA_HITCOUNT = METRICS_BLOCKCACHE_PREFIX + "data.hitcount";
String METRICS_BLOCKCACHE_DATA_REQUESTCOUNT = METRICS_BLOCKCACHE_PREFIX + "data.requestcount";
+ String METRICS_BLOCKCACHE_DATA_EVICTIONCOUNT = METRICS_BLOCKCACHE_PREFIX + "data.evictioncount";
String METRICS_BLOCKCACHE_SUMMARY_HITCOUNT = METRICS_BLOCKCACHE_PREFIX + "summary.hitcount";
String METRICS_BLOCKCACHE_SUMMARY_REQUESTCOUNT =
METRICS_BLOCKCACHE_PREFIX + "summary.requestcount";
+ String METRICS_BLOCKCACHE_SUMMARY_EVICTIONCOUNT =
+ METRICS_BLOCKCACHE_PREFIX + "summary.evictioncount";
/**
* Build Micrometer Meter objects and register them with the registry
diff --git a/core/src/main/java/org/apache/accumulo/core/protobuf/ProtobufUtil.java b/core/src/main/java/org/apache/accumulo/core/protobuf/ProtobufUtil.java
deleted file mode 100644
index b7ab266..0000000
--- a/core/src/main/java/org/apache/accumulo/core/protobuf/ProtobufUtil.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.protobuf;
-
-import org.apache.accumulo.core.data.Value;
-
-import com.google.protobuf.GeneratedMessageV3;
-import com.google.protobuf.TextFormat;
-
-/**
- * Helper methods for interacting with Protocol Buffers and Accumulo
- */
-public class ProtobufUtil {
- private static final char LEFT_BRACKET = '[', RIGHT_BRACKET = ']';
-
- public static Value toValue(GeneratedMessageV3 msg) {
- return new Value(msg.toByteArray());
- }
-
- public static String toString(GeneratedMessageV3 msg) {
- // Too much typing
- return LEFT_BRACKET + TextFormat.shortDebugString(msg) + RIGHT_BRACKET;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/AccumuloReplicationReplayer.java b/core/src/main/java/org/apache/accumulo/core/replication/AccumuloReplicationReplayer.java
deleted file mode 100644
index 512c732..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/AccumuloReplicationReplayer.java
+++ /dev/null
@@ -1,32 +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.replication;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.replication.thrift.RemoteReplicationException;
-import org.apache.accumulo.core.replication.thrift.WalEdits;
-
-@Deprecated(since = "2.1.0")
-public interface AccumuloReplicationReplayer {
-
- long replicateLog(ClientContext context, String tableName, WalEdits data)
- throws RemoteReplicationException, AccumuloException, AccumuloSecurityException;
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java
deleted file mode 100644
index a2ada12..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java
+++ /dev/null
@@ -1,46 +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.replication;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-
-/**
- * Encapsulates configuration semantics around replication
- */
-@Deprecated
-public class ReplicationConfigurationUtil {
-
- /**
- * Determines if the replication is enabled for the given {@link KeyExtent}
- *
- * @param extent The {@link KeyExtent} for the Tablet in question
- * @param conf The {@link AccumuloConfiguration} for that Tablet (table or namespace)
- * @return True if this extent is a candidate for replication at the given point in time.
- */
- public static boolean isEnabled(KeyExtent extent, AccumuloConfiguration conf) {
- if (extent.isMeta() || extent.isRootTablet()) {
- return false;
- }
-
- return conf.getBoolean(Property.TABLE_REPLICATION);
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConstants.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConstants.java
deleted file mode 100644
index 4adde40..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConstants.java
+++ /dev/null
@@ -1,29 +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.replication;
-
-@Deprecated
-public class ReplicationConstants {
-
- // Constants for replication information in zookeeper
- public static final String ZOO_BASE = "/replication";
- public static final String ZOO_WORK_QUEUE = ZOO_BASE + "/workqueue";
- public static final String ZOO_TSERVERS = ZOO_BASE + "/tservers";
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
deleted file mode 100644
index e22a918..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
+++ /dev/null
@@ -1,281 +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.replication;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
-
-import java.nio.charset.CharacterCodingException;
-
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.lexicoder.ULongLexicoder;
-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.Mutation;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ReplicationSchema {
- private static final Logger log = LoggerFactory.getLogger(ReplicationSchema.class);
-
- /**
- * Portion of a file that must be replication to the given target: peer and some identifying
- * location on that peer, e.g. remote table ID
- *
- * <pre>
- * <code>
- * hdfs://localhost:8020/accumulo/wal/tserver+port/WAL work:serialized_ReplicationTarget []
- * -> Status Protobuf
- * </code>
- * </pre>
- */
- @Deprecated
- public static class WorkSection {
- public static final Text NAME = new Text("work");
- private static final ByteSequence BYTE_SEQ_NAME = new ArrayByteSequence("work");
-
- public static void getFile(Key k, Text buff) {
- requireNonNull(k);
- requireNonNull(buff);
- checkArgument(BYTE_SEQ_NAME.equals(k.getColumnFamilyData()),
- "Given replication work key with incorrect colfam");
- _getFile(k, buff);
- }
-
- public static ReplicationTarget getTarget(Key k, Text buff) {
- checkArgument(BYTE_SEQ_NAME.equals(k.getColumnFamilyData()),
- "Given replication work key with incorrect colfam");
- k.getColumnQualifier(buff);
-
- return ReplicationTarget.from(buff);
- }
-
- /**
- * Limit the scanner to only pull replication work records
- */
- public static void limit(ScannerBase scanner) {
- scanner.fetchColumnFamily(NAME);
- }
-
- public static Mutation add(Mutation m, Text serializedTarget, Value v) {
- m.put(NAME, serializedTarget, v);
- return m;
- }
- }
-
- /**
- * Holds replication markers tracking status for files
- *
- * <pre>
- * <code>
- * hdfs://localhost:8020/accumulo/wal/tserver+port/WAL repl:local_table_id []
- * -> Status Protobuf
- * </code>
- * </pre>
- */
- @Deprecated
- public static class StatusSection {
- public static final Text NAME = new Text("repl");
- private static final ByteSequence BYTE_SEQ_NAME = new ArrayByteSequence("repl");
-
- /**
- * Extract the table ID from the key (inefficiently if called repeatedly)
- *
- * @param k Key to extract from
- * @return The table ID
- */
- public static TableId getTableId(Key k) {
- requireNonNull(k);
- return TableId.of(k.getColumnQualifier().toString());
- }
-
- /**
- * Extract the file name from the row suffix into the given {@link Text}
- *
- * @param k Key to extract from
- * @param buff Text to place file name into
- */
- public static void getFile(Key k, Text buff) {
- requireNonNull(k);
- requireNonNull(buff);
- checkArgument(BYTE_SEQ_NAME.equals(k.getColumnFamilyData()),
- "Given replication status key with incorrect colfam");
-
- _getFile(k, buff);
- }
-
- /**
- * Limit the scanner to only return Status records
- */
- public static void limit(ScannerBase scanner) {
- scanner.fetchColumnFamily(NAME);
- }
-
- public static Mutation add(Mutation m, TableId tableId, Value v) {
- m.put(NAME, new Text(tableId.canonical()), v);
- return m;
- }
- }
-
- /**
- * Holds the order in which files needed for replication were closed. The intent is to be able to
- * guarantee that files which were closed earlier were replicated first and we don't replay data
- * in the wrong order on our peers
- *
- * <pre>
- * <code>
- * encodedTimeOfClosure\x00hdfs://localhost:8020/accumulo/wal/tserver+port/WAL
- * order:source_table_id [] -> Status Protobuf
- * </code>
- * </pre>
- */
- @Deprecated
- public static class OrderSection {
- public static final Text NAME = new Text("order");
- public static final Text ROW_SEPARATOR = new Text(new byte[] {0});
- private static final ULongLexicoder longEncoder = new ULongLexicoder();
-
- /**
- * Extract the table ID from the given key
- *
- * @param k OrderSection key
- * @param buff Text to place table ID into
- */
- public static void getTableId(Key k, Text buff) {
- requireNonNull(k);
- requireNonNull(buff);
-
- k.getColumnQualifier(buff);
- }
-
- /**
- * Limit the scanner to only return Order records
- */
- public static void limit(ScannerBase scanner) {
- scanner.fetchColumnFamily(NAME);
- }
-
- /**
- * Creates the Mutation for the Order section for the given file and time
- *
- * @param file Filename
- * @param timeInMillis Time in millis that the file was closed
- * @return Mutation for the Order section
- */
- public static Mutation createMutation(String file, long timeInMillis) {
- requireNonNull(file);
- checkArgument(timeInMillis >= 0, "timeInMillis must be greater than zero");
-
- // Encode the time so it sorts properly
- byte[] rowPrefix = longEncoder.encode(timeInMillis);
- Text row = new Text(rowPrefix);
-
- // Normalize the file using Path
- Path p = new Path(file);
- String pathString = p.toUri().toString();
-
- log.trace("Normalized {} into {}", file, pathString);
-
- // Append the file as a suffix to the row
- row.append((ROW_SEPARATOR + pathString).getBytes(UTF_8), 0,
- pathString.length() + ROW_SEPARATOR.getLength());
-
- // Make the mutation and add the column update
- return new Mutation(row);
- }
-
- /**
- * Add a column update to the given mutation with the provided tableId and value
- *
- * @param m Mutation for OrderSection
- * @param tableId Source table id
- * @param v Serialized Status msg
- * @return The original Mutation
- */
- public static Mutation add(Mutation m, TableId tableId, Value v) {
- m.put(NAME, new Text(tableId.canonical()), v);
- return m;
- }
-
- public static long getTimeClosed(Key k) {
- return getTimeClosed(k, new Text());
- }
-
- public static long getTimeClosed(Key k, Text buff) {
- k.getRow(buff);
- int offset = 0;
- // find the last offset
- while (true) {
- int nextOffset = buff.find(ROW_SEPARATOR.toString(), offset + 1);
- if (nextOffset == -1) {
- break;
- }
- offset = nextOffset;
- }
-
- if (offset == -1) {
- throw new IllegalArgumentException(
- "Row does not contain expected separator for OrderSection");
- }
-
- byte[] encodedLong = new byte[offset];
- System.arraycopy(buff.getBytes(), 0, encodedLong, 0, offset);
- return longEncoder.decode(encodedLong);
- }
-
- public static String getFile(Key k) {
- Text buff = new Text();
- return getFile(k, buff);
- }
-
- public static String getFile(Key k, Text buff) {
- k.getRow(buff);
- int offset = 0;
- // find the last offset
- while (true) {
- int nextOffset = buff.find(ROW_SEPARATOR.toString(), offset + 1);
- if (nextOffset == -1) {
- break;
- }
- offset = nextOffset;
- }
-
- if (offset == -1) {
- throw new IllegalArgumentException(
- "Row does not contain expected separator for OrderSection");
- }
-
- try {
- return Text.decode(buff.getBytes(), offset + 1, buff.getLength() - (offset + 1));
- } catch (CharacterCodingException e) {
- throw new IllegalArgumentException("Could not decode file path", e);
- }
- }
- }
-
- private static void _getFile(Key k, Text buff) {
- k.getRow(buff);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
deleted file mode 100644
index 522393a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
+++ /dev/null
@@ -1,106 +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.replication;
-
-import java.util.Collections;
-import java.util.Map;
-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.BatchScanner;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.TableOfflineException;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Deprecated
-public class ReplicationTable {
- private static final Logger log = LoggerFactory.getLogger(ReplicationTable.class);
-
- public static final TableId ID = TableId.of("+rep");
- public static final String NAME = Namespace.ACCUMULO.name() + ".replication";
-
- public static final String COMBINER_NAME = "statuscombiner";
-
- public static final String STATUS_LG_NAME = StatusSection.NAME.toString();
- public static final Set<Text> STATUS_LG_COLFAMS = Collections.singleton(StatusSection.NAME);
- public static final String WORK_LG_NAME = WorkSection.NAME.toString();
- public static final Set<Text> WORK_LG_COLFAMS = Collections.singleton(WorkSection.NAME);
- public static final Map<String,Set<Text>> LOCALITY_GROUPS =
- Map.of(STATUS_LG_NAME, STATUS_LG_COLFAMS, WORK_LG_NAME, WORK_LG_COLFAMS);
-
- public static Scanner getScanner(AccumuloClient client) throws ReplicationTableOfflineException {
- try {
- return client.createScanner(NAME, Authorizations.EMPTY);
- } catch (TableNotFoundException e) {
- throw new AssertionError(NAME + " should exist, but doesn't.");
- } catch (TableOfflineException e) {
- throw new ReplicationTableOfflineException(e);
- }
- }
-
- public static BatchWriter getBatchWriter(AccumuloClient client)
- throws ReplicationTableOfflineException {
- try {
- return client.createBatchWriter(NAME);
- } catch (TableNotFoundException e) {
- throw new AssertionError(NAME + " should exist, but doesn't.");
- } catch (TableOfflineException e) {
- throw new ReplicationTableOfflineException(e);
- }
- }
-
- public static BatchScanner getBatchScanner(AccumuloClient client, int queryThreads)
- throws ReplicationTableOfflineException {
- try {
- return client.createBatchScanner(NAME, Authorizations.EMPTY, queryThreads);
- } catch (TableNotFoundException e) {
- throw new AssertionError(NAME + " should exist, but doesn't.");
- } catch (TableOfflineException e) {
- throw new ReplicationTableOfflineException(e);
- }
- }
-
- public static boolean isOnline(AccumuloClient client) {
- return ((ClientContext) client).getTableState(ID) == TableState.ONLINE;
- }
-
- public static void setOnline(AccumuloClient client)
- throws AccumuloSecurityException, AccumuloException {
- try {
- log.info("Bringing replication table online");
- client.tableOperations().online(NAME, true);
- } catch (TableNotFoundException e) {
- throw new AssertionError(NAME + " should exist, but doesn't.");
- }
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTableOfflineException.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTableOfflineException.java
deleted file mode 100644
index 008423f..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTableOfflineException.java
+++ /dev/null
@@ -1,32 +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.replication;
-
-import org.apache.accumulo.core.client.TableOfflineException;
-
-@Deprecated
-public class ReplicationTableOfflineException extends Exception {
-
- private static final long serialVersionUID = 1L;
-
- public ReplicationTableOfflineException(TableOfflineException e) {
- super(e);
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
deleted file mode 100644
index a40c867..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
+++ /dev/null
@@ -1,198 +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.replication;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * Container for where some work needs to be replicated
- */
-@Deprecated
-public class ReplicationTarget implements Writable {
-
- private String peerName;
- private String remoteIdentifier;
- private TableId sourceTableId;
-
- public ReplicationTarget() {}
-
- public ReplicationTarget(String peerName, String remoteIdentifier, TableId sourceTableId) {
- this.peerName = peerName;
- this.remoteIdentifier = remoteIdentifier;
- this.sourceTableId = sourceTableId;
- }
-
- public String getPeerName() {
- return peerName;
- }
-
- public void setPeerName(String peerName) {
- this.peerName = peerName;
- }
-
- public String getRemoteIdentifier() {
- return remoteIdentifier;
- }
-
- public void setRemoteIdentifier(String remoteIdentifier) {
- this.remoteIdentifier = remoteIdentifier;
- }
-
- public TableId getSourceTableId() {
- return sourceTableId;
- }
-
- public void setSourceTableId(TableId sourceTableId) {
- this.sourceTableId = sourceTableId;
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
- if (peerName == null) {
- out.writeBoolean(false);
- } else {
- out.writeBoolean(true);
- WritableUtils.writeString(out, peerName);
- }
-
- if (remoteIdentifier == null) {
- out.writeBoolean(false);
- } else {
- out.writeBoolean(true);
- WritableUtils.writeString(out, remoteIdentifier);
- }
-
- if (sourceTableId == null) {
- out.writeBoolean(false);
- } else {
- out.writeBoolean(true);
- WritableUtils.writeString(out, sourceTableId.canonical());
- }
- }
-
- @Override
- public void readFields(DataInput in) throws IOException {
- if (in.readBoolean()) {
- this.peerName = WritableUtils.readString(in);
- }
- if (in.readBoolean()) {
- this.remoteIdentifier = WritableUtils.readString(in);
- }
- if (in.readBoolean()) {
- this.sourceTableId = TableId.of(WritableUtils.readString(in));
- }
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder(64);
- sb.append("Remote Name: ").append(peerName).append(" Remote identifier: ")
- .append(remoteIdentifier).append(" Source Table ID: ").append(sourceTableId);
- return sb.toString();
- }
-
- @Override
- public int hashCode() {
- return peerName.hashCode() ^ remoteIdentifier.hashCode() ^ sourceTableId.hashCode();
- }
-
- @Override
- public boolean equals(Object o) {
- if (o instanceof ReplicationTarget) {
- ReplicationTarget other = (ReplicationTarget) o;
-
- return peerName.equals(other.peerName) && remoteIdentifier.equals(other.remoteIdentifier)
- && sourceTableId.equals(other.sourceTableId);
- }
-
- return false;
- }
-
- /**
- * Deserialize a ReplicationTarget
- *
- * @param t Serialized copy
- * @return the deserialized version
- */
- public static ReplicationTarget from(Text t) {
- ReplicationTarget target = new ReplicationTarget();
- DataInputBuffer buffer = new DataInputBuffer();
- buffer.reset(t.getBytes(), t.getLength());
-
- try {
- target.readFields(buffer);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
-
- return target;
- }
-
- /**
- * Deserialize a ReplicationTarget
- *
- * @param s Serialized copy
- * @return the deserialized version
- */
- public static ReplicationTarget from(String s) {
- ReplicationTarget target = new ReplicationTarget();
- DataInputBuffer buffer = new DataInputBuffer();
- buffer.reset(s.getBytes(UTF_8), s.length());
-
- try {
- target.readFields(buffer);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
-
- return target;
- }
-
- /**
- * Convenience method to serialize a ReplicationTarget to {@link Text} using the {@link Writable}
- * methods without caring about performance penalties due to excessive object creation
- *
- * @return The serialized representation of the object
- */
- public Text toText() {
- DataOutputBuffer buffer = new DataOutputBuffer();
-
- try {
- this.write(buffer);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
-
- Text t = new Text();
- // Throw it in a text for the mutation
- t.set(buffer.getData(), 0, buffer.getLength());
- return t;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java b/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java
index bb19d5b..0aaba98 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java
@@ -21,9 +21,11 @@
import static java.util.Objects.requireNonNull;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
+import java.util.Objects;
import java.util.Properties;
import javax.security.auth.callback.CallbackHandler;
@@ -166,7 +168,7 @@
protected void updatePrincipalFromUgi() {
// Ensure we're using Kerberos auth for Hadoop UGI
if (!UserGroupInformation.isSecurityEnabled()) {
- throw new RuntimeException("Cannot use SASL if Hadoop security is not enabled");
+ throw new IllegalStateException("Cannot use SASL if Hadoop security is not enabled");
}
// Get the current user
@@ -174,13 +176,13 @@
try {
currentUser = UserGroupInformation.getCurrentUser();
} catch (IOException e) {
- throw new RuntimeException("Failed to get current user", e);
+ throw new UncheckedIOException("Failed to get current user", e);
}
// The full name is our principal
this.principal = currentUser.getUserName();
if (this.principal == null) {
- throw new RuntimeException("Got null username from " + currentUser);
+ throw new IllegalStateException("Got null username from " + currentUser);
}
}
@@ -260,11 +262,7 @@
if (!mechanism.equals(other.mechanism)) {
return false;
}
- if (callbackHandler == null) {
- if (other.callbackHandler != null) {
- return false;
- }
- } else if (!callbackHandler.equals(other.callbackHandler)) {
+ if (!Objects.equals(callbackHandler, other.callbackHandler)) {
return false;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java b/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java
index 1e1f4a3..88f9ade 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/TTimeoutTransport.java
@@ -28,7 +28,6 @@
import java.net.SocketAddress;
import java.nio.channels.spi.SelectorProvider;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.hadoop.net.NetUtils;
import org.apache.thrift.transport.TIOStreamTransport;
import org.apache.thrift.transport.TTransport;
@@ -36,6 +35,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.net.HostAndPort;
+
/**
* A utility class for setting up a {@link TTransport} with various necessary configurations for
* ideal performance in Accumulo. These configurations include:
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
index 41c4650..44c5892 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
@@ -18,18 +18,18 @@
*/
package org.apache.accumulo.core.rpc;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.InetAddress;
import java.nio.channels.ClosedByInterruptException;
-import java.security.SecureRandom;
import java.util.HashMap;
import java.util.Map;
import org.apache.accumulo.core.clientImpl.ClientContext;
import org.apache.accumulo.core.rpc.SaslConnectionParams.SaslMechanism;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
import org.apache.thrift.TException;
@@ -44,6 +44,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.net.HostAndPort;
+
/**
* Factory methods for creating Thrift client objects
*/
@@ -58,7 +60,6 @@
public static final String GSSAPI = "GSSAPI", DIGEST_MD5 = "DIGEST-MD5";
- private static final SecureRandom random = new SecureRandom();
private static final int RELOGIN_MAX_BACKOFF = 5000;
/**
@@ -339,7 +340,7 @@
if (loginUser == null || !loginUser.hasKerberosCredentials()) {
// We should have already checked that we're logged in and have credentials. A
// precondition-like check.
- throw new RuntimeException("Expected to find Kerberos UGI credentials, but did not");
+ throw new IllegalStateException("Expected to find Kerberos UGI credentials, but did not");
}
UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
// A Proxy user is the "effective user" (in name only), riding on top of the "real user"'s Krb
@@ -358,7 +359,7 @@
// Avoid the replay attack protection, sleep 1 to 5000ms
try {
- Thread.sleep(random.nextInt(RELOGIN_MAX_BACKOFF) + 1);
+ Thread.sleep(RANDOM.get().nextInt(RELOGIN_MAX_BACKOFF) + 1);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return;
@@ -371,7 +372,7 @@
// The inability to check is worrisome and deserves a RuntimeException instead of a propagated
// IO-like Exception.
log.warn("Failed to check (and/or perform) Kerberos client re-login", e);
- throw new RuntimeException(e);
+ throw new UncheckedIOException(e);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransport.java b/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransport.java
index 913cd19..133999c 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransport.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransport.java
@@ -19,6 +19,7 @@
package org.apache.accumulo.core.rpc;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.security.PrivilegedExceptionAction;
import java.util.concurrent.atomic.AtomicReference;
@@ -56,8 +57,10 @@
}
return null;
});
- } catch (IOException | InterruptedException e) {
- throw new RuntimeException(e);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ } catch (InterruptedException e) {
+ throw new IllegalStateException(e);
}
// Make sure the transport exception gets (re)thrown if it happened
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransportFactory.java b/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransportFactory.java
index e9eb384..707a41f 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransportFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/UGIAssumingTransportFactory.java
@@ -52,7 +52,7 @@
try {
return wrapped.getTransport(trans);
} catch (TTransportException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
});
}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java
index 5a6a912..c1dc995 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ClientServiceThriftClient.java
@@ -24,6 +24,7 @@
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.clientImpl.ClientContext;
import org.apache.accumulo.core.clientImpl.thrift.ClientService.Client;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
import org.apache.accumulo.core.util.Pair;
import org.apache.thrift.transport.TTransportException;
import org.slf4j.Logger;
@@ -40,10 +41,10 @@
}
@Override
- public Pair<String,Client> getTabletServerConnection(ClientContext context,
+ public Pair<String,Client> getThriftServerConnection(ClientContext context,
boolean preferCachedConnections) throws TTransportException {
- return getTabletServerConnection(LOG, this, context, preferCachedConnections,
- warnedAboutTServersBeingDown);
+ return getThriftServerConnection(LOG, this, context, preferCachedConnections,
+ warnedAboutTServersBeingDown, ThriftService.CLIENT);
}
@Override
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java
index 2561d60..d0076e6 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerClient.java
@@ -25,11 +25,12 @@
import org.apache.accumulo.core.clientImpl.ClientContext;
import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.thrift.TServiceClient;
import org.apache.thrift.transport.TTransportException;
import org.slf4j.Logger;
+import com.google.common.net.HostAndPort;
+
public interface ManagerClient<C extends TServiceClient> {
default C getManagerConnection(Logger log, ThriftClientTypes<C> type, ClientContext context) {
@@ -54,7 +55,7 @@
Throwable cause = tte.getCause();
if (cause != null && cause instanceof UnknownHostException) {
// do not expect to recover from this
- throw new RuntimeException(tte);
+ throw new IllegalStateException(tte);
}
log.debug("Failed to connect to manager=" + manager + ", will retry... ", tte);
return null;
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerThriftClient.java
index 21a3b1c..eb353fe 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ManagerThriftClient.java
@@ -18,8 +18,8 @@
*/
package org.apache.accumulo.core.rpc.clients;
+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.ConcurrentModificationException;
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationCoordinatorThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationCoordinatorThriftClient.java
deleted file mode 100644
index da6457d..0000000
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationCoordinatorThriftClient.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.rpc.clients;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
-
-import java.util.List;
-
-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.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.fate.zookeeper.ZooReader;
-import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator.Client;
-import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ReplicationCoordinatorThriftClient extends ThriftClientTypes<Client> {
-
- private static final Logger LOG =
- LoggerFactory.getLogger(ReplicationCoordinatorThriftClient.class);
-
- ReplicationCoordinatorThriftClient(String serviceName) {
- super(serviceName, new Client.Factory());
- }
-
- @Override
- public Client getConnection(ClientContext context) {
-
- List<String> locations = context.getManagerLocations();
-
- if (locations.isEmpty()) {
- LOG.debug("No managers for replication to instance {}", context.getInstanceName());
- return null;
- }
-
- // This is the manager thrift service, we just want the hostname, not the port
- String managerThriftService = locations.get(0);
- if (managerThriftService.endsWith(":0")) {
- LOG.warn("Manager found for {} did not have real location {}", context.getInstanceName(),
- managerThriftService);
- return null;
- }
-
- String zkPath = context.getZooKeeperRoot() + Constants.ZMANAGER_REPLICATION_COORDINATOR_ADDR;
- String replCoordinatorAddr;
-
- LOG.debug("Using ZooKeeper quorum at {} with path {} to find peer Manager information",
- context.getZooKeepers(), zkPath);
-
- // Get the coordinator port for the manager we're trying to connect to
- try {
- ZooReader reader = context.getZooReader();
- replCoordinatorAddr = new String(reader.getData(zkPath), UTF_8);
- } catch (KeeperException | InterruptedException e) {
- LOG.error("Could not fetch remote coordinator port", e);
- return null;
- }
-
- // Throw the hostname and port through HostAndPort to get some normalization
- HostAndPort coordinatorAddr = HostAndPort.fromString(replCoordinatorAddr);
-
- LOG.debug("Connecting to manager at {}", coordinatorAddr);
-
- try {
- // Manager requests can take a long time: don't ever time out
- return ThriftUtil.getClientNoTimeout(ThriftClientTypes.REPLICATION_COORDINATOR,
- coordinatorAddr, context);
- } catch (TTransportException tte) {
- LOG.debug("Failed to connect to manager coordinator service ({})", coordinatorAddr, tte);
- return null;
- }
- }
-
- @Override
- public Client getConnectionWithRetry(ClientContext context) {
- requireNonNull(context);
-
- for (int attempts = 1; attempts <= 10; attempts++) {
-
- Client result = getConnection(context);
- if (result != null) {
- return result;
- }
- LOG.debug("Could not get ReplicationCoordinator connection to {}, will retry",
- context.getInstanceName());
- try {
- Thread.sleep(attempts * 250L);
- } catch (InterruptedException e) {
- throw new RuntimeException(e);
- }
- }
-
- throw new RuntimeException(
- "Timed out trying to communicate with manager from " + context.getInstanceName());
- }
-
- @Override
- public <R> R execute(ClientContext context, Exec<R,Client> exec)
- throws AccumuloException, AccumuloSecurityException {
- Client client = null;
- for (int i = 0; i < 10; i++) {
- try {
- client = getConnectionWithRetry(context);
- return exec.execute(client);
- } catch (TTransportException tte) {
- LOG.debug("ReplicationClient coordinator request failed, retrying ... ", tte);
- try {
- Thread.sleep(100);
- } catch (InterruptedException e) {
- throw new AccumuloException(e);
- }
- } catch (ThriftSecurityException e) {
- throw new AccumuloSecurityException(e.user, e.code, e);
- } catch (Exception e) {
- throw new AccumuloException(e);
- } finally {
- if (client != null) {
- ThriftUtil.close(client, context);
- }
- }
- }
-
- throw new AccumuloException(
- "Could not connect to ReplicationCoordinator at " + context.getInstanceName());
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java
index 5f01766..4027f4b 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java
@@ -19,13 +19,14 @@
package org.apache.accumulo.core.rpc.clients;
import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
+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 static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
+import java.util.Optional;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.accumulo.core.Constants;
@@ -34,15 +35,14 @@
import org.apache.accumulo.core.clientImpl.AccumuloServerException;
import org.apache.accumulo.core.clientImpl.ClientContext;
import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
+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.rpc.ThriftUtil;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes.Exec;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes.ExecVoid;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
import org.apache.thrift.TApplicationException;
import org.apache.thrift.TException;
import org.apache.thrift.TServiceClient;
@@ -50,14 +50,16 @@
import org.apache.thrift.transport.TTransportException;
import org.slf4j.Logger;
+import com.google.common.net.HostAndPort;
+
public interface TServerClient<C extends TServiceClient> {
- Pair<String,C> getTabletServerConnection(ClientContext context, boolean preferCachedConnections)
+ Pair<String,C> getThriftServerConnection(ClientContext context, boolean preferCachedConnections)
throws TTransportException;
- default Pair<String,C> getTabletServerConnection(Logger LOG, ThriftClientTypes<C> type,
- ClientContext context, boolean preferCachedConnections, AtomicBoolean warned)
- throws TTransportException {
+ default Pair<String,C> getThriftServerConnection(Logger LOG, ThriftClientTypes<C> type,
+ ClientContext context, boolean preferCachedConnections, AtomicBoolean warned,
+ ThriftService service) throws TTransportException {
checkArgument(context != null, "context is null");
if (preferCachedConnections) {
@@ -71,29 +73,42 @@
}
final long rpcTimeout = context.getClientTimeoutInMillis();
+ final String tserverZooPath = context.getZooKeeperRoot() + Constants.ZTSERVERS;
+ final String sserverZooPath = context.getZooKeeperRoot() + Constants.ZSSERVERS;
+ final String compactorZooPath = context.getZooKeeperRoot() + Constants.ZCOMPACTORS;
final ZooCache zc = context.getZooCache();
- final List<String> tservers = new ArrayList<>();
- tservers.addAll(zc.getChildren(context.getZooKeeperRoot() + Constants.ZTSERVERS));
+ final List<String> serverPaths = new ArrayList<>();
+ zc.getChildren(tserverZooPath).forEach(tserverAddress -> {
+ serverPaths.add(tserverZooPath + "/" + tserverAddress);
+ });
+ if (type == ThriftClientTypes.CLIENT) {
+ zc.getChildren(sserverZooPath).forEach(sserverAddress -> {
+ serverPaths.add(sserverZooPath + "/" + sserverAddress);
+ });
+ zc.getChildren(compactorZooPath).forEach(compactorGroup -> {
+ zc.getChildren(compactorZooPath + "/" + compactorGroup).forEach(compactorAddress -> {
+ serverPaths.add(compactorZooPath + "/" + compactorGroup + "/" + compactorAddress);
+ });
+ });
+ }
- if (tservers.isEmpty()) {
+ if (serverPaths.isEmpty()) {
if (warned.compareAndSet(false, true)) {
- LOG.warn("There are no tablet servers: check that zookeeper and accumulo are running.");
+ LOG.warn(
+ "There are no servers serving the {} api: check that zookeeper and accumulo are running.",
+ type);
}
throw new TTransportException("There are no servers for type: " + type);
}
+ Collections.shuffle(serverPaths, RANDOM.get());
- // Try to connect to an online tserver
- Collections.shuffle(tservers);
- for (String tserver : tservers) {
- var zLocPath =
- ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + tserver);
- byte[] data = zc.getLockData(zLocPath);
- if (data != null) {
- String strData = new String(data, UTF_8);
- if (!strData.equals("manager")) {
- final HostAndPort tserverClientAddress =
- new ServerServices(strData).getAddress(Service.TSERV_CLIENT);
+ for (String serverPath : serverPaths) {
+ var zLocPath = ServiceLock.path(serverPath);
+ Optional<ServiceLockData> data = zc.getLockData(zLocPath);
+ if (data != null && data.isPresent()) {
+ HostAndPort tserverClientAddress = data.orElseThrow().getAddress(service);
+ if (tserverClientAddress != null) {
try {
TTransport transport = context.getTransportPool().getTransport(type,
tserverClientAddress, rpcTimeout, context, preferCachedConnections);
@@ -107,9 +122,10 @@
}
}
}
+
if (warned.compareAndSet(false, true)) {
LOG.warn("Failed to find an available server in the list of servers: {} for API type: {}",
- tservers, type);
+ serverPaths, type);
}
throw new TTransportException("Failed to connect to any server for API type " + type);
}
@@ -120,7 +136,7 @@
String server = null;
C client = null;
try {
- Pair<String,C> pair = getTabletServerConnection(context, true);
+ Pair<String,C> pair = getThriftServerConnection(context, true);
server = pair.getFirst();
client = pair.getSecond();
return exec.execute(client);
@@ -147,7 +163,7 @@
String server = null;
C client = null;
try {
- Pair<String,C> pair = getTabletServerConnection(context, true);
+ Pair<String,C> pair = getThriftServerConnection(context, true);
server = pair.getFirst();
client = pair.getSecond();
exec.execute(client);
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletIngestClientServiceThriftClient.java
similarity index 67%
copy from core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
copy to core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletIngestClientServiceThriftClient.java
index d347979..a34ecf7 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletIngestClientServiceThriftClient.java
@@ -18,11 +18,15 @@
*/
package org.apache.accumulo.core.rpc.clients;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Client;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService.Client;
-public class ReplicationServicerThriftClient extends ThriftClientTypes<Client> {
+/**
+ * Client side object that can be used to interact with services that support ingest operations
+ * against tablets. See TabletIngestClientService$Iface for a list of supported operations.
+ */
+public class TabletIngestClientServiceThriftClient extends ThriftClientTypes<Client> {
- ReplicationServicerThriftClient(String serviceName) {
+ public TabletIngestClientServiceThriftClient(String serviceName) {
super(serviceName, new Client.Factory());
}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletManagementClientServiceThriftClient.java
similarity index 66%
copy from core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
copy to core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletManagementClientServiceThriftClient.java
index d347979..d19a2a4 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletManagementClientServiceThriftClient.java
@@ -18,12 +18,15 @@
*/
package org.apache.accumulo.core.rpc.clients;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Client;
+import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService.Client;
-public class ReplicationServicerThriftClient extends ThriftClientTypes<Client> {
+/**
+ * Client side object that can be used to interact with services that support management operations
+ * against tablets. See TabletManagementClientService$Iface for a list of supported operations.
+ */
+public class TabletManagementClientServiceThriftClient extends ThriftClientTypes<Client> {
- ReplicationServicerThriftClient(String serviceName) {
+ public TabletManagementClientServiceThriftClient(String serviceName) {
super(serviceName, new Client.Factory());
}
-
}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletScanClientServiceThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletScanClientServiceThriftClient.java
index 55f7d53..15e9ce4 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletScanClientServiceThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletScanClientServiceThriftClient.java
@@ -18,8 +18,12 @@
*/
package org.apache.accumulo.core.rpc.clients;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService.Client;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService.Client;
+/**
+ * Client side object that can be used to interact with services that support scan operations
+ * against tablets. See TabletScanClientService$Iface for a list of supported operations.
+ */
public class TabletScanClientServiceThriftClient extends ThriftClientTypes<Client> {
TabletScanClientServiceThriftClient(String serviceName) {
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java
index 647bdec..3f2f71b 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TabletServerThriftClient.java
@@ -23,12 +23,17 @@
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client;
import org.apache.accumulo.core.util.Pair;
import org.apache.thrift.transport.TTransportException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+/**
+ * Client side object that can be used to interact with services that support operations against
+ * TabletServers. See TabletServerThriftClient$Iface for a list of supported operations.
+ */
public class TabletServerThriftClient extends ThriftClientTypes<Client>
implements TServerClient<Client> {
@@ -40,10 +45,10 @@
}
@Override
- public Pair<String,Client> getTabletServerConnection(ClientContext context,
+ public Pair<String,Client> getThriftServerConnection(ClientContext context,
boolean preferCachedConnections) throws TTransportException {
- return getTabletServerConnection(LOG, this, context, preferCachedConnections,
- warnedAboutTServersBeingDown);
+ return getThriftServerConnection(LOG, this, context, preferCachedConnections,
+ warnedAboutTServersBeingDown, ThriftService.TSERV);
}
@Override
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java
index 9559550..d7f34c4 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/ThriftClientTypes.java
@@ -18,8 +18,8 @@
*/
package org.apache.accumulo.core.rpc.clients;
+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 org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -46,18 +46,18 @@
public static final ManagerThriftClient MANAGER = new ManagerThriftClient("mgr");
- public static final ReplicationCoordinatorThriftClient REPLICATION_COORDINATOR =
- new ReplicationCoordinatorThriftClient("replCoord");
-
- public static final ReplicationServicerThriftClient REPLICATION_SERVICER =
- new ReplicationServicerThriftClient("replServ");
-
public static final TabletServerThriftClient TABLET_SERVER =
- new TabletServerThriftClient("tablet");
+ new TabletServerThriftClient("tserver");
public static final TabletScanClientServiceThriftClient TABLET_SCAN =
new TabletScanClientServiceThriftClient("scan");
+ public static final TabletIngestClientServiceThriftClient TABLET_INGEST =
+ new TabletIngestClientServiceThriftClient("ingest");
+
+ public static final TabletManagementClientServiceThriftClient TABLET_MGMT =
+ new TabletManagementClientServiceThriftClient("tablet");
+
/**
* execute method with supplied client returning object of type R
*
diff --git a/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java b/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java
index d77c5a4..a97bf5e 100644
--- a/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/sample/impl/SamplerConfigurationImpl.java
@@ -32,7 +32,7 @@
import org.apache.accumulo.core.client.sample.SamplerConfiguration;
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletscan.thrift.TSamplerConfiguration;
import org.apache.accumulo.core.util.Pair;
import org.apache.hadoop.io.Writable;
diff --git a/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java b/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java
index 1cfeebb..380a183 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java
@@ -385,4 +385,21 @@
return sb.toString();
}
+
+ /**
+ * Converts to an Accumulo Access Authorizations object.
+ *
+ * @since 3.1.0
+ */
+ public org.apache.accumulo.access.Authorizations toAccessAuthorizations() {
+ if (auths.isEmpty()) {
+ return org.apache.accumulo.access.Authorizations.of();
+ } else {
+ Set<String> auths = new HashSet<>(authsList.size());
+ for (var auth : authsList) {
+ auths.add(new String(auth, UTF_8));
+ }
+ return org.apache.accumulo.access.Authorizations.of(auths);
+ }
+ }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java b/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
index 94772fe..bc49bac 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
@@ -27,7 +27,10 @@
import java.util.Comparator;
import java.util.List;
import java.util.TreeSet;
+import java.util.function.Supplier;
+import org.apache.accumulo.access.AccessExpression;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
import org.apache.accumulo.core.data.ArrayByteSequence;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.util.BadArgumentException;
@@ -35,6 +38,8 @@
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableComparator;
+import com.google.common.base.Suppliers;
+
/**
* Validate the column visibility is a valid expression and set the visibility for a Mutation. See
* {@link ColumnVisibility#ColumnVisibility(byte[])} for the definition of an expression.
@@ -76,8 +81,10 @@
*/
public class ColumnVisibility {
- Node node = null;
- private byte[] expression;
+ // This functionality is deprecated so its setup as a supplier so it is only computed if the
+ // deprecated functionality is called.
+ private final Supplier<Node> nodeSupplier;
+ private final byte[] expression;
/**
* Accessor for the underlying byte string.
@@ -91,7 +98,8 @@
/**
* The node types in a parse tree for a visibility expression.
*/
- public static enum NodeType {
+ @Deprecated(since = "3.1.0")
+ public enum NodeType {
EMPTY, TERM, OR, AND,
}
@@ -103,6 +111,7 @@
/**
* A node in the parse tree for a visibility expression.
*/
+ @Deprecated(since = "3.1.0")
public static class Node {
/**
* An empty list of nodes.
@@ -151,7 +160,7 @@
public ByteSequence getTerm(byte[] expression) {
if (type != NodeType.TERM) {
- throw new RuntimeException();
+ throw new IllegalStateException();
}
if (expression[start] == '"') {
@@ -169,6 +178,7 @@
* A node comparator. Nodes sort according to node type, terms sort lexicographically. AND and OR
* nodes sort by number of children, or if the same by corresponding children.
*/
+ @Deprecated(since = "3.1.0")
public static class NodeComparator implements Comparator<Node>, Serializable {
private static final long serialVersionUID = 1L;
@@ -216,6 +226,7 @@
* Convenience method that delegates to normalize with a new NodeComparator constructed using the
* supplied expression.
*/
+ @Deprecated(since = "3.1.0")
public static Node normalize(Node root, byte[] expression) {
return normalize(root, expression, new NodeComparator(expression));
}
@@ -228,6 +239,7 @@
* 3) dedupes labels (`a&b&a` becomes `a&b`)
*/
// @formatter:on
+ @Deprecated(since = "3.1.0")
public static Node normalize(Node root, byte[] expression, NodeComparator comparator) {
if (root.type != NodeType.TERM) {
TreeSet<Node> rolledUp = new TreeSet<>(comparator);
@@ -256,6 +268,7 @@
* Walks an expression's AST and appends a string representation to a supplied StringBuilder. This
* method adds parens where necessary.
*/
+ @Deprecated(since = "3.1.0")
public static void stringify(Node root, byte[] expression, StringBuilder out) {
if (root.type == NodeType.TERM) {
out.append(new String(expression, root.start, root.end - root.start, UTF_8));
@@ -282,13 +295,12 @@
*
* @return normalized expression in byte[] form
*/
+ @Deprecated(since = "3.1.0")
public byte[] flatten() {
- Node normRoot = normalize(node, expression);
- StringBuilder builder = new StringBuilder(expression.length);
- stringify(normRoot, expression, builder);
- return builder.toString().getBytes(UTF_8);
+ return AccessExpression.of(expression, true).getExpression().getBytes(UTF_8);
}
+ @Deprecated
private static class ColumnVisibilityParser {
private int index = 0;
private int parens = 0;
@@ -455,16 +467,17 @@
}
}
- private void validate(byte[] expression) {
+ private Node createNodeTree(byte[] expression) {
if (expression != null && expression.length > 0) {
ColumnVisibilityParser p = new ColumnVisibilityParser();
- node = p.parse(expression);
+ return p.parse(expression);
} else {
- node = EMPTY_NODE;
+ return EMPTY_NODE;
}
- this.expression = expression;
}
+ private static final byte[] EMPTY_BYTES = new byte[0];
+
/**
* Creates an empty visibility. Normally, elements with empty visibility can be seen by everyone.
* Though, one could change this behavior with filters.
@@ -472,7 +485,8 @@
* @see #ColumnVisibility(String)
*/
public ColumnVisibility() {
- this(new byte[] {});
+ expression = EMPTY_BYTES;
+ nodeSupplier = Suppliers.memoize(() -> createNodeTree(expression));
}
/**
@@ -496,13 +510,34 @@
}
/**
- * Creates a column visibility for a Mutation from a string already encoded in UTF-8 bytes.
+ * Creates a column visibility for a Mutation from bytes already encoded in UTF-8.
*
* @param expression visibility expression, encoded as UTF-8 bytes
* @see #ColumnVisibility(String)
*/
public ColumnVisibility(byte[] expression) {
- validate(expression);
+ this.expression = expression;
+ try {
+ AccessExpression.validate(this.expression);
+ } catch (InvalidAccessExpressionException e) {
+ // This is thrown for compatability with the exception this class used to throw when it parsed
+ // exceptions itself.
+ throw new BadArgumentException(e);
+ }
+ nodeSupplier = Suppliers.memoize(() -> createNodeTree(this.expression));
+ }
+
+ /**
+ * Creates a column visibility for a Mutation from an AccessExpression.
+ *
+ * @param expression visibility expression, encoded as UTF-8 bytes
+ * @see #ColumnVisibility(String)
+ * @since 3.1.0
+ */
+ public ColumnVisibility(AccessExpression expression) {
+ // AccessExpression is a validated immutable object, so no need to re validate
+ this.expression = expression.getExpression().getBytes(UTF_8);
+ nodeSupplier = Suppliers.memoize(() -> createNodeTree(this.expression));
}
@Override
@@ -542,8 +577,9 @@
*
* @return parse tree node
*/
+ @Deprecated(since = "3.1.0")
public Node getParseTree() {
- return node;
+ return nodeSupplier.get();
}
/**
@@ -564,9 +600,11 @@
*
* @param term term to quote
* @return quoted term (unquoted if unnecessary)
+ * @deprecated use {@link AccessExpression#quote(String)}
*/
+ @Deprecated(since = "3.1.0")
public static String quote(String term) {
- return new String(quote(term.getBytes(UTF_8)), UTF_8);
+ return AccessExpression.quote(term);
}
/**
@@ -576,21 +614,10 @@
* @param term term to quote, encoded as UTF-8 bytes
* @return quoted term (unquoted if unnecessary), encoded as UTF-8 bytes
* @see #quote(String)
+ * @deprecated use {@link AccessExpression#quote(byte[])}
*/
+ @Deprecated(since = "3.1.0")
public static byte[] quote(byte[] term) {
- boolean needsQuote = false;
-
- for (byte b : term) {
- if (!Authorizations.isValidAuthChar(b)) {
- needsQuote = true;
- break;
- }
- }
-
- if (!needsQuote) {
- return term;
- }
-
- return VisibilityEvaluator.escape(term, true);
+ return AccessExpression.quote(term);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java b/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java
index dd07d64..3c97c26 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java
@@ -18,92 +18,18 @@
*/
package org.apache.accumulo.core.security;
-import java.util.ArrayList;
-
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
import org.apache.accumulo.core.data.ArrayByteSequence;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.security.ColumnVisibility.Node;
/**
* A class which evaluates visibility expressions against a set of authorizations.
+ *
+ * @deprecated since 3.1.0 Use Accumulo Access library instead
*/
+@Deprecated(since = "3.1.0")
public class VisibilityEvaluator {
- private AuthorizationContainer auths;
-
- /**
- * Authorizations in column visibility expression are in escaped form. Column visibility parsing
- * does not unescape. This class wraps an AuthorizationContainer and unescapes auths before
- * checking the wrapped container.
- */
- private static class UnescapingAuthorizationContainer implements AuthorizationContainer {
-
- private AuthorizationContainer wrapped;
-
- UnescapingAuthorizationContainer(AuthorizationContainer wrapee) {
- this.wrapped = wrapee;
- }
-
- @Override
- public boolean contains(ByteSequence auth) {
- return wrapped.contains(unescape(auth));
- }
- }
-
- static ByteSequence unescape(ByteSequence auth) {
- int escapeCharCount = 0;
- for (int i = 0; i < auth.length(); i++) {
- byte b = auth.byteAt(i);
- if (b == '"' || b == '\\') {
- escapeCharCount++;
- }
- }
-
- if (escapeCharCount > 0) {
- if (escapeCharCount % 2 == 1) {
- throw new IllegalArgumentException("Illegal escape sequence in auth : " + auth);
- }
-
- byte[] unescapedCopy = new byte[auth.length() - escapeCharCount / 2];
- int pos = 0;
- for (int i = 0; i < auth.length(); i++) {
- byte b = auth.byteAt(i);
- if (b == '\\') {
- i++;
- b = auth.byteAt(i);
- if (b != '"' && b != '\\') {
- throw new IllegalArgumentException("Illegal escape sequence in auth : " + auth);
- }
- } else if (b == '"') {
- // should only see quote after a slash
- throw new IllegalArgumentException("Illegal escape sequence in auth : " + auth);
- }
-
- unescapedCopy[pos++] = b;
- }
-
- return new ArrayByteSequence(unescapedCopy);
- } else {
- return auth;
- }
- }
-
- /**
- * Creates a new {@link Authorizations} object with escaped forms of the authorizations in the
- * given object.
- *
- * @param auths original authorizations
- * @return authorizations object with escaped authorization strings
- * @see #escape(byte[], boolean)
- */
- static Authorizations escape(Authorizations auths) {
- ArrayList<byte[]> retAuths = new ArrayList<>(auths.getAuthorizations().size());
-
- for (byte[] auth : auths.getAuthorizations()) {
- retAuths.add(escape(auth, false));
- }
-
- return new Authorizations(retAuths);
- }
+ private final AccessEvaluator accessEvaluator;
/**
* Properly escapes an authorization string. The string can be quoted if desired.
@@ -147,7 +73,9 @@
* @since 1.7.0
*/
public VisibilityEvaluator(AuthorizationContainer authsContainer) {
- this.auths = new UnescapingAuthorizationContainer(authsContainer);
+ // TODO need to look into efficiency and correctness of this
+ this.accessEvaluator =
+ AccessEvaluator.of(auth -> authsContainer.contains(new ArrayByteSequence(auth)));
}
/**
@@ -157,7 +85,7 @@
* @param authorizations authorizations object
*/
public VisibilityEvaluator(Authorizations authorizations) {
- this.auths = escape(authorizations);
+ this.accessEvaluator = AccessEvaluator.of(authorizations.toAccessAuthorizations());
}
/**
@@ -171,42 +99,12 @@
* subexpression is of an unknown type
*/
public boolean evaluate(ColumnVisibility visibility) throws VisibilityParseException {
- // The VisibilityEvaluator computes a trie from the given Authorizations, that ColumnVisibility
- // expressions can be evaluated against.
- return evaluate(visibility.getExpression(), visibility.getParseTree());
- }
-
- private final boolean evaluate(final byte[] expression, final Node root)
- throws VisibilityParseException {
- if (expression.length == 0) {
- return true;
- }
- switch (root.type) {
- case TERM:
- return auths.contains(root.getTerm(expression));
- case AND:
- if (root.children == null || root.children.size() < 2) {
- throw new VisibilityParseException("AND has less than 2 children", expression,
- root.start);
- }
- for (Node child : root.children) {
- if (!evaluate(expression, child)) {
- return false;
- }
- }
- return true;
- case OR:
- if (root.children == null || root.children.size() < 2) {
- throw new VisibilityParseException("OR has less than 2 children", expression, root.start);
- }
- for (Node child : root.children) {
- if (evaluate(expression, child)) {
- return true;
- }
- }
- return false;
- default:
- throw new VisibilityParseException("No such node type", expression, root.start);
+ try {
+ return accessEvaluator.canAccess(visibility.getExpression());
+ } catch (InvalidAccessExpressionException e) {
+ // This is thrown for compatability with the exception this class used to evaluate expressions
+ // itself.
+ throw new VisibilityParseException(e);
}
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java b/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java
index 297e575..2d01c2c 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java
@@ -22,6 +22,8 @@
import java.text.ParseException;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
+
/**
* An exception thrown when a visibility string cannot be parsed.
*/
@@ -41,6 +43,15 @@
this.visibility = new String(visibility, UTF_8);
}
+ /**
+ * @since 3.1.0
+ */
+ public VisibilityParseException(InvalidAccessExpressionException e) {
+ // TODO need to look at output for this
+ super(e.getDescription(), e.getIndex());
+ this.visibility = e.getPattern();
+ }
+
@Override
public String getMessage() {
return super.getMessage() + " in string '" + visibility + "' at position "
diff --git a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java
index 5f3e151..9a38059 100644
--- a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java
+++ b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java
@@ -32,8 +32,7 @@
* Historically, Accumulo client code that used Connector had no control over these singletons. The
* new AccumuloClient API that replaces Connector is closeable. When all AccumuloClients are closed
* then resources used by the singletons are released. This class coordinates releasing those
- * resources. For compatibility purposes this class will not release resources when the user has
- * created Connectors.
+ * resources.
*
* <p>
* This class is intermediate solution to resource management. Ideally there would be no static
@@ -62,11 +61,6 @@
*/
SERVER,
/**
- * In this mode singletons are never disabled unless the mode is set back to CLIENT. The user
- * can do this by using util.CleanUp (an old API created for users).
- */
- CONNECTOR,
- /**
* In this mode singletons are permanently disabled and entering this mode prevents
* transitioning to other modes.
*/
@@ -77,7 +71,6 @@
private static long reservations;
private static Mode mode;
private static boolean enabled;
- private static boolean transitionedFromClientToConnector;
private static List<SingletonService> services;
@VisibleForTesting
@@ -85,7 +78,6 @@
reservations = 0;
mode = Mode.CLIENT;
enabled = true;
- transitionedFromClientToConnector = false;
services = new ArrayList<>();
}
@@ -160,17 +152,6 @@
if (SingletonManager.mode == Mode.CLOSED) {
throw new IllegalStateException("Cannot leave closed mode once entered");
}
- if (SingletonManager.mode == Mode.CLIENT && mode == Mode.CONNECTOR) {
- if (transitionedFromClientToConnector) {
- throw new IllegalStateException("Can only transition from " + Mode.CLIENT + " to "
- + Mode.CONNECTOR + " once. This error indicates that "
- + "org.apache.accumulo.core.util.CleanUp.shutdownNow() was called and then later a "
- + "Connector was created. Connectors can not be created after CleanUp.shutdownNow()"
- + " is called.");
- }
-
- transitionedFromClientToConnector = true;
- }
/*
* Always allow transition to closed and only allow transition to client/connector when the
@@ -198,10 +179,9 @@
}
} else {
// if we're in a disabled state AND
- // the mode is CONNECTOR or SERVER or if there are active clients,
+ // the mode is SERVER or if there are active clients,
// then enable everything
- if (mode == Mode.CONNECTOR || mode == Mode.SERVER
- || (mode == Mode.CLIENT && reservations > 0)) {
+ if (mode == Mode.SERVER || (mode == Mode.CLIENT && reservations > 0)) {
services.forEach(SingletonManager::enable);
enabled = true;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
index 0b89e5d..2fd703d 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
@@ -19,10 +19,10 @@
package org.apache.accumulo.core.spi.balancer;
import static java.util.concurrent.TimeUnit.HOURS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import java.net.InetAddress;
import java.net.UnknownHostException;
-import java.security.SecureRandom;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
@@ -62,9 +62,8 @@
import org.slf4j.LoggerFactory;
import org.slf4j.event.Level;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
@@ -96,7 +95,6 @@
*/
public class HostRegexTableLoadBalancer extends TableLoadBalancer {
- private static final SecureRandom random = new SecureRandom();
private static final String PROP_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey();
private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
@@ -267,7 +265,7 @@
}
private void checkTableConfig(TableId tableId) {
- Map<String,String> tableRegexes = tablesRegExCache.getUnchecked(tableId).get();
+ Map<String,String> tableRegexes = tablesRegExCache.get(tableId).get();
if (!hrtlbConf.get().regexes.equals(tableRegexes)) {
LoggerFactory.getLogger(HostRegexTableLoadBalancer.class).warn(
@@ -327,13 +325,8 @@
this.hrtlbConf = balancerEnvironment.getConfiguration().getDerived(HrtlbConf::new);
tablesRegExCache =
- CacheBuilder.newBuilder().expireAfterAccess(1, HOURS).build(new CacheLoader<>() {
- @Override
- public Supplier<Map<String,String>> load(TableId key) {
- return balancerEnvironment.getConfiguration(key)
- .getDerived(HostRegexTableLoadBalancer::getRegexes);
- }
- });
+ Caffeine.newBuilder().expireAfterAccess(1, HOURS).build(key -> balancerEnvironment
+ .getConfiguration(key).getDerived(HostRegexTableLoadBalancer::getRegexes));
LOG.info("{}", this);
}
@@ -427,7 +420,7 @@
String poolName = getPoolNameForTable(table);
SortedMap<TabletServerId,TServerStatus> currentView = currentGrouped.get(poolName);
if (currentView != null) {
- int skip = random.nextInt(currentView.size());
+ int skip = RANDOM.get().nextInt(currentView.size());
Iterator<TabletServerId> iter = currentView.keySet().iterator();
for (int i = 0; i < skip; i++) {
iter.next();
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java
index b593a10..d42b7d2 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java
@@ -57,12 +57,6 @@
* if possible, but otherwise assignments are made in a random fashion across all available tablet
* servers.
*
- * <p>
- * This balancer replaces the deprecated
- * org.apache.accumulo.server.master.balancer.DefaultLoadBalancer which will be removed in a future
- * release. This balancer has the same functionality but uses the stable SPI which does not expose
- * internal types on public methods.
- *
* @since 2.1.0
*/
public class SimpleLoadBalancer implements TabletBalancer {
@@ -337,8 +331,10 @@
TabletId mostRecentlySplit = null;
long splitTime = 0;
for (Entry<TabletId,TabletStatistics> entry : extents.entrySet()) {
- if (entry.getValue().getSplitCreationTime() >= splitTime) {
- splitTime = entry.getValue().getSplitCreationTime();
+ @SuppressWarnings("deprecation")
+ long splitCreationTime = entry.getValue().getSplitCreationTime();
+ if (splitCreationTime >= splitTime) {
+ splitTime = splitCreationTime;
mostRecentlySplit = entry.getKey();
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
index 99ac7a6..793f00c 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
@@ -28,6 +28,7 @@
long getNumEntries();
+ @Deprecated(since = "3.1")
long getSplitCreationTime();
double getIngestRate();
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCache.java b/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCache.java
index d900149..9ebe3fc 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCache.java
@@ -111,5 +111,12 @@
* @return the number of lookups
*/
long requestCount();
+
+ /**
+ * @return The number of entries evicted from the cache.
+ *
+ * @since 3.1.0
+ */
+ long evictionCount();
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCacheManager.java b/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCacheManager.java
index 9af0b6d..7885fbc 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCacheManager.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/cache/BlockCacheManager.java
@@ -21,9 +21,6 @@
import java.util.HashMap;
import java.util.Map;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
-
/**
* @since 2.0.0
* @see org.apache.accumulo.core.spi
@@ -142,38 +139,4 @@
*/
protected abstract BlockCache createCache(Configuration conf, CacheType type);
- /**
- * A convenience method that returns a string of the from
- * {@code tserver.cache.config.<prefix>.default.} this method is useful for configuring a cache
- * manager.
- *
- * @param prefix A unique identifier that corresponds to a particular BlockCacheManager
- * implementation.
- * @see Configuration#getProperties(String, CacheType)
- * @deprecated since 2.1.0 because this method does not support scan servers, only tservers. Use
- * {@link Configuration#getProperties(String, CacheType)} instead.
- */
- @Deprecated(since = "2.1.0")
- public static String getFullyQualifiedPropertyPrefix(String prefix) {
- return BlockCacheConfiguration.getFullyQualifiedPropertyPrefix(Property.TSERV_PREFIX, prefix);
- }
-
- /**
- * A convenience method that returns a string of the from
- * {@code tserver.cache.config.<prefix>.<type>.} this method is useful for configuring a cache
- * manager.
- *
- * @param prefix A unique identifier that corresponds to a particular BlockCacheManager
- * implementation.
- * @see Configuration#getProperties(String, CacheType)
- *
- * @deprecated since 2.1.0 because this method does not support scan servers, only tservers. Use
- * {@link Configuration#getProperties(String, CacheType)} instead.
- */
- @Deprecated(since = "2.1.0")
- public static String getFullyQualifiedPropertyPrefix(String prefix, CacheType type) {
- return BlockCacheConfiguration.getFullyQualifiedPropertyPrefix(Property.TSERV_PREFIX, prefix,
- type);
- }
-
}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/common/ContextClassLoaderFactory.java b/core/src/main/java/org/apache/accumulo/core/spi/common/ContextClassLoaderFactory.java
index 3d9c186..c0e1863 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/common/ContextClassLoaderFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/common/ContextClassLoaderFactory.java
@@ -55,16 +55,17 @@
default void init(ContextClassLoaderEnvironment env) {}
/**
- * Get the class loader for the given contextName. Callers should not cache the ClassLoader result
- * as it may change if/when the ClassLoader reloads. Implementations should throw a
- * RuntimeException of some type (such as IllegalArgumentException) if the provided contextName is
- * not supported or fails to be constructed.
+ * Get the class loader for the given context. Callers should not cache the ClassLoader result as
+ * it may change if/when the ClassLoader reloads. Implementations should throw a RuntimeException
+ * of some type (such as IllegalArgumentException) if the provided context is not supported or
+ * fails to be constructed.
*
- * @param contextName the name of the context that represents a class loader that is managed by
- * this factory. Currently, Accumulo will only call this method for non-null and non-empty
+ * @param context the name of the context that represents a class loader that is managed by this
+ * factory. Currently, Accumulo will only call this method for non-null and non-empty
* context. For empty or null context, Accumulo will use the system classloader without
* consulting this plugin.
- * @return the class loader for the given contextName
+ * @return the class loader for the given context
*/
- ClassLoader getClassLoader(String contextName);
+ ClassLoader getClassLoader(String context);
+
}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java
index 996ca88..32f0dea 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java
@@ -18,6 +18,7 @@
*/
package org.apache.accumulo.core.spi.compaction;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
import org.apache.accumulo.core.client.admin.compaction.CompactionSelector;
/**
@@ -30,16 +31,20 @@
*/
SYSTEM,
/**
- * Set of files selected by a {@link CompactionSelector} or CompactionStrategy configured for a
- * table.
+ * Set of files selected by a {@link CompactionSelector} configured for a table.
+ *
+ * @deprecated since 3.1. Use of selector compactions should be replaced with user compactions
+ * initiated via
+ * {@link org.apache.accumulo.core.client.admin.TableOperations#compact(String, CompactionConfig)}.
+ * Everything that can be done with selector compactions can also be done with user
+ * compactions. User compactions offer more control over when compactions run, the
+ * range of data compacted, and the ability to cancel. Selector compactions offer none
+ * of these features and were deprecated in favor of only offering user compactions.
*/
+ @Deprecated(since = "3.1")
SELECTOR,
/**
* A user initiated a one time compaction using an Accumulo client.
*/
- USER,
- /**
- * A compaction executed prior to merging tablets.
- */
- CHOP
+ USER
}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionPlanner.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionPlanner.java
index 4b933fc..222ee65 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionPlanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionPlanner.java
@@ -45,11 +45,10 @@
/**
* @return The configured options. For example if the system properties
- * {@code tserver.compaction.major.service.s1.planner.opts.p1=abc} and
- * {@code tserver.compaction.major.service.s1.planner.opts.p9=123} were set, then this
- * map would contain {@code p1=abc} and {@code p9=123}. In this example {@code s1} is
- * the identifier for the compaction service. Each compaction service has a single
- * planner.
+ * {@code compaction.service.s1.planner.opts.p1=abc} and
+ * {@code compaction.service.s1.planner.opts.p9=123} were set, then this map would
+ * contain {@code p1=abc} and {@code p9=123}. In this example {@code s1} is the
+ * identifier for the compaction service. Each compaction service has a single planner.
*/
Map<String,String> getOptions();
@@ -57,8 +56,7 @@
* @return For a given key from the map returned by {@link #getOptions()} determines the fully
* qualified tablet property for that key. For example if a planner was being
* initialized for compaction service {@code CS9} and this method were passed
- * {@code prop1} then it would return
- * {@code tserver.compaction.major.service.CS9.planner.opts.prop1}.
+ * {@code prop1} then it would return {@code compaction.service.CS9.planner.opts.prop1}.
*/
String getFullyQualifiedOption(String key);
@@ -141,27 +139,9 @@
* the candidates will contain the files it did not compact and the results of any previous
* compactions it scheduled. The planner must eventually compact all of the files in the candidate
* set down to a single file. The compaction service will keep calling the planner until it does.
- * <li>CompactionKind.CHOP. The planner is required to eventually compact all candidates. One
- * major difference with USER compactions is this kind is not required to compact all files to a
- * single file. It is ok to return a compaction plan that compacts a subset of the candidates.
- * When the planner compacts a subset, it will eventually be called later. When it is called later
- * the candidates will contain the files it did not compact.
* </ul>
*
* <p>
- * For a chop compaction assume the following happens.
- * <ol>
- * <li>The candidate set passed to makePlan contains the files {@code [F1,F2,F3,F4]} and kind is
- * CHOP
- * <li>The planner returns a job to compact files {@code [F1,F2]} on executor E1
- * <li>The compaction runs compacting {@code [F1,F2]} into file {@code [F5]}
- * </ol>
- *
- * <p>
- * For the case above, eventually the planner will called again with a candidate set of
- * {@code [F3,F4]} and it must eventually compact those two files.
- *
- * <p>
* For a user and selector compaction assume the same thing happens, it will result in a slightly
* different outcome.
* <ol>
@@ -173,9 +153,7 @@
*
* <p>
* For the case above, eventually the planner will called again with a candidate set of
- * {@code [F3,F4,F5]} and it must eventually compact those three files to one. The difference with
- * CHOP compactions is that the result of intermediate compactions are included in the candidate
- * set.
+ * {@code [F3,F4,F5]} and it must eventually compact those three files to one.
*
* <p>
* When a planner returns a compactions plan, task will be queued on executors. Previously queued
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
index 73952b7..ee07bc2 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
@@ -18,9 +18,13 @@
*/
package org.apache.accumulo.core.spi.compaction;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
+
+import java.lang.reflect.Field;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
@@ -28,17 +32,22 @@
import java.util.List;
import java.util.Objects;
import java.util.Set;
+import java.util.stream.Collectors;
import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.spi.common.ServiceEnvironment;
import org.apache.accumulo.core.util.compaction.CompactionJobPrioritizer;
+import org.apache.accumulo.core.util.compaction.DeprecatedCompactionKind;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.Preconditions;
-import com.google.gson.Gson;
+import com.google.gson.JsonArray;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParseException;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
@@ -51,8 +60,8 @@
* compaction service you are configuring.
*
* <ul>
- * <li>{@code tserver.compaction.major.service.<service>.opts.executors} This is a json array of
- * objects where each object has the fields:
+ * <li>{@code compaction.service.<service>.opts.executors} This is a json array of objects where
+ * each object has the fields:
* <table>
* <caption>Default Compaction Planner Executor options</caption>
* <tr>
@@ -83,16 +92,19 @@
* </tr>
* </table>
* <br>
- * The maxSize field determines the maximum size of compaction that will run on an executor. The
- * maxSize field can have a suffix of K,M,G for kilobytes, megabytes, or gigabytes and represents
- * the sum of the input files for a given compaction. One executor can have no max size and it will
- * run everything that is too large for the other executors. If all executors have a max size, then
- * system compactions will only run for compactions smaller than the largest max size. User, chop,
- * and selector compactions will always run, even if there is no executor for their size. These
- * compactions will run on the executor with the largest max size. The following example value for
- * this property will create 3 threads to run compactions of files whose file size sum is less than
- * 100M, 3 threads to run compactions of files whose file size sum is less than 500M, and run all
- * other compactions on Compactors configured to run compactions for Queue1:
+ * Note: The "executors" option has been deprecated in 3.1 and will be removed in a future release.
+ * This example uses the new `compaction.service` prefix. The property prefix
+ * "tserver.compaction.major.service" has also been deprecated in 3.1 and will be removed in a
+ * future release. The maxSize field determines the maximum size of compaction that will run on an
+ * executor. The maxSize field can have a suffix of K,M,G for kilobytes, megabytes, or gigabytes and
+ * represents the sum of the input files for a given compaction. One executor can have no max size
+ * and it will run everything that is too large for the other executors. If all executors have a max
+ * size, then system compactions will only run for compactions smaller than the largest max size.
+ * User, chop, and selector compactions will always run, even if there is no executor for their
+ * size. These compactions will run on the executor with the largest max size. The following example
+ * value for this property will create 3 threads to run compactions of files whose file size sum is
+ * less than 100M, 3 threads to run compactions of files whose file size sum is less than 500M, and
+ * run all other compactions on Compactors configured to run compactions for Queue1:
*
* <pre>
* {@code
@@ -105,8 +117,28 @@
*
* Note that the use of 'external' requires that the CompactionCoordinator and at least one
* Compactor for Queue1 is running.
- * <li>{@code tserver.compaction.major.service.<service>.opts.maxOpen} This determines the maximum
- * number of files that will be included in a single compaction.
+ * <li>{@code compaction.service.<service>.opts.maxOpen} This determines the maximum number of files
+ * that will be included in a single compaction.
+ * <li>{@code compaction.service.<service>.opts.queues} This is a json array of queue objects which
+ * have the following fields:
+ * <table>
+ * <caption>Default Compaction Planner Queue options</caption>
+ * <tr>
+ * <th>Field Name</th>
+ * <th>Description</th>
+ * </tr>
+ * <tr>
+ * <td>name</td>
+ * <td>name or alias of the queue (required)</td>
+ * </tr>
+ * <tr>
+ * <td>maxSize</td>
+ * <td>threshold sum of the input files (required for all but one of the configs)</td>
+ * </tr>
+ * </table>
+ * <br>
+ * This 'queues' object is used for defining external compaction queues without needing to use the
+ * thread-based 'executors' property.
* </ul>
*
* <p>
@@ -124,13 +156,13 @@
* ratio less than 3 that results in a compaction.
*
*
- * @since 2.1.0
+ * @since 3.1.0
* @see org.apache.accumulo.core.spi.compaction
*/
public class DefaultCompactionPlanner implements CompactionPlanner {
- private static final Logger log = LoggerFactory.getLogger(DefaultCompactionPlanner.class);
+ private final static Logger log = LoggerFactory.getLogger(DefaultCompactionPlanner.class);
private static class ExecutorConfig {
String type;
@@ -140,6 +172,11 @@
String queue;
}
+ private static class QueueConfig {
+ String name;
+ String maxSize;
+ }
+
private static class Executor {
final CompactionExecutorId ceid;
final Long maxSize;
@@ -167,19 +204,23 @@
justification = "Field is written by Gson")
@Override
public void init(InitParameters params) {
+ List<Executor> tmpExec = new ArrayList<>();
+ String values;
if (params.getOptions().containsKey("executors")
&& !params.getOptions().get("executors").isBlank()) {
+ values = params.getOptions().get("executors");
- ExecutorConfig[] execConfigs =
- new Gson().fromJson(params.getOptions().get("executors"), ExecutorConfig[].class);
+ // Generate a list of fields from the desired object.
+ final List<String> execFields = Arrays.stream(ExecutorConfig.class.getDeclaredFields())
+ .map(Field::getName).collect(Collectors.toList());
- List<Executor> tmpExec = new ArrayList<>();
+ for (JsonElement element : GSON.get().fromJson(values, JsonArray.class)) {
+ validateConfig(element, execFields, ExecutorConfig.class.getName());
+ ExecutorConfig executorConfig = GSON.get().fromJson(element, ExecutorConfig.class);
- for (ExecutorConfig executorConfig : execConfigs) {
Long maxSize = executorConfig.maxSize == null ? null
: ConfigurationTypeHelper.getFixedMemoryAsBytes(executorConfig.maxSize);
-
CompactionExecutorId ceid;
// If not supplied, GSON will leave type null. Default to internal
@@ -207,143 +248,169 @@
}
tmpExec.add(new Executor(ceid, maxSize));
}
-
- Collections.sort(tmpExec, Comparator.comparing(Executor::getMaxSize,
- Comparator.nullsLast(Comparator.naturalOrder())));
-
- executors = List.copyOf(tmpExec);
-
- if (executors.stream().filter(e -> e.getMaxSize() == null).count() > 1) {
- throw new IllegalArgumentException(
- "Can only have one executor w/o a maxSize. " + params.getOptions().get("executors"));
- }
-
- // use the add method on the Set interface to check for duplicate maxSizes
- Set<Long> maxSizes = new HashSet<>();
- executors.forEach(e -> {
- if (!maxSizes.add(e.getMaxSize())) {
- throw new IllegalArgumentException(
- "Duplicate maxSize set in executors. " + params.getOptions().get("executors"));
- }
- });
- } else {
- throw new IllegalStateException("No defined executors for this planner");
}
+
+ if (params.getOptions().containsKey("queues") && !params.getOptions().get("queues").isBlank()) {
+ values = params.getOptions().get("queues");
+
+ // Generate a list of fields from the desired object.
+ final List<String> queueFields = Arrays.stream(QueueConfig.class.getDeclaredFields())
+ .map(Field::getName).collect(Collectors.toList());
+
+ for (JsonElement element : GSON.get().fromJson(values, JsonArray.class)) {
+ validateConfig(element, queueFields, QueueConfig.class.getName());
+ QueueConfig queueConfig = GSON.get().fromJson(element, QueueConfig.class);
+
+ Long maxSize = queueConfig.maxSize == null ? null
+ : ConfigurationTypeHelper.getFixedMemoryAsBytes(queueConfig.maxSize);
+
+ CompactionExecutorId ceid;
+ String queue = Objects.requireNonNull(queueConfig.name, "'name' must be specified");
+ ceid = params.getExecutorManager().getExternalExecutor(queue);
+ tmpExec.add(new Executor(ceid, maxSize));
+ }
+ }
+
+ if (tmpExec.size() < 1) {
+ throw new IllegalStateException("No defined executors or queues for this planner");
+ }
+
+ tmpExec.sort(Comparator.comparing(Executor::getMaxSize,
+ Comparator.nullsLast(Comparator.naturalOrder())));
+
+ executors = List.copyOf(tmpExec);
+
+ if (executors.stream().filter(e -> e.getMaxSize() == null).count() > 1) {
+ throw new IllegalArgumentException(
+ "Can only have one executor w/o a maxSize. " + params.getOptions().get("executors"));
+ }
+
+ // use the add method on the Set interface to check for duplicate maxSizes
+ Set<Long> maxSizes = new HashSet<>();
+ executors.forEach(e -> {
+ if (!maxSizes.add(e.getMaxSize())) {
+ throw new IllegalArgumentException(
+ "Duplicate maxSize set in executors. " + params.getOptions().get("executors"));
+ }
+ });
+
determineMaxFilesToCompact(params);
}
- @SuppressWarnings("removal")
+ @SuppressWarnings("deprecation")
private void determineMaxFilesToCompact(InitParameters params) {
- String fqo = params.getFullyQualifiedOption("maxOpen");
- if (!params.getServiceEnvironment().getConfiguration().isSet(fqo)
- && params.getServiceEnvironment().getConfiguration()
- .isSet(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey())) {
- log.warn("The property " + Property.TSERV_MAJC_THREAD_MAXOPEN.getKey()
- + " was set, it is deprecated. Set the " + fqo + " option instead.");
- this.maxFilesToCompact = Integer.parseInt(params.getServiceEnvironment().getConfiguration()
- .get(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey()));
- } else {
- this.maxFilesToCompact = Integer.parseInt(params.getOptions().getOrDefault("maxOpen",
- Property.TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN.getDefaultValue()));
+
+ String maxOpen = params.getOptions().get("maxOpen");
+ if (maxOpen == null) {
+ maxOpen = Property.TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN.getDefaultValue();
+ log.trace("default maxOpen not set, defaulting to {}", maxOpen);
+ }
+ this.maxFilesToCompact = Integer.parseInt(maxOpen);
+ }
+
+ private void validateConfig(JsonElement json, List<String> fields, String className) {
+
+ JsonObject jsonObject = GSON.get().fromJson(json, JsonObject.class);
+
+ List<String> objectProperties = new ArrayList<>(jsonObject.keySet());
+ HashSet<String> classFieldNames = new HashSet<>(fields);
+
+ if (!classFieldNames.containsAll(objectProperties)) {
+ objectProperties.removeAll(classFieldNames);
+ throw new JsonParseException(
+ "Invalid fields: " + objectProperties + " provided for class: " + className);
}
}
@Override
public CompactionPlan makePlan(PlanningParameters params) {
- try {
+ if (params.getCandidates().isEmpty()) {
+ return params.createPlanBuilder().build();
+ }
- if (params.getCandidates().isEmpty()) {
- return params.createPlanBuilder().build();
+ Set<CompactableFile> filesCopy = new HashSet<>(params.getCandidates());
+
+ long maxSizeToCompact = getMaxSizeToCompact(params.getKind());
+
+ Collection<CompactableFile> group;
+ if (params.getRunningCompactions().isEmpty()) {
+ group =
+ findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact, maxSizeToCompact);
+
+ if (!group.isEmpty() && group.size() < params.getCandidates().size()
+ && params.getCandidates().size() <= maxFilesToCompact
+ && (params.getKind() == CompactionKind.USER
+ || params.getKind() == DeprecatedCompactionKind.SELECTOR)) {
+ // USER and SELECTOR compactions must eventually compact all files. When a subset of files
+ // that meets the compaction ratio is selected, look ahead and see if the next compaction
+ // would also meet the compaction ratio. If not then compact everything to avoid doing
+ // more than logarithmic work across multiple comapctions.
+
+ filesCopy.removeAll(group);
+ filesCopy.add(getExpected(group, 0));
+
+ if (findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact,
+ maxSizeToCompact).isEmpty()) {
+ // The next possible compaction does not meet the compaction ratio, so compact
+ // everything.
+ group = Set.copyOf(params.getCandidates());
+ }
+
}
- Set<CompactableFile> filesCopy = new HashSet<>(params.getCandidates());
+ } else if (params.getKind() == CompactionKind.SYSTEM) {
+ // This code determines if once the files compacting finish would they be included in a
+ // compaction with the files smaller than them? If so, then wait for the running compaction
+ // to complete.
- long maxSizeToCompact = getMaxSizeToCompact(params.getKind());
+ // The set of files running compactions may produce
+ var expectedFiles = getExpected(params.getRunningCompactions());
- Collection<CompactableFile> group;
- if (params.getRunningCompactions().isEmpty()) {
- group = findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact,
- maxSizeToCompact);
+ if (!Collections.disjoint(filesCopy, expectedFiles)) {
+ throw new AssertionError();
+ }
- if (!group.isEmpty() && group.size() < params.getCandidates().size()
- && params.getCandidates().size() <= maxFilesToCompact
- && (params.getKind() == CompactionKind.USER
- || params.getKind() == CompactionKind.SELECTOR)) {
- // USER and SELECTOR compactions must eventually compact all files. When a subset of files
- // that meets the compaction ratio is selected, look ahead and see if the next compaction
- // would also meet the compaction ratio. If not then compact everything to avoid doing
- // more than logarithmic work across multiple comapctions.
+ filesCopy.addAll(expectedFiles);
- filesCopy.removeAll(group);
- filesCopy.add(getExpected(group, 0));
+ group =
+ findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact, maxSizeToCompact);
- if (findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact,
- maxSizeToCompact).isEmpty()) {
- // The next possible compaction does not meet the compaction ratio, so compact
- // everything.
- group = Set.copyOf(params.getCandidates());
- }
-
- }
-
- } else if (params.getKind() == CompactionKind.SYSTEM) {
- // This code determines if once the files compacting finish would they be included in a
- // compaction with the files smaller than them? If so, then wait for the running compaction
- // to complete.
-
- // The set of files running compactions may produce
- var expectedFiles = getExpected(params.getRunningCompactions());
-
- if (!Collections.disjoint(filesCopy, expectedFiles)) {
- throw new AssertionError();
- }
-
- filesCopy.addAll(expectedFiles);
-
- group = findDataFilesToCompact(filesCopy, params.getRatio(), maxFilesToCompact,
- maxSizeToCompact);
-
- if (!Collections.disjoint(group, expectedFiles)) {
- // file produced by running compaction will eventually compact with existing files, so
- // wait.
- group = Set.of();
- }
- } else {
+ if (!Collections.disjoint(group, expectedFiles)) {
+ // file produced by running compaction will eventually compact with existing files, so
+ // wait.
group = Set.of();
}
+ } else {
+ group = Set.of();
+ }
- if (group.isEmpty()) {
-
- if ((params.getKind() == CompactionKind.USER || params.getKind() == CompactionKind.SELECTOR
- || params.getKind() == CompactionKind.CHOP)
- && params.getRunningCompactions().stream()
- .noneMatch(job -> job.getKind() == params.getKind())) {
- group = findMaximalRequiredSetToCompact(params.getCandidates(), maxFilesToCompact);
- } else if (params.getKind() == CompactionKind.SYSTEM
- && params.getRunningCompactions().isEmpty()
- && params.getAll().size() == params.getCandidates().size()) {
- int maxTabletFiles = getMaxTabletFiles(
- params.getServiceEnvironment().getConfiguration(params.getTableId()));
- if (params.getAll().size() > maxTabletFiles) {
- // The tablet is above its max files, there are no compactions running, all files are
- // candidates for a system compaction, and no files were found to compact. Attempt to
- // find a set of files to compact by lowering the compaction ratio.
- group = findFilesToCompactWithLowerRatio(params, maxSizeToCompact, maxTabletFiles);
- }
+ if (group.isEmpty()) {
+ if ((params.getKind() == CompactionKind.USER
+ || params.getKind() == DeprecatedCompactionKind.SELECTOR)
+ && params.getRunningCompactions().stream()
+ .noneMatch(job -> job.getKind() == params.getKind())) {
+ group = findMaximalRequiredSetToCompact(params.getCandidates(), maxFilesToCompact);
+ } else if (params.getKind() == CompactionKind.SYSTEM
+ && params.getRunningCompactions().isEmpty()
+ && params.getAll().size() == params.getCandidates().size()) {
+ int maxTabletFiles =
+ getMaxTabletFiles(params.getServiceEnvironment().getConfiguration(params.getTableId()));
+ if (params.getAll().size() > maxTabletFiles) {
+ // The tablet is above its max files, there are no compactions running, all files are
+ // candidates for a system compaction, and no files were found to compact. Attempt to
+ // find a set of files to compact by lowering the compaction ratio.
+ group = findFilesToCompactWithLowerRatio(params, maxSizeToCompact, maxTabletFiles);
}
}
+ }
- if (group.isEmpty()) {
- return params.createPlanBuilder().build();
- } else {
- // determine which executor to use based on the size of the files
- var ceid = getExecutor(group);
+ if (group.isEmpty()) {
+ return params.createPlanBuilder().build();
+ } else {
+ // determine which executor to use based on the size of the files
+ var ceid = getExecutor(group);
- return params.createPlanBuilder().addJob(createPriority(params, group), ceid, group)
- .build();
- }
- } catch (RuntimeException e) {
- throw e;
+ return params.createPlanBuilder().addJob(createPriority(params, group), ceid, group).build();
}
}
@@ -438,7 +505,7 @@
new URI("hdfs://fake/accumulo/tables/adef/t-zzFAKEzz/FAKE-0000" + count + ".rf"), size,
0);
} catch (URISyntaxException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/compaction/package-info.java b/core/src/main/java/org/apache/accumulo/core/spi/compaction/package-info.java
index fd7d094..e1bea0f 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/compaction/package-info.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/compaction/package-info.java
@@ -65,9 +65,6 @@
* to periodically select files to compact. This supports use cases like periodically compacting all
* files because there are too many deletes. See
* {@link org.apache.accumulo.core.client.admin.compaction.CompactionSelector}
- * <li><b>Compaction Strategy</b> A deprecated pluggable component replaced by the Selector and
- * Configurer. See {@code org.apache.accumulo.core.client.admin.CompactionStrategyConfig}'s own
- * documentation for more information about why it was deprecated.
* </ul>
* </ul>
* </ul>
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java b/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java
index 15e5f45..4ed55ac 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/crypto/AESCryptoService.java
@@ -19,6 +19,7 @@
package org.apache.accumulo.core.spi.crypto;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
@@ -78,7 +79,6 @@
private static final String NO_CRYPTO_VERSION = "U+1F47B";
private static final String URI = "uri";
private static final String KEY_WRAP_TRANSFORM = "AESWrap";
- private static final SecureRandom random = new SecureRandom();
private Key encryptingKek = null;
private String keyLocation = null;
@@ -358,8 +358,8 @@
} catch (NoSuchAlgorithmException | NoSuchPaddingException e) {
throw new CryptoException("Error obtaining cipher for transform " + transformation, e);
}
- this.fek = generateKey(random, KEY_LENGTH_IN_BYTES);
- random.nextBytes(this.initVector);
+ this.fek = generateKey(RANDOM.get(), KEY_LENGTH_IN_BYTES);
+ RANDOM.get().nextBytes(this.initVector);
this.firstInitVector = Arrays.copyOf(this.initVector, this.initVector.length);
this.decryptionParameters =
createCryptoParameters(VERSION, encryptingKek, keyLocation, keyManager, fek);
@@ -499,14 +499,14 @@
} catch (NoSuchAlgorithmException | NoSuchPaddingException e) {
throw new CryptoException("Error obtaining cipher for transform " + transformation, e);
}
- this.fek = generateKey(random, KEY_LENGTH_IN_BYTES);
+ this.fek = generateKey(RANDOM.get(), KEY_LENGTH_IN_BYTES);
this.decryptionParameters =
createCryptoParameters(VERSION, encryptingKek, keyLocation, keyManager, fek);
}
@Override
public OutputStream encryptStream(OutputStream outputStream) throws CryptoException {
- random.nextBytes(initVector);
+ RANDOM.get().nextBytes(initVector);
try {
outputStream.write(initVector);
} catch (IOException e) {
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/fs/DelegatingChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/DelegatingChooser.java
index e76a1b7..eb39ada 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/fs/DelegatingChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/DelegatingChooser.java
@@ -89,7 +89,7 @@
+ Property.GENERAL_ARBITRARY_PROP_PREFIX + DEFAULT_SCOPED_VOLUME_CHOOSER
+ " must be a valid " + VolumeChooser.class.getSimpleName() + " to use the "
+ getClass().getSimpleName();
- throw new RuntimeException(msg);
+ throw new IllegalStateException(msg);
}
return createVolumeChooser(env, clazz, TABLE_CUSTOM_SUFFIX, env.getTable().orElseThrow(),
@@ -115,7 +115,7 @@
+ Property.GENERAL_ARBITRARY_PROP_PREFIX + DEFAULT_SCOPED_VOLUME_CHOOSER
+ " must be a valid " + VolumeChooser.class.getSimpleName() + " to use the "
+ getClass().getSimpleName();
- throw new RuntimeException(msg);
+ throw new IllegalStateException(msg);
}
property = DEFAULT_SCOPED_VOLUME_CHOOSER;
@@ -155,10 +155,10 @@
} else {
return env.getServiceEnv().instantiate(className, VolumeChooser.class);
}
- } catch (Exception e) {
+ } catch (ReflectiveOperationException e) {
String msg = "Failed to create instance for " + key + " configured to use " + className
+ " via " + property;
- throw new RuntimeException(msg, e);
+ throw new IllegalStateException(msg, e);
}
});
}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java
index 22c57ce..e941910 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java
@@ -182,7 +182,7 @@
if (preferredVolumes == null || preferredVolumes.isEmpty()) {
String msg = "Property " + TABLE_CUSTOM_SUFFIX + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES
+ " must be a subset of " + options + " to use the " + getClass().getSimpleName();
- throw new RuntimeException(msg);
+ throw new IllegalArgumentException(msg);
}
return parsePreferred(TABLE_CUSTOM_SUFFIX, preferredVolumes, options);
@@ -208,7 +208,7 @@
if (preferredVolumes == null || preferredVolumes.isEmpty()) {
String msg = "Property " + property + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES
+ " must be a subset of " + options + " to use the " + getClass().getSimpleName();
- throw new RuntimeException(msg);
+ throw new IllegalArgumentException(msg);
}
property = DEFAULT_SCOPED_PREFERRED_VOLUMES;
@@ -226,13 +226,13 @@
if (preferred.isEmpty()) {
String msg = "No volumes could be parsed from '" + property + "', which had a value of '"
+ preferredVolumes + "'";
- throw new RuntimeException(msg);
+ throw new IllegalArgumentException(msg);
}
// preferred volumes should also exist in the original options (typically, from
// instance.volumes)
if (Collections.disjoint(preferred, options)) {
String msg = "Some volumes in " + preferred + " are not valid volumes from " + options;
- throw new RuntimeException(msg);
+ throw new IllegalArgumentException(msg);
}
return preferred;
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java
index 7754991..55f1781 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java
@@ -18,7 +18,8 @@
*/
package org.apache.accumulo.core.spi.fs;
-import java.security.SecureRandom;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
import java.util.Set;
/**
@@ -27,7 +28,6 @@
* @since 2.1.0
*/
public class RandomVolumeChooser implements VolumeChooser {
- private static final SecureRandom random = new SecureRandom();
/**
* Selects a volume at random from the provided set of volumes. The environment scope is not
@@ -36,7 +36,7 @@
@Override
public String choose(VolumeChooserEnvironment env, Set<String> options) {
String[] optionsArray = options.toArray(new String[0]);
- return optionsArray[random.nextInt(optionsArray.length)];
+ return optionsArray[RANDOM.get().nextInt(optionsArray.length)];
}
/**
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java
index 20b92e0..99a29db 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java
@@ -19,13 +19,12 @@
package org.apache.accumulo.core.spi.fs;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import java.io.IOException;
-import java.security.SecureRandom;
import java.util.NavigableMap;
import java.util.Set;
import java.util.TreeMap;
-import java.util.concurrent.ExecutionException;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
@@ -35,9 +34,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
/**
* A {@link PreferredVolumeChooser} that takes remaining HDFS space into account when making a
@@ -48,8 +46,6 @@
*/
public class SpaceAwareVolumeChooser extends PreferredVolumeChooser {
- private static final SecureRandom random = new SecureRandom();
-
public static final String RECOMPUTE_INTERVAL = "spaceaware.volume.chooser.recompute.interval";
// Default time to wait in ms. Defaults to 5 min
@@ -68,11 +64,7 @@
@Override
public String choose(VolumeChooserEnvironment env, Set<String> options) {
- try {
- return getCache(env).get(getPreferredVolumes(env, options)).next();
- } catch (ExecutionException e) {
- throw new IllegalStateException("Execution exception when attempting to cache choice", e);
- }
+ return getCache(env).get(getPreferredVolumes(env, options)).next();
}
private synchronized LoadingCache<Set<String>,WeightedRandomCollection>
@@ -84,13 +76,8 @@
long computationCacheDuration = StringUtils.isNotBlank(propertyValue)
? Long.parseLong(propertyValue) : defaultComputationCacheDuration;
- choiceCache = CacheBuilder.newBuilder()
- .expireAfterWrite(computationCacheDuration, MILLISECONDS).build(new CacheLoader<>() {
- @Override
- public WeightedRandomCollection load(Set<String> key) {
- return new WeightedRandomCollection(key, env);
- }
- });
+ choiceCache = Caffeine.newBuilder().expireAfterWrite(computationCacheDuration, MILLISECONDS)
+ .build(key -> new WeightedRandomCollection(key, env));
}
return choiceCache;
@@ -134,7 +121,7 @@
}
public String next() {
- double value = random.nextDouble() * total;
+ double value = RANDOM.get().nextDouble() * total;
return map.higherEntry(value).getValue();
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java
index d21a879..f43f21e 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerHostSelector.java
@@ -18,6 +18,8 @@
*/
package org.apache.accumulo.core.spi.scan;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
@@ -139,7 +141,7 @@
} else {
final HashCode hashCode = hashTablet(tablet, profile.getSalt(numberOfPreviousAttempts));
final int serverIndex =
- (Math.abs(hashCode.asInt()) + RANDOM.nextInt(numServersToUseInAttemptPlan))
+ (Math.abs(hashCode.asInt()) + RANDOM.get().nextInt(numServersToUseInAttemptPlan))
% hostIndex.size();
final String hostToUse = hostIndex.get(serverIndex);
final List<String> scanServersOnHost = scanServerHosts.get(hostToUse);
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
index e0b9ddb..a7900645 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
@@ -19,9 +19,10 @@
package org.apache.accumulo.core.spi.scan;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
import java.lang.reflect.Type;
-import java.security.SecureRandom;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
@@ -42,7 +43,6 @@
import com.google.common.collect.Sets;
import com.google.common.hash.HashCode;
import com.google.common.hash.Hashing;
-import com.google.gson.Gson;
import com.google.gson.reflect.TypeToken;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
@@ -174,7 +174,6 @@
*/
public class ConfigurableScanServerSelector implements ScanServerSelector {
- protected static final SecureRandom RANDOM = new SecureRandom();
public static final String PROFILES_DEFAULT = "[{'isDefault':true,'maxBusyTimeout':'5m',"
+ "'busyTimeoutMultiplier':8, 'scanTypeActivations':[], "
+ "'attemptPlans':[{'servers':'3', 'busyTimeout':'33ms', 'salt':'one'},"
@@ -298,9 +297,8 @@
private void parseProfiles(Map<String,String> options) {
Type listType = new TypeToken<ArrayList<Profile>>() {}.getType();
- Gson gson = new Gson();
List<Profile> profList =
- gson.fromJson(options.getOrDefault("profiles", PROFILES_DEFAULT), listType);
+ GSON.get().fromJson(options.getOrDefault("profiles", PROFILES_DEFAULT), listType);
profiles = new HashMap<>();
defaultProfile = null;
@@ -436,8 +434,8 @@
var hashCode = hashTablet(tablet, profile.getSalt(attempts));
- int serverIndex =
- (Math.abs(hashCode.asInt()) + RANDOM.nextInt(numServers)) % orderedScanServers.size();
+ int serverIndex = (Math.abs(hashCode.asInt()) + RANDOM.get().nextInt(numServers))
+ % orderedScanServers.size();
serverToUse = orderedScanServers.get(serverIndex);
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java
index c6f362e..19bcbe8 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java
@@ -64,31 +64,6 @@
}
/**
- * The method parameters for {@link ScanDispatcher#dispatch(DispatchParmaters)}. This interface
- * exists so the API can evolve and additional parameters can be passed to the method in the
- * future.
- *
- * @since 2.0.0
- * @deprecated since 2.1.0 replaced by {@link DispatchParameters} and
- * {@link ScanDispatcher#dispatch(DispatchParameters)}
- */
- @Deprecated(since = "2.1.0")
- public interface DispatchParmaters extends DispatchParameters {}
-
- /**
- * @return Should return one of the executors named params.getScanExecutors().keySet()
- *
- * @deprecated since 2.1.0 please implement {@link #dispatch(DispatchParameters)} instead of this.
- * Accumulo will only call {@link #dispatch(DispatchParameters)} directly, it will
- * never call this. However the default implementation of
- * {@link #dispatch(DispatchParameters)} calls this method.
- */
- @Deprecated(since = "2.1.0")
- default String dispatch(DispatchParmaters params) {
- throw new UnsupportedOperationException();
- }
-
- /**
* The method parameters for {@link ScanDispatcher#dispatch(DispatchParameters)}. This interface
* exists so the API can evolve and additional parameters can be passed to the method in the
* future.
@@ -115,13 +90,5 @@
*
* @since 2.1.0
*/
-
- default ScanDispatch dispatch(DispatchParameters params) {
- String executor = dispatch((DispatchParmaters) params);
- if (executor.equals(DefaultScanDispatch.DEFAULT_SCAN_DISPATCH.getExecutorName())) {
- return DefaultScanDispatch.DEFAULT_SCAN_DISPATCH;
- }
-
- return ScanDispatch.builder().setExecutorName(executor).build();
- }
+ ScanDispatch dispatch(DispatchParameters params);
}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
index d74199a..a39450e 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
@@ -27,6 +27,7 @@
import org.apache.accumulo.core.client.ScannerBase;
import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.lock.ServiceLockData;
import org.apache.accumulo.core.spi.common.ServiceEnvironment;
import com.google.common.base.Preconditions;
@@ -46,7 +47,7 @@
/**
* The scan server group name that will be used when one is not specified.
*/
- String DEFAULT_SCAN_SERVER_GROUP_NAME = "default";
+ String DEFAULT_SCAN_SERVER_GROUP_NAME = ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME;
/**
* This method is called once after a {@link ScanServerSelector} is instantiated.
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
index 3cb22ea..e9ee45d 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
@@ -51,6 +51,7 @@
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
@@ -60,20 +61,18 @@
import org.apache.accumulo.core.dataImpl.thrift.TRowRange;
import org.apache.accumulo.core.dataImpl.thrift.TSummaries;
import org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest;
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
import org.apache.accumulo.core.metadata.schema.TabletMetadata;
import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
import org.apache.accumulo.core.rpc.ThriftUtil;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
import org.apache.accumulo.core.spi.cache.BlockCache;
import org.apache.accumulo.core.spi.crypto.CryptoService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client;
import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
import org.apache.accumulo.core.util.ByteBufferUtil;
import org.apache.accumulo.core.util.CancelFlagFuture;
import org.apache.accumulo.core.util.CompletableFutureUtil;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.TextUtil;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -84,9 +83,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.github.benmanes.caffeine.cache.Cache;
import com.google.common.base.Preconditions;
-import com.google.common.cache.Cache;
import com.google.common.hash.Hashing;
+import com.google.common.net.HostAndPort;
/**
* This class implements using multiple tservers to gather summaries.
@@ -163,31 +163,31 @@
* @return A map of the form : {@code map<tserver location, map<path, list<range>>} . The ranges
* associated with a file represent the tablets that use the file.
*/
- private Map<String,Map<TabletFile,List<TRowRange>>>
- getFilesGroupedByLocation(Predicate<TabletFile> fileSelector) {
-
- Iterable<TabletMetadata> tmi = TabletsMetadata.builder(ctx).forTable(tableId)
- .overlapping(startRow, endRow).fetch(FILES, LOCATION, LAST, PREV_ROW).build();
+ private Map<String,Map<StoredTabletFile,List<TRowRange>>>
+ getFilesGroupedByLocation(Predicate<StoredTabletFile> fileSelector) {
// get a subset of files
- Map<TabletFile,List<TabletMetadata>> files = new HashMap<>();
+ Map<StoredTabletFile,List<TabletMetadata>> files = new HashMap<>();
- for (TabletMetadata tm : tmi) {
- for (TabletFile file : tm.getFiles()) {
- if (fileSelector.test(file)) {
- // TODO push this filtering to server side and possibly use batch scanner
- files.computeIfAbsent(file, s -> new ArrayList<>()).add(tm);
+ try (TabletsMetadata tmi = TabletsMetadata.builder(ctx).forTable(tableId)
+ .overlapping(startRow, endRow).fetch(FILES, LOCATION, LAST, PREV_ROW).build()) {
+ for (TabletMetadata tm : tmi) {
+ for (StoredTabletFile file : tm.getFiles()) {
+ if (fileSelector.test(file)) {
+ // TODO push this filtering to server side and possibly use batch scanner
+ files.computeIfAbsent(file, s -> new ArrayList<>()).add(tm);
+ }
}
}
}
// group by location, then file
- Map<String,Map<TabletFile,List<TRowRange>>> locations = new HashMap<>();
+ Map<String,Map<StoredTabletFile,List<TRowRange>>> locations = new HashMap<>();
List<String> tservers = null;
- for (Entry<TabletFile,List<TabletMetadata>> entry : files.entrySet()) {
+ for (Entry<StoredTabletFile,List<TabletMetadata>> entry : files.entrySet()) {
String location = entry.getValue().stream().filter(tm -> tm.getLocation() != null) // filter
// tablets
@@ -209,9 +209,8 @@
// When no location, the approach below will consistently choose the same tserver for the
// same file (as long as the set of tservers is stable).
- int idx = Math
- .abs(Hashing.murmur3_32_fixed().hashString(entry.getKey().getPathStr(), UTF_8).asInt())
- % tservers.size();
+ int idx = Math.abs(Hashing.murmur3_32_fixed()
+ .hashString(entry.getKey().getNormalizedPathStr(), UTF_8).asInt()) % tservers.size();
location = tservers.get(idx);
}
@@ -261,7 +260,7 @@
private static class ProcessedFiles {
final SummaryCollection summaries;
- final Set<TabletFile> failedFiles;
+ final Set<StoredTabletFile> failedFiles;
public ProcessedFiles() {
this.summaries = new SummaryCollection();
@@ -286,12 +285,12 @@
private class FilesProcessor implements Supplier<ProcessedFiles> {
HostAndPort location;
- Map<TabletFile,List<TRowRange>> allFiles;
+ Map<StoredTabletFile,List<TRowRange>> allFiles;
private TInfo tinfo;
private AtomicBoolean cancelFlag;
public FilesProcessor(TInfo tinfo, HostAndPort location,
- Map<TabletFile,List<TRowRange>> allFiles, AtomicBoolean cancelFlag) {
+ Map<StoredTabletFile,List<TRowRange>> allFiles, AtomicBoolean cancelFlag) {
this.location = location;
this.allFiles = allFiles;
this.tinfo = tinfo;
@@ -306,7 +305,7 @@
try {
client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, ctx);
// partition files into smaller chunks so that not too many are sent to a tserver at once
- for (Map<TabletFile,List<TRowRange>> files : partition(allFiles, 500)) {
+ for (Map<StoredTabletFile,List<TRowRange>> files : partition(allFiles, 500)) {
if (!pfiles.failedFiles.isEmpty()) {
// there was a previous failure on this tserver, so just fail the rest of the files
pfiles.failedFiles.addAll(files.keySet());
@@ -315,8 +314,8 @@
try {
TSummaries tSums = client.startGetSummariesFromFiles(tinfo, ctx.rpcCreds(),
- getRequest(), files.entrySet().stream().collect(
- Collectors.toMap(entry -> entry.getKey().getPathStr(), Entry::getValue)));
+ getRequest(), files.entrySet().stream().collect(Collectors
+ .toMap(entry -> entry.getKey().getNormalizedPathStr(), Entry::getValue)));
while (!tSums.finished && !cancelFlag.get()) {
tSums = client.contiuneGetSummaries(tinfo, tSums.sessionId);
}
@@ -326,7 +325,7 @@
pfiles.failedFiles.addAll(files.keySet());
continue;
} catch (TException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
@@ -337,7 +336,7 @@
}
if (cancelFlag.get()) {
- throw new RuntimeException("Operation canceled");
+ throw new IllegalStateException("Operation canceled");
}
return pfiles;
@@ -350,13 +349,13 @@
PartitionFuture(TInfo tinfo, ExecutorService execSrv, int modulus, int remainder) {
Function<ProcessedFiles,CompletableFuture<ProcessedFiles>> go = previousWork -> {
- Predicate<TabletFile> fileSelector = file -> Math
- .abs(Hashing.murmur3_32_fixed().hashString(file.getPathStr(), UTF_8).asInt()) % modulus
- == remainder;
+ Predicate<StoredTabletFile> fileSelector = file -> Math
+ .abs(Hashing.murmur3_32_fixed().hashString(file.getNormalizedPathStr(), UTF_8).asInt())
+ % modulus == remainder;
if (previousWork != null) {
fileSelector = fileSelector.and(previousWork.failedFiles::contains);
}
- Map<String,Map<TabletFile,List<TRowRange>>> filesGBL;
+ Map<String,Map<StoredTabletFile,List<TRowRange>>> filesGBL;
filesGBL = getFilesGroupedByLocation(fileSelector);
List<CompletableFuture<ProcessedFiles>> futures = new ArrayList<>();
@@ -365,9 +364,9 @@
.completedFuture(new ProcessedFiles(previousWork.summaries, factory)));
}
- for (Entry<String,Map<TabletFile,List<TRowRange>>> entry : filesGBL.entrySet()) {
+ for (Entry<String,Map<StoredTabletFile,List<TRowRange>>> entry : filesGBL.entrySet()) {
HostAndPort location = HostAndPort.fromString(entry.getKey());
- Map<TabletFile,List<TRowRange>> allFiles = entry.getValue();
+ Map<StoredTabletFile,List<TRowRange>> allFiles = entry.getValue();
futures.add(CompletableFuture
.supplyAsync(new FilesProcessor(tinfo, location, allFiles, cancelFlag), execSrv));
@@ -448,8 +447,10 @@
private int countFiles() {
// TODO use a batch scanner + iterator to parallelize counting files
- return TabletsMetadata.builder(ctx).forTable(tableId).overlapping(startRow, endRow)
- .fetch(FILES, PREV_ROW).build().stream().mapToInt(tm -> tm.getFiles().size()).sum();
+ try (TabletsMetadata tabletsMetadata = TabletsMetadata.builder(ctx).forTable(tableId)
+ .overlapping(startRow, endRow).fetch(FILES, PREV_ROW).build()) {
+ return tabletsMetadata.stream().mapToInt(tm -> tm.getFiles().size()).sum();
+ }
}
private class GatherRequest implements Supplier<SummaryCollection> {
@@ -481,11 +482,11 @@
return tsr;
});
} catch (AccumuloException | AccumuloSecurityException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
if (cancelFlag.get()) {
- throw new RuntimeException("Operation canceled");
+ throw new IllegalStateException("Operation canceled");
}
return new SummaryCollection(tSums);
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java b/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java
index 48b123b..5b15a90 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummarizerFactory.java
@@ -19,6 +19,7 @@
package org.apache.accumulo.core.summary;
import java.io.IOException;
+import java.io.UncheckedIOException;
import org.apache.accumulo.core.classloader.ClassLoaderUtil;
import org.apache.accumulo.core.client.summary.Summarizer;
@@ -55,8 +56,10 @@
public Summarizer getSummarizer(SummarizerConfiguration conf) {
try {
return newSummarizer(conf.getClassName());
- } catch (ReflectiveOperationException | IOException e) {
- throw new RuntimeException(e);
+ } catch (ReflectiveOperationException e) {
+ throw new IllegalStateException(e);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
}
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java b/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
index b4f96de..ed69454 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/SummaryReader.java
@@ -46,7 +46,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.WritableUtils;
-import com.google.common.cache.Cache;
+import com.github.benmanes.caffeine.cache.Cache;
public class SummaryReader {
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
index efcd26c..6bd5ba5 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
@@ -18,97 +18,171 @@
*/
package org.apache.accumulo.core.tabletserver.log;
-import java.io.IOException;
-import java.util.Arrays;
import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.UUID;
import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.Text;
+import org.checkerframework.checker.nullness.qual.NonNull;
-public class LogEntry {
- private final KeyExtent extent;
- public final long timestamp;
- public final String filename;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
- public LogEntry(KeyExtent extent, long timestamp, String filename) {
- // note the prevEndRow in the extent does not matter, and is not used by LogEntry
- this.extent = extent;
- this.timestamp = timestamp;
- this.filename = filename;
+public final class LogEntry {
+
+ private final String path;
+ private final HostAndPort tserver;
+ private final UUID uniqueId;
+ private final Text columnQualifier;
+
+ private LogEntry(String path, HostAndPort tserver, UUID uniqueId, Text columnQualifier) {
+ this.path = path;
+ this.tserver = tserver;
+ this.uniqueId = uniqueId;
+ this.columnQualifier = columnQualifier;
}
- // make copy, but with a different filename
- public LogEntry switchFile(String filename) {
- return new LogEntry(extent, timestamp, filename);
+ /**
+ * Creates a new LogEntry object after validating the expected format of the path. We expect the
+ * path to contain a tserver (host+port) followed by a UUID as the file name as the last two
+ * components.<br>
+ * For example, file:///some/dir/path/localhost+1234/927ba659-d109-4bce-b0a5-bcbbcb9942a2 is a
+ * valid path.
+ *
+ * @param path path to validate
+ * @return an object representation of this log entry
+ * @throws IllegalArgumentException if the path is invalid
+ */
+ public static LogEntry fromPath(String path) {
+ return validatedLogEntry(path, null);
+ }
+
+ private static LogEntry validatedLogEntry(String path, Text columnQualifier) {
+ String[] parts = path.split("/");
+
+ if (parts.length < 2) {
+ throw new IllegalArgumentException(
+ "Invalid path format. The path should end with tserver/UUID.");
+ }
+
+ String tserverPart = parts[parts.length - 2];
+ String uuidPart = parts[parts.length - 1];
+
+ String badTServerMsg =
+ "Invalid tserver in path. Expected: host+port. Found '" + tserverPart + "'";
+ if (tserverPart.contains(":") || !tserverPart.contains("+")) {
+ throw new IllegalArgumentException(badTServerMsg);
+ }
+ HostAndPort tserver;
+ try {
+ tserver = HostAndPort.fromString(tserverPart.replace("+", ":"));
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgumentException(badTServerMsg);
+ }
+
+ String badUuidMsg = "Expected valid UUID. Found '" + uuidPart + "'";
+ UUID uuid;
+ try {
+ uuid = UUID.fromString(uuidPart);
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgumentException(badUuidMsg);
+ }
+ if (!uuid.toString().equals(uuidPart)) {
+ throw new IllegalArgumentException(badUuidMsg);
+ }
+
+ return new LogEntry(path, tserver, uuid, columnQualifier);
+ }
+
+ /**
+ * Construct a new LogEntry object after deserializing it from a metadata entry.
+ *
+ * @param entry the metadata entry
+ * @return a new LogEntry object constructed from the path stored in the column qualifier
+ * @throws IllegalArgumentException if the path stored in the metadata entry is invalid or if the
+ * serialized format of the entry is unrecognized
+ */
+ public static LogEntry fromMetaWalEntry(Entry<Key,Value> entry) {
+ Text fam = entry.getKey().getColumnFamily();
+ Preconditions.checkArgument(LogColumnFamily.NAME.equals(fam),
+ "The provided metadata entry's column family is %s instead of %s", fam,
+ LogColumnFamily.NAME);
+ Text qualifier = entry.getKey().getColumnQualifier();
+ String[] parts = qualifier.toString().split("/", 2);
+ Preconditions.checkArgument(parts.length == 2, "Malformed write-ahead log %s", qualifier);
+ return validatedLogEntry(parts[1], qualifier);
+ }
+
+ @NonNull
+ @VisibleForTesting
+ HostAndPort getTServer() {
+ return tserver;
+ }
+
+ @NonNull
+ public String getPath() {
+ return path;
+ }
+
+ @NonNull
+ public UUID getUniqueID() {
+ return uniqueId;
}
@Override
public String toString() {
- return extent.toMetaRow() + " " + filename;
+ return path;
}
- // unused; kept only for reference with corresponding fromBytes method
- @Deprecated(since = "2.1.0", forRemoval = true)
- public byte[] toBytes() throws IOException {
- DataOutputBuffer out = new DataOutputBuffer();
- extent.writeTo(out);
- out.writeLong(timestamp);
- // this next string used to store server, but this is no longer used
- out.writeUTF("-");
- out.writeUTF(filename);
- return Arrays.copyOf(out.getData(), out.getLength());
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (other instanceof LogEntry) {
+ return path.equals(((LogEntry) other).path);
+ }
+ return false;
}
- // kept only for upgrade code to upgrade WAL entries for the root table
- @Deprecated(since = "2.1.0", forRemoval = true)
- public static LogEntry fromBytes(byte[] bytes) throws IOException {
- DataInputBuffer inp = new DataInputBuffer();
- inp.reset(bytes, bytes.length);
- KeyExtent extent = KeyExtent.readFrom(inp);
- long timestamp = inp.readLong();
- // this next string used to store the server, but this is no longer used
- inp.readUTF();
- String filename = inp.readUTF();
- return new LogEntry(extent, timestamp, filename);
+ @Override
+ public int hashCode() {
+ return Objects.hash(path);
}
- public static LogEntry fromMetaWalEntry(Entry<Key,Value> entry) {
- final Key key = entry.getKey();
- final Value value = entry.getValue();
- KeyExtent extent = KeyExtent.fromMetaRow(key.getRow());
- // qualifier.split("/")[0] used to store the server, but this is no longer used, and the
- // qualifier can be ignored
- // the following line handles old-style log entry values that specify log sets
- String[] parts = value.toString().split("\\|")[0].split(";");
- String filename = parts[parts.length - 1];
- long timestamp = key.getTimestamp();
- return new LogEntry(extent, timestamp, filename);
+ /**
+ * Get the Text that should be used as the column qualifier to store this as a metadata entry.
+ */
+ @VisibleForTesting
+ Text getColumnQualifier() {
+ return columnQualifier == null ? newCQ() : new Text(columnQualifier);
}
- public Text getRow() {
- return extent.toMetaRow();
+ private Text newCQ() {
+ return new Text("-/" + getPath());
}
- public Text getColumnFamily() {
- return LogColumnFamily.NAME;
+ /**
+ * Put a delete marker in the provided mutation for this LogEntry.
+ *
+ * @param mutation the mutation to update
+ */
+ public void deleteFromMutation(Mutation mutation) {
+ mutation.putDelete(LogColumnFamily.NAME, getColumnQualifier());
}
- public String getUniqueID() {
- String[] parts = filename.split("/");
- return parts[parts.length - 1];
- }
-
- public Text getColumnQualifier() {
- return new Text("-/" + filename);
- }
-
- public Value getValue() {
- return new Value(filename);
+ /**
+ * Put this LogEntry into the provided mutation.
+ *
+ * @param mutation the mutation to update
+ */
+ public void addToMutation(Mutation mutation) {
+ mutation.put(LogColumnFamily.NAME, newCQ(), new Value());
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java b/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java
index 0c85bf4..78d658a 100644
--- a/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java
@@ -25,9 +25,9 @@
import java.util.concurrent.Callable;
import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.trace.thrift.TInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -131,8 +131,8 @@
}
/**
- * Obtain {@link org.apache.accumulo.core.trace.thrift.TInfo} for the current context. This is
- * used to send the current trace information to a remote process
+ * Obtain {@link org.apache.accumulo.core.clientImpl.thrift.TInfo} for the current context. This
+ * is used to send the current trace information to a remote process
*/
public static TInfo traceInfo() {
TInfo tinfo = new TInfo();
diff --git a/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java b/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java
index 5fb0d40..b6bb6f9 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/AddressUtil.java
@@ -25,15 +25,16 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.net.HostAndPort;
+
public class AddressUtil {
private static final Logger log = LoggerFactory.getLogger(AddressUtil.class);
- public static HostAndPort parseAddress(String address, boolean ignoreMissingPort)
- throws NumberFormatException {
- address = address.replace('+', ':');
- HostAndPort hap = HostAndPort.fromString(address);
- if (!ignoreMissingPort && !hap.hasPort()) {
+ public static HostAndPort parseAddress(final String address) throws NumberFormatException {
+ String normalized = normalizePortSeparator(address);
+ HostAndPort hap = HostAndPort.fromString(normalized);
+ if (!hap.hasPort()) {
throw new IllegalArgumentException(
"Address was expected to contain port. address=" + address);
}
@@ -41,8 +42,13 @@
return hap;
}
- public static HostAndPort parseAddress(String address, int defaultPort) {
- return parseAddress(address, true).withDefaultPort(defaultPort);
+ public static HostAndPort parseAddress(final String address, final int defaultPort) {
+ String normalized = normalizePortSeparator(address);
+ return HostAndPort.fromString(normalized).withDefaultPort(defaultPort);
+ }
+
+ private static String normalizePortSeparator(final String address) {
+ return address.replace('+', ':');
}
/**
diff --git a/core/src/main/java/org/apache/accumulo/core/util/BadArgumentException.java b/core/src/main/java/org/apache/accumulo/core/util/BadArgumentException.java
index ac9cafe..374c4f8 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/BadArgumentException.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/BadArgumentException.java
@@ -20,10 +20,17 @@
import java.util.regex.PatternSyntaxException;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
+
public final class BadArgumentException extends PatternSyntaxException {
private static final long serialVersionUID = 1L;
public BadArgumentException(String desc, String badarg, int index) {
super(desc, badarg, index);
}
+
+ public BadArgumentException(InvalidAccessExpressionException e) {
+ super(e.getDescription(), e.getPattern(), e.getIndex());
+ super.initCause(e);
+ }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/CleanUp.java b/core/src/main/java/org/apache/accumulo/core/util/CleanUp.java
deleted file mode 100644
index 0324a3b..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/CleanUp.java
+++ /dev/null
@@ -1,86 +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.util;
-
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.clientImpl.ConnectorImpl;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Connector uses static resource that create threads and use memory. The only way to clean up these
- * static resource related to Connectors created using ZookeeperInstance is to use this class.
- *
- * <p>
- * This class is not needed when only using {@link AccumuloClient}. The new AccumuloClient API that
- * replaces Connector is closable. For code that only uses AccumuloClient, when all AccumuloClients
- * are closed resources are cleaned up. Connectors that are derived from an AccumuloClient do not
- * necessitate the use of this code.
- *
- * @deprecated since 2.0.0 Use only {@link AccumuloClient} instead. Also, make sure you close the
- * AccumuloClient instances.
- */
-@Deprecated(since = "2.0.0")
-public class CleanUp {
-
- private static final Logger log = LoggerFactory.getLogger(CleanUp.class);
-
- /**
- * kills all threads created by internal Accumulo singleton resources. After this method is
- * called, no Connector will work in the current classloader.
- *
- * @param conn If available, Connector object to close resources on. Will accept null otherwise.
- */
- public static void shutdownNow(Connector conn) {
- SingletonManager.setMode(Mode.CLIENT);
- waitForZooKeeperClientThreads();
- if (conn != null) {
- ConnectorImpl connImpl = (ConnectorImpl) conn;
- connImpl.getAccumuloClient().close();
- }
- }
-
- /**
- * As documented in https://issues.apache.org/jira/browse/ZOOKEEPER-1816, ZooKeeper.close() is a
- * non-blocking call. This method will wait on the ZooKeeper internal threads to exit.
- */
- private static void waitForZooKeeperClientThreads() {
- Set<Thread> threadSet = Thread.getAllStackTraces().keySet();
- for (Thread thread : threadSet) {
- // find ZooKeeper threads that were created in the same ClassLoader as the current thread.
- if (thread.getClass().getName().startsWith("org.apache.zookeeper.ClientCnxn") && thread
- .getContextClassLoader().equals(Thread.currentThread().getContextClassLoader())) {
-
- // wait for the thread the die
- while (thread.isAlive()) {
- try {
- Thread.sleep(100);
- } catch (InterruptedException e) {
- log.error("{}", e.getMessage(), e);
- }
- }
- }
- }
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/CreateToken.java b/core/src/main/java/org/apache/accumulo/core/util/CreateToken.java
index a545c0d..cccd00f 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/CreateToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/CreateToken.java
@@ -86,34 +86,35 @@
pass = opts.securePassword;
}
- try {
- String principal = opts.principal;
- if (principal == null) {
- principal = getConsoleReader().readLine("Username (aka principal): ");
- }
-
- AuthenticationToken token = Class.forName(opts.tokenClassName)
- .asSubclass(AuthenticationToken.class).getDeclaredConstructor().newInstance();
- Properties props = new Properties();
- for (TokenProperty tp : token.getProperties()) {
- String input;
- if (pass != null && tp.getKey().equals("password")) {
- input = pass;
- } else {
- if (tp.getMask()) {
- input = getConsoleReader().readLine(tp.getDescription() + ": ", '*');
- } else {
- input = getConsoleReader().readLine(tp.getDescription() + ": ");
- }
- }
- props.put(tp.getKey(), input);
- token.init(props);
- }
- System.out.println("auth.type = " + opts.tokenClassName);
- System.out.println("auth.principal = " + principal);
- System.out.println("auth.token = " + ClientProperty.encodeToken(token));
- } catch (ReflectiveOperationException e) {
- throw new RuntimeException(e);
+ String principal = opts.principal;
+ if (principal == null) {
+ principal = getConsoleReader().readLine("Username (aka principal): ");
}
+
+ AuthenticationToken token;
+ try {
+ token = Class.forName(opts.tokenClassName).asSubclass(AuthenticationToken.class)
+ .getDeclaredConstructor().newInstance();
+ } catch (ReflectiveOperationException e) {
+ throw new IllegalStateException(e);
+ }
+ Properties props = new Properties();
+ for (TokenProperty tp : token.getProperties()) {
+ String input;
+ if (pass != null && tp.getKey().equals("password")) {
+ input = pass;
+ } else {
+ if (tp.getMask()) {
+ input = getConsoleReader().readLine(tp.getDescription() + ": ", '*');
+ } else {
+ input = getConsoleReader().readLine(tp.getDescription() + ": ");
+ }
+ }
+ props.put(tp.getKey(), input);
+ token.init(props);
+ }
+ System.out.println("auth.type = " + opts.tokenClassName);
+ System.out.println("auth.principal = " + principal);
+ System.out.println("auth.token = " + ClientProperty.encodeToken(token));
}
}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/FastFormat.java b/core/src/main/java/org/apache/accumulo/core/util/FastFormat.java
index f762110..da693b0 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/FastFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/FastFormat.java
@@ -31,8 +31,8 @@
String strNum = Long.toString(num, radix);
byte[] ret = new byte[Math.max(strNum.length(), width) + prefix.length];
if (toZeroPaddedString(ret, 0, strNum, width, prefix) != ret.length) {
- throw new RuntimeException(" Did not format to expected width " + num + " " + width + " "
- + radix + " " + new String(prefix, UTF_8));
+ throw new IllegalArgumentException(" Did not format to expected width " + num + " " + width
+ + " " + radix + " " + new String(prefix, UTF_8));
}
return ret;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Halt.java b/core/src/main/java/org/apache/accumulo/core/util/Halt.java
index ec822b4..da4939a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Halt.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Halt.java
@@ -18,8 +18,8 @@
*/
package org.apache.accumulo.core.util;
+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 org.apache.accumulo.core.util.threads.Threads;
import org.slf4j.Logger;
diff --git a/core/src/main/java/org/apache/accumulo/core/util/HostAndPort.java b/core/src/main/java/org/apache/accumulo/core/util/HostAndPort.java
deleted file mode 100644
index 08c5569..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/HostAndPort.java
+++ /dev/null
@@ -1,289 +0,0 @@
-/*
- * Copyright (C) 2011 The Guava Authors
- *
- * Licensed 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.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-import java.io.Serializable;
-import java.util.Comparator;
-import java.util.Objects;
-
-/**
- * This class was copied from Guava release 23.0 to replace the older Guava 14 version that had been
- * used in Accumulo. It was annotated as Beta by Google, therefore unstable to use in a core
- * Accumulo library. We learned this the hard way when Guava version 20 deprecated the getHostText
- * method and then removed the method all together in version 22. See ACCUMULO-4702
- *
- * Unused methods and annotations were removed to reduce maintenance costs.
- *
- * Javadoc from Guava 23.0 release: An immutable representation of a host and port.
- *
- * <p>
- * Example usage:
- *
- * <pre>
- * HostAndPort hp =
- * HostAndPort.fromString("[2001:db8::1]").withDefaultPort(80).requireBracketsForIPv6();
- * hp.getHost(); // returns "2001:db8::1"
- * hp.getPort(); // returns 80
- * hp.toString(); // returns "[2001:db8::1]:80"
- * </pre>
- *
- * <p>
- * Here are some examples of recognized formats:
- * <ul>
- * <li>example.com
- * <li>example.com:80
- * <li>192.0.2.1
- * <li>192.0.2.1:80
- * <li>[2001:db8::1] - {@link #getHost()} omits brackets
- * <li>[2001:db8::1]:80 - {@link #getHost()} omits brackets
- * <li>2001:db8::1 - Use requireBracketsForIPv6() to prohibit this
- * </ul>
- *
- * <p>
- * Note that this is not an exhaustive list, because these methods are only concerned with brackets,
- * colons, and port numbers. Full validation of the host field (if desired) is the caller's
- * responsibility.
- */
-public final class HostAndPort implements Serializable, Comparable<HostAndPort> {
- /** Magic value indicating the absence of a port number. */
- private static final int NO_PORT = -1;
-
- /** Hostname, IPv4/IPv6 literal, or unvalidated nonsense. */
- private final String host;
-
- /** Validated port number in the range [0..65535], or NO_PORT */
- private final int port;
-
- /** True if the parsed host has colons, but no surrounding brackets. */
- private final boolean hasBracketlessColons;
-
- private HostAndPort(String host, int port, boolean hasBracketlessColons) {
- this.host = host;
- this.port = port;
- this.hasBracketlessColons = hasBracketlessColons;
- }
-
- private static final Comparator<HostAndPort> COMPARATOR = Comparator.nullsFirst(
- Comparator.comparing(HostAndPort::getHost).thenComparingInt(h -> h.getPortOrDefault(0)));
-
- /**
- * Returns the portion of this {@code HostAndPort} instance that should represent the hostname or
- * IPv4/IPv6 literal.
- *
- * <p>
- * A successful parse does not imply any degree of sanity in this field. For additional
- * validation, see the HostSpecifier class.
- *
- * @since 20.0 (since 10.0 as {@code getHostText})
- */
- public String getHost() {
- return host;
- }
-
- /** Return true if this instance has a defined port. */
- public boolean hasPort() {
- return port >= 0;
- }
-
- /**
- * Get the current port number, failing if no port is defined.
- *
- * @return a validated port number, in the range [0..65535]
- * @throws IllegalStateException if no port is defined. You can use {@link #withDefaultPort(int)}
- * to prevent this from occurring.
- */
- public int getPort() {
- checkState(hasPort(), "the address does not include a port");
- return port;
- }
-
- /**
- * Build a HostAndPort instance from separate host and port values.
- *
- * <p>
- * Note: Non-bracketed IPv6 literals are allowed. Use #requireBracketsForIPv6() to prohibit these.
- *
- * @param host the host string to parse. Must not contain a port number.
- * @param port a port number from [0..65535]
- * @return if parsing was successful, a populated HostAndPort object.
- * @throws IllegalArgumentException if {@code host} contains a port number, or {@code port} is out
- * of range.
- */
- public static HostAndPort fromParts(String host, int port) {
- checkArgument(isValidPort(port), "Port out of range: %s", port);
- HostAndPort parsedHost = fromString(host);
- checkArgument(!parsedHost.hasPort(), "Host has a port: %s", host);
- return new HostAndPort(parsedHost.host, port, parsedHost.hasBracketlessColons);
- }
-
- /**
- * Split a freeform string into a host and port, without strict validation.
- *
- * Note that the host-only formats will leave the port field undefined. You can use
- * {@link #withDefaultPort(int)} to patch in a default value.
- *
- * @param hostPortString the input string to parse.
- * @return if parsing was successful, a populated HostAndPort object.
- * @throws IllegalArgumentException if nothing meaningful could be parsed.
- */
- public static HostAndPort fromString(String hostPortString) {
- Objects.requireNonNull(hostPortString, "hostPortString variable was null!");
- String host;
- String portString = null;
- boolean hasBracketlessColons = false;
-
- if (hostPortString.startsWith("[")) {
- String[] hostAndPort = getHostAndPortFromBracketedHost(hostPortString);
- host = hostAndPort[0];
- portString = hostAndPort[1];
- } else {
- int colonPos = hostPortString.indexOf(':');
- if (colonPos >= 0 && hostPortString.indexOf(':', colonPos + 1) == -1) {
- // Exactly 1 colon. Split into host:port.
- host = hostPortString.substring(0, colonPos);
- portString = hostPortString.substring(colonPos + 1);
- } else {
- // 0 or 2+ colons. Bare hostname or IPv6 literal.
- host = hostPortString;
- hasBracketlessColons = (colonPos >= 0);
- }
- }
-
- int port = NO_PORT;
- if (portString != null && !portString.trim().isEmpty()) {
- // Try to parse the whole port string as a number.
- // JDK7 accepts leading plus signs. We don't want to.
- checkArgument(!portString.startsWith("+"), "Unparseable port number: %s", hostPortString);
- try {
- port = Integer.parseInt(portString);
- } catch (NumberFormatException e) {
- throw new IllegalArgumentException("Unparseable port number: " + hostPortString);
- }
- checkArgument(isValidPort(port), "Port number out of range: %s", hostPortString);
- }
-
- return new HostAndPort(host, port, hasBracketlessColons);
- }
-
- /**
- * Parses a bracketed host-port string, throwing IllegalArgumentException if parsing fails.
- *
- * @param hostPortString the full bracketed host-port specification. Post might not be specified.
- * @return an array with 2 strings: host and port, in that order.
- * @throws IllegalArgumentException if parsing the bracketed host-port string fails.
- */
- private static String[] getHostAndPortFromBracketedHost(String hostPortString) {
- int colonIndex = 0;
- int closeBracketIndex = 0;
- checkArgument(hostPortString.charAt(0) == '[',
- "Bracketed host-port string must start with a bracket: %s", hostPortString);
- colonIndex = hostPortString.indexOf(':');
- closeBracketIndex = hostPortString.lastIndexOf(']');
- checkArgument(colonIndex > -1 && closeBracketIndex > colonIndex,
- "Invalid bracketed host/port: %s", hostPortString);
-
- String host = hostPortString.substring(1, closeBracketIndex);
- if (closeBracketIndex + 1 == hostPortString.length()) {
- return new String[] {host, ""};
- } else {
- checkArgument(hostPortString.charAt(closeBracketIndex + 1) == ':',
- "Only a colon may follow a close bracket: %s", hostPortString);
- for (int i = closeBracketIndex + 2; i < hostPortString.length(); ++i) {
- checkArgument(Character.isDigit(hostPortString.charAt(i)), "Port must be numeric: %s",
- hostPortString);
- }
- return new String[] {host, hostPortString.substring(closeBracketIndex + 2)};
- }
- }
-
- /**
- * Provide a default port if the parsed string contained only a host.
- *
- * You can chain this after {@link #fromString(String)} to include a port in case the port was
- * omitted from the input string. If a port was already provided, then this method is a no-op.
- *
- * @param defaultPort a port number, from [0..65535]
- * @return a HostAndPort instance, guaranteed to have a defined port.
- */
- public HostAndPort withDefaultPort(int defaultPort) {
- checkArgument(isValidPort(defaultPort));
- if (hasPort() || port == defaultPort) {
- return this;
- }
- return new HostAndPort(host, defaultPort, hasBracketlessColons);
- }
-
- @Override
- public boolean equals(Object other) {
- if (this == other) {
- return true;
- }
- if (other instanceof HostAndPort) {
- HostAndPort that = (HostAndPort) other;
- return java.util.Objects.equals(this.host, that.host) && this.port == that.port
- && this.hasBracketlessColons == that.hasBracketlessColons;
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return java.util.Objects.hash(host, port, hasBracketlessColons);
- }
-
- /** Rebuild the host:port string, including brackets if necessary. */
- @Override
- public String toString() {
- // "[]:12345" requires 8 extra bytes.
- StringBuilder builder = new StringBuilder(host.length() + 8);
- if (host.indexOf(':') >= 0) {
- builder.append('[').append(host).append(']');
- } else {
- builder.append(host);
- }
- if (hasPort()) {
- builder.append(':').append(port);
- }
- return builder.toString();
- }
-
- /** Return true for valid port numbers. */
- private static boolean isValidPort(int port) {
- return port >= 0 && port <= 65535;
- }
-
- private static final long serialVersionUID = 0;
-
- /**
- * Returns the current port number, with a default if no port is defined.
- */
- public int getPortOrDefault(int defaultPort) {
- return hasPort() ? port : defaultPort;
- }
-
- /**
- * HostAndPort must implement compareTo. This method orders HostAndPort values using a String
- * compare on the Host value with a secondary integer compare on the Port value.
- */
- @Override
- public int compareTo(HostAndPort other) {
- return COMPARATOR.compare(this, other);
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java b/core/src/main/java/org/apache/accumulo/core/util/HostAndPortComparator.java
similarity index 63%
copy from core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
copy to core/src/main/java/org/apache/accumulo/core/util/HostAndPortComparator.java
index d347979..548267f 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/HostAndPortComparator.java
@@ -16,14 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.accumulo.core.rpc.clients;
+package org.apache.accumulo.core.util;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Client;
+import java.util.Comparator;
-public class ReplicationServicerThriftClient extends ThriftClientTypes<Client> {
+import com.google.common.net.HostAndPort;
- ReplicationServicerThriftClient(String serviceName) {
- super(serviceName, new Client.Factory());
+public class HostAndPortComparator implements Comparator<HostAndPort> {
+
+ private static final Comparator<HostAndPort> COMPARATOR = Comparator.nullsFirst(
+ Comparator.comparing(HostAndPort::getHost).thenComparingInt(h -> h.getPortOrDefault(0)));
+
+ @Override
+ public int compare(HostAndPort o1, HostAndPort o2) {
+ return COMPARATOR.compare(o1, o2);
}
-
}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/LazySingletons.java b/core/src/main/java/org/apache/accumulo/core/util/LazySingletons.java
new file mode 100644
index 0000000..aa57737
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/LazySingletons.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.util;
+
+import java.security.SecureRandom;
+import java.util.function.Supplier;
+
+import com.google.common.base.Suppliers;
+import com.google.gson.Gson;
+
+/**
+ * This class provides easy access to global, immutable, lazily-instantiated, and thread-safe
+ * singleton resources. These should be used with static imports.
+ */
+public class LazySingletons {
+
+ // prevent instantiating this utility class
+ private LazySingletons() {}
+
+ /**
+ * A Gson instance constructed with defaults. Construct your own if you need custom settings.
+ */
+ public static final Supplier<Gson> GSON = Suppliers.memoize(Gson::new);
+
+ /**
+ * A SecureRandom instance created with the default constructor.
+ */
+ public static final Supplier<SecureRandom> RANDOM = Suppliers.memoize(SecureRandom::new);
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Merge.java b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
index 41f5a67..4eba179 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Merge.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
@@ -36,7 +36,7 @@
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
import org.apache.accumulo.core.metadata.schema.DataFileValue;
import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
import org.apache.accumulo.core.trace.TraceUtil;
@@ -144,24 +144,48 @@
public void mergomatic(AccumuloClient client, String table, Text start, Text end, long goalSize,
boolean force) throws MergeException {
try {
- if (table.equals(MetadataTable.NAME)) {
+ if (table.equals(AccumuloTable.METADATA.tableName())) {
throw new IllegalArgumentException("cannot merge tablets on the metadata table");
}
List<Size> sizes = new ArrayList<>();
long totalSize = 0;
- // Merge any until you get larger than the goal size, and then merge one less tablet
- Iterator<Size> sizeIterator = getSizeIterator(client, table, start, end);
- while (sizeIterator.hasNext()) {
- Size next = sizeIterator.next();
- totalSize += next.size;
- sizes.add(next);
- if (totalSize > goalSize) {
- totalSize = mergeMany(client, table, sizes, goalSize, force, false);
+
+ TableId tableId;
+ ClientContext context = (ClientContext) client;
+ try {
+ tableId = context.getTableId(table);
+ } catch (Exception e) {
+ throw new MergeException(e);
+ }
+
+ try (TabletsMetadata tablets = TabletsMetadata.builder(context).scanMetadataTable()
+ .overRange(new KeyExtent(tableId, end, start).toMetaRange()).fetch(FILES, PREV_ROW)
+ .build()) {
+
+ Iterator<Size> sizeIterator = tablets.stream().map(tm -> {
+ long size = tm.getFilesMap().values().stream().mapToLong(DataFileValue::getSize).sum();
+ return new Size(tm.getExtent(), size);
+ }).iterator();
+
+ while (sizeIterator.hasNext()) {
+ Size next = sizeIterator.next();
+ totalSize += next.size;
+ sizes.add(next);
+
+ if (totalSize > goalSize) {
+ mergeMany(client, table, sizes, goalSize, force, false);
+ sizes.clear();
+ sizes.add(next);
+ totalSize = next.size;
+ }
}
}
+
+ // merge one less tablet
if (sizes.size() > 1) {
mergeMany(client, table, sizes, goalSize, force, true);
}
+
} catch (Exception ex) {
throw new MergeException(ex);
}
@@ -239,26 +263,4 @@
}
}
- protected Iterator<Size> getSizeIterator(AccumuloClient client, String tablename, Text start,
- Text end) throws MergeException {
- // open up metadata, walk through the tablets.
-
- TableId tableId;
- TabletsMetadata tablets;
- try {
- ClientContext context = (ClientContext) client;
- tableId = context.getTableId(tablename);
- tablets = TabletsMetadata.builder(context).scanMetadataTable()
- .overRange(new KeyExtent(tableId, end, start).toMetaRange()).fetch(FILES, PREV_ROW)
- .build();
- } catch (Exception e) {
- throw new MergeException(e);
- }
-
- return tablets.stream().map(tm -> {
- long size = tm.getFilesMap().values().stream().mapToLong(DataFileValue::getSize).sum();
- return new Size(tm.getExtent(), size);
- }).iterator();
- }
-
}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Retry.java b/core/src/main/java/org/apache/accumulo/core/util/Retry.java
index 952d980..916537d 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Retry.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Retry.java
@@ -18,11 +18,9 @@
*/
package org.apache.accumulo.core.util;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
-import java.security.SecureRandom;
-import java.util.concurrent.TimeUnit;
+import java.time.Duration;
import org.slf4j.Logger;
@@ -35,18 +33,17 @@
*/
public class Retry {
private long maxRetries; // not final for testing
- private long waitIncrement; // not final for testing
- private long maxWait; // not final for testing
- private final long logIntervalNanoSec;
+ private Duration waitIncrement; // not final for testing
+ private Duration maxWait; // not final for testing
+ private final Duration logInterval;
private double backOffFactor;
private long retriesDone;
- private long currentWait;
- private long initialWait;
+ private Duration currentWait;
+ private Duration initialWait;
private boolean hasNeverLogged;
private boolean hasLoggedWarn = false;
private long lastRetryLog;
- private static final SecureRandom random = new SecureRandom();
private double currentBackOffFactor;
private boolean doTimeJitter = true;
@@ -57,15 +54,15 @@
* @param waitIncrement The amount of time (ms) to increment next wait time by
* @param logInterval The amount of time (ms) between logging retries
*/
- private Retry(long maxRetries, long startWait, long waitIncrement, long maxWait, long logInterval,
- double backOffFactor) {
+ private Retry(long maxRetries, Duration startWait, Duration waitIncrement, Duration maxWait,
+ Duration logInterval, double backOffFactor) {
this.maxRetries = maxRetries;
this.maxWait = maxWait;
this.waitIncrement = waitIncrement;
this.retriesDone = 0;
this.currentWait = startWait;
this.initialWait = startWait;
- this.logIntervalNanoSec = MILLISECONDS.toNanos(logInterval);
+ this.logInterval = logInterval;
this.hasNeverLogged = true;
this.lastRetryLog = -1;
this.backOffFactor = backOffFactor;
@@ -94,19 +91,19 @@
// Visible for testing
@VisibleForTesting
- long getCurrentWait() {
+ Duration getCurrentWait() {
return currentWait;
}
// Visible for testing
@VisibleForTesting
- long getWaitIncrement() {
+ Duration getWaitIncrement() {
return waitIncrement;
}
// Visible for testing
@VisibleForTesting
- long getMaxWait() {
+ Duration getMaxWait() {
return maxWait;
}
@@ -118,20 +115,20 @@
// Visible for testing
@VisibleForTesting
- void setStartWait(long startWait) {
+ void setStartWait(Duration startWait) {
this.currentWait = startWait;
this.initialWait = startWait;
}
// Visible for testing
@VisibleForTesting
- void setWaitIncrement(long waitIncrement) {
+ void setWaitIncrement(Duration waitIncrement) {
this.waitIncrement = waitIncrement;
}
// Visible for testing
@VisibleForTesting
- void setMaxWait(long maxWait) {
+ void setMaxWait(Duration maxWait) {
this.maxWait = maxWait;
}
@@ -145,8 +142,8 @@
return maxRetries < 0;
}
- public long getLogInterval() {
- return NANOSECONDS.toMillis(logIntervalNanoSec);
+ public Duration getLogInterval() {
+ return logInterval;
}
public boolean canRetry() {
@@ -171,29 +168,35 @@
public void waitForNextAttempt(Logger log, String operationDescription)
throws InterruptedException {
- double waitFactor = (1 + (random.nextDouble() - 0.5) / 10.0) * currentBackOffFactor;
+ double waitFactor = (1 + (RANDOM.get().nextDouble() - 0.5) / 10.0) * currentBackOffFactor;
if (!doTimeJitter) {
waitFactor = currentBackOffFactor;
}
currentBackOffFactor = currentBackOffFactor * backOffFactor;
- log.debug("Sleeping for {}ms before retrying operation : {} ", currentWait,
+ log.debug("Sleeping for {}ms before retrying operation : {} ", currentWait.toMillis(),
operationDescription);
sleep(currentWait);
if (backOffFactor == 1) {
- currentWait = Math.min(maxWait, currentWait + waitIncrement);
+ currentWait = currentWait.plus(waitIncrement);
+ if (currentWait.compareTo(maxWait) > 0) {
+ currentWait = maxWait;
+ }
} else if (backOffFactor > 1.0) {
- if (currentWait < maxWait) {
- waitIncrement = (long) Math.ceil(waitFactor * this.initialWait);
- currentWait = Math.min(maxWait, initialWait + waitIncrement);
+ waitIncrement = Duration.ofMillis((long) Math.ceil(waitFactor * initialWait.toMillis()));
+ Duration tempWait = initialWait.plus(waitIncrement);
+ if (tempWait.compareTo(maxWait) > 0) {
+ currentWait = maxWait;
+ } else {
+ currentWait = tempWait;
}
}
}
- protected void sleep(long wait) throws InterruptedException {
- Thread.sleep(wait);
+ protected void sleep(Duration wait) throws InterruptedException {
+ Thread.sleep(wait.toMillis());
}
public void logRetry(Logger log, String message, Throwable t) {
@@ -205,7 +208,7 @@
}
hasNeverLogged = false;
lastRetryLog = now;
- } else if ((now - lastRetryLog) > logIntervalNanoSec) {
+ } else if ((now - lastRetryLog) > logInterval.toNanos()) {
log.warn(getMessage(message), t);
lastRetryLog = now;
hasLoggedWarn = true;
@@ -225,7 +228,7 @@
}
hasNeverLogged = false;
lastRetryLog = now;
- } else if ((now - lastRetryLog) > logIntervalNanoSec) {
+ } else if ((now - lastRetryLog) > logInterval.toNanos()) {
log.warn(getMessage(message));
lastRetryLog = now;
hasLoggedWarn = true;
@@ -269,33 +272,37 @@
* @return this builder with the maximum number of retries set to the provided value
*/
NeedsRetryDelay maxRetries(long max);
+
+ /**
+ * @return this builder with the maximum number of retries set to the number of retries that can
+ * occur within the given duration
+ */
+ NeedsRetryDelay maxRetriesWithinDuration(Duration duration);
}
public interface NeedsRetryDelay {
/**
- * @param duration the amount of time to wait before the first retry; input is converted to
- * milliseconds, rounded down to the nearest
+ * @param duration the amount of time to wait before the first retry
* @return this builder with the initial wait period set
*/
- NeedsTimeIncrement retryAfter(long duration, TimeUnit unit);
+ NeedsTimeIncrement retryAfter(Duration duration);
}
public interface NeedsTimeIncrement {
/**
- * @param duration the amount of additional time to add before each subsequent retry; input is
- * converted to milliseconds, rounded down to the nearest
+ * @param duration the amount of additional time to add before each subsequent retry
* @return this builder with the increment amount set
*/
- NeedsMaxWait incrementBy(long duration, TimeUnit unit);
+ NeedsMaxWait incrementBy(Duration duration);
}
public interface NeedsMaxWait {
/**
* @param duration the maximum amount of time to which the waiting period between retries can be
- * incremented; input is converted to milliseconds, rounded down to the nearest
+ * incremented
* @return this builder with a maximum time limit set
*/
- NeedsBackOffFactor maxWait(long duration, TimeUnit unit);
+ NeedsBackOffFactor maxWait(Duration duration);
}
public interface NeedsBackOffFactor {
@@ -309,11 +316,10 @@
public interface NeedsLogInterval {
/**
- * @param duration the minimum time interval between logging that a retry is occurring; input is
- * converted to milliseconds, rounded down to the nearest
+ * @param duration the minimum time interval between logging that a retry is occurring
* @return this builder with a logging interval set
*/
- BuilderDone logInterval(long duration, TimeUnit unit);
+ BuilderDone logInterval(Duration duration);
}
public interface BuilderDone {
@@ -353,11 +359,12 @@
private boolean modifiable = true;
private long maxRetries;
- private long initialWait;
- private long maxWait;
- private long waitIncrement;
- private long logInterval;
+ private Duration initialWait;
+ private Duration maxWait;
+ private Duration waitIncrement;
+ private Duration logInterval;
private double backOffFactor = 1.5;
+ private Duration retriesForDuration = null;
RetryFactoryBuilder() {}
@@ -382,19 +389,62 @@
}
@Override
- public NeedsTimeIncrement retryAfter(long duration, TimeUnit unit) {
+ public NeedsRetryDelay maxRetriesWithinDuration(Duration duration) {
checkState();
- Preconditions.checkArgument(duration >= 0, "Initial waiting period must not be negative");
- this.initialWait = unit.toMillis(duration);
+ Preconditions.checkArgument(!duration.isNegative(),
+ "Duration for retries must not be negative");
+ this.retriesForDuration = duration;
+ return this;
+ }
+
+ /**
+ * Calculate the maximum number of retries that can occur within {@link #retriesForDuration}
+ */
+ private void calculateRetriesWithinDuration() {
+ long numberOfRetries = 0;
+ long cumulativeWaitTimeMillis = 0;
+ long currentWaitTimeMillis = initialWait.toMillis();
+ final long retriesForDurationMillis = retriesForDuration.toMillis();
+
+ // set an upper bound for the number of retries
+ final long maxRetries = Duration.ofHours(1).toMillis();
+
+ while (cumulativeWaitTimeMillis + currentWaitTimeMillis <= retriesForDurationMillis
+ && numberOfRetries < maxRetries) {
+
+ cumulativeWaitTimeMillis += currentWaitTimeMillis;
+ numberOfRetries++;
+
+ if (backOffFactor > 1.0) {
+ currentWaitTimeMillis = (long) Math.ceil(currentWaitTimeMillis * backOffFactor);
+ } else {
+ currentWaitTimeMillis += waitIncrement.toMillis();
+ }
+
+ if (currentWaitTimeMillis > maxWait.toMillis()) {
+ currentWaitTimeMillis = maxWait.toMillis(); // Ensure wait time does not exceed maxWait
+ }
+
+ }
+
+ this.maxRetries = numberOfRetries;
+ }
+
+ @Override
+ public NeedsTimeIncrement retryAfter(Duration duration) {
+ checkState();
+ Preconditions.checkArgument(!duration.isNegative(),
+ "Initial waiting period must not be negative");
+ this.initialWait = duration;
return this;
}
@Override
- public NeedsMaxWait incrementBy(long duration, TimeUnit unit) {
+ public NeedsMaxWait incrementBy(Duration duration) {
checkState();
- Preconditions.checkArgument(duration >= 0,
+ Preconditions.checkArgument(!duration.isNegative(),
"Amount of time to increment the wait between each retry must not be negative");
- this.waitIncrement = unit.toMillis(duration);
+ this.waitIncrement = duration;
return this;
}
@@ -408,20 +458,20 @@
}
@Override
- public NeedsBackOffFactor maxWait(long duration, TimeUnit unit) {
+ public NeedsBackOffFactor maxWait(Duration duration) {
checkState();
- this.maxWait = unit.toMillis(duration);
- Preconditions.checkArgument(maxWait >= initialWait,
+ this.maxWait = duration;
+ Preconditions.checkArgument(maxWait.compareTo(initialWait) >= 0,
"Maximum wait between retries must not be less than the initial delay");
return this;
}
@Override
- public BuilderDone logInterval(long duration, TimeUnit unit) {
+ public BuilderDone logInterval(Duration duration) {
checkState();
- Preconditions.checkArgument(duration >= 0,
+ Preconditions.checkArgument(!duration.isNegative(),
"The amount of time between logging retries must not be negative");
- this.logInterval = unit.toMillis(duration);
+ this.logInterval = duration;
return this;
}
@@ -433,6 +483,10 @@
@Override
public Retry createRetry() {
+ if (retriesForDuration != null) {
+ calculateRetriesWithinDuration();
+ }
+ this.modifiable = false;
return new Retry(maxRetries, initialWait, waitIncrement, maxWait, logInterval, backOffFactor);
}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/RowRangeUtil.java b/core/src/main/java/org/apache/accumulo/core/util/RowRangeUtil.java
new file mode 100644
index 0000000..d61e17b
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/RowRangeUtil.java
@@ -0,0 +1,85 @@
+/*
+ * 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.util;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+
+import com.google.common.base.Preconditions;
+
+public class RowRangeUtil {
+
+ /**
+ * Validates a range is a valid KeyExtent data range, which is a special type of row range. These
+ * ranges are created by calling {@code new Range(startRow, false, endRow, true);} which is what
+ * {@link KeyExtent#toDataRange()} does.
+ *
+ * A KeyExtent data row range is defined as:
+ * <ul>
+ * <li>A range that has an inclusive start and exclusive end</li>
+ * <li>A range that only has the row portion set</li>
+ * <li>A range where both the start and end key end with a zero byte</li>
+ * </ul>
+ *
+ * @param range The range to validate
+ * @return The original range
+ */
+ public static Range requireKeyExtentDataRange(Range range) {
+ String errorMsg = "Range is not a KeyExtent data range";
+
+ if (!range.isInfiniteStartKey()) {
+ Preconditions.checkArgument(range.isStartKeyInclusive(),
+ "%s, start key must be inclusive. %s", errorMsg, range);
+ Preconditions.checkArgument(isOnlyRowSet(range.getStartKey()),
+ "%s, start key must only contain a row. %s", errorMsg, range);
+ Preconditions.checkArgument(isRowSuffixZeroByte(range.getStartKey()),
+ "%s, start key does not end with zero byte. %s, ", errorMsg, range);
+ }
+
+ if (!range.isInfiniteStopKey()) {
+ Preconditions.checkArgument(!range.isEndKeyInclusive(), "%s, end key must be exclusive. %s",
+ errorMsg, range);
+ Preconditions.checkArgument(isOnlyRowSet(range.getEndKey()),
+ "%s, end key must only contain a row. %s", errorMsg, range);
+ Preconditions.checkArgument(isRowSuffixZeroByte(range.getEndKey()),
+ "%s, end key does not end with a zero byte. %s, ", errorMsg, range);
+ }
+
+ return range;
+ }
+
+ public static boolean isOnlyRowSet(Key key) {
+ return key.getColumnFamilyData().length() == 0 && key.getColumnQualifierData().length() == 0
+ && key.getColumnVisibilityData().length() == 0 && key.getTimestamp() == Long.MAX_VALUE;
+ }
+
+ public static boolean isRowSuffixZeroByte(Key key) {
+ var row = key.getRowData();
+ return row.length() > 0 && row.byteAt(row.length() - 1) == (byte) 0x00;
+ }
+
+ public static ByteSequence stripZeroTail(ByteSequence row) {
+ if (row.byteAt(row.length() - 1) == (byte) 0x00) {
+ return row.subSequence(0, row.length() - 1);
+ }
+ return row;
+ }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java b/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java
deleted file mode 100644
index fd90077..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java
+++ /dev/null
@@ -1,91 +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.util;
-
-import java.util.EnumMap;
-
-public class ServerServices implements Comparable<ServerServices> {
- public static enum Service {
- TSERV_CLIENT, GC_CLIENT, COMPACTOR_CLIENT, SSERV_CLIENT;
- }
-
- public static final String SERVICE_SEPARATOR = ";";
- public static final String SEPARATOR_CHAR = "=";
-
- private EnumMap<Service,String> services;
- private String stringForm = null;
-
- public ServerServices(String services) {
- this.services = new EnumMap<>(Service.class);
-
- String[] addresses = services.split(SERVICE_SEPARATOR);
- for (String address : addresses) {
- String[] sa = address.split(SEPARATOR_CHAR, 2);
- this.services.put(Service.valueOf(sa[0]), sa[1]);
- }
- }
-
- public ServerServices(String address, Service service) {
- this(service.name() + SEPARATOR_CHAR + address);
- }
-
- public String getAddressString(Service service) {
- return services.get(service);
- }
-
- public HostAndPort getAddress(Service service) {
- return AddressUtil.parseAddress(getAddressString(service), false);
- }
-
- // DON'T CHANGE THIS; WE'RE USING IT FOR SERIALIZATION!!!
- @Override
- public String toString() {
- if (stringForm == null) {
- StringBuilder sb = new StringBuilder();
- String prefix = "";
- for (Service service : new Service[] {Service.TSERV_CLIENT, Service.GC_CLIENT}) {
- if (services.containsKey(service)) {
- sb.append(prefix).append(service.name()).append(SEPARATOR_CHAR)
- .append(services.get(service));
- prefix = SERVICE_SEPARATOR;
- }
- }
- stringForm = sb.toString();
- }
- return stringForm;
- }
-
- @Override
- public int hashCode() {
- return toString().hashCode();
- }
-
- @Override
- public boolean equals(Object o) {
- if (o instanceof ServerServices) {
- return toString().equals(o.toString());
- }
- return false;
- }
-
- @Override
- public int compareTo(ServerServices other) {
- return toString().compareTo(other.toString());
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/UtilWaitThread.java b/core/src/main/java/org/apache/accumulo/core/util/UtilWaitThread.java
index a05e1d2..1ad269a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/UtilWaitThread.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/UtilWaitThread.java
@@ -18,10 +18,6 @@
*/
package org.apache.accumulo.core.util;
-import static java.util.concurrent.TimeUnit.NANOSECONDS;
-
-import java.util.concurrent.TimeUnit;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -35,36 +31,4 @@
log.error("{}", e.getMessage(), e);
}
}
-
- /**
- * Copied from Guava release 23. The Uniterruptibles class was annotated as Beta by Google,
- * therefore unstable to use. The following javadoc was copied from
- * com.google.common.util.concurrent.Uninterruptibles:
- *
- * Utilities for treating interruptible operations as uninterruptible. In all cases, if a thread
- * is interrupted during such a call, the call continues to block until the result is available or
- * the timeout elapses, and only then re-interrupts the thread.
- *
- */
- public static void sleepUninterruptibly(long sleepFor, TimeUnit unit) {
- boolean interrupted = false;
- try {
- long remainingNanos = unit.toNanos(sleepFor);
- long end = System.nanoTime() + remainingNanos;
- while (true) {
- try {
- // TimeUnit.sleep() treats negative timeouts just like zero.
- NANOSECONDS.sleep(remainingNanos);
- return;
- } catch (InterruptedException e) {
- interrupted = true;
- remainingNanos = end - System.nanoTime();
- }
- }
- } finally {
- if (interrupted) {
- Thread.currentThread().interrupt();
- }
- }
- }
}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/UuidUtil.java b/core/src/main/java/org/apache/accumulo/core/util/UuidUtil.java
deleted file mode 100644
index 83d41ab..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/UuidUtil.java
+++ /dev/null
@@ -1,46 +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.util;
-
-public class UuidUtil {
- /**
- * A fast method for verifying a suffix of a string looks like a uuid.
- *
- * @param offset location where the uuid starts. Its expected the uuid occupies the rest of the
- * string.
- */
- public static boolean isUUID(String uuid, int offset) {
- if (uuid.length() - offset != 36) {
- return false;
- }
- for (int i = 0; i < 36; i++) {
- var c = uuid.charAt(i + offset);
- if (i == 8 || i == 13 || i == 18 || i == 23) {
- if (c != '-') {
- // expect '-' char at above positions, did not see it
- return false;
- }
- } else if (c < '0' || (c > '9' && c < 'A') || (c > 'F' && c < 'a') || c > 'f') {
- // expected hex at all other positions, did not see hex chars
- return false;
- }
- }
- return true;
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Validators.java b/core/src/main/java/org/apache/accumulo/core/util/Validators.java
index 3c756ea..084c912 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Validators.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Validators.java
@@ -24,8 +24,7 @@
import org.apache.accumulo.core.clientImpl.Namespace;
import org.apache.accumulo.core.data.TableId;
-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.util.tables.TableNameUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -155,7 +154,8 @@
public static final Validator<String> NEW_TABLE_NAME =
new Validator<>(tableName -> _tableName(tableName, false));
- private static final List<String> metadataTables = List.of(RootTable.NAME, MetadataTable.NAME);
+ private static final List<String> metadataTables =
+ List.of(AccumuloTable.ROOT.tableName(), AccumuloTable.METADATA.tableName());
public static final Validator<String> NOT_METADATA_TABLE = new Validator<>(t -> {
if (t == null) {
return NameSegment.Table.isNull();
@@ -192,9 +192,7 @@
if (id == null) {
return Optional.of("Table id must not be null");
}
- @SuppressWarnings("deprecation")
- TableId replicationId = org.apache.accumulo.core.replication.ReplicationTable.ID;
- if (RootTable.ID.equals(id) || MetadataTable.ID.equals(id) || replicationId.equals(id)
+ if (AccumuloTable.ROOT.tableId().equals(id) || AccumuloTable.METADATA.tableId().equals(id)
|| VALID_ID_PATTERN.matcher(id.canonical()).matches()) {
return Validator.OK;
}
@@ -206,12 +204,13 @@
if (id == null) {
return Optional.of("Table id must not be null");
}
- if (id.equals(MetadataTable.ID)) {
- return Optional.of("Cloning " + MetadataTable.NAME + " is dangerous and no longer supported,"
- + " see https://github.com/apache/accumulo/issues/1309.");
+ if (id.equals(AccumuloTable.METADATA.tableId())) {
+ return Optional.of(
+ "Cloning " + AccumuloTable.METADATA.tableName() + " is dangerous and no longer supported,"
+ + " see https://github.com/apache/accumulo/issues/1309.");
}
- if (id.equals(RootTable.ID)) {
- return Optional.of("Unable to clone " + RootTable.NAME);
+ if (id.equals(AccumuloTable.ROOT.tableId())) {
+ return Optional.of("Unable to clone " + AccumuloTable.ROOT.tableName());
}
return Validator.OK;
});
@@ -220,9 +219,9 @@
if (id == null) {
return Optional.of("Table id must not be null");
}
- if (RootTable.ID.equals(id)) {
- return Optional
- .of("Table must not be the " + RootTable.NAME + "(Id: " + RootTable.ID + ") table");
+ if (AccumuloTable.ROOT.tableId().equals(id)) {
+ return Optional.of("Table must not be the " + AccumuloTable.ROOT.tableName() + "(Id: "
+ + AccumuloTable.ROOT.tableId() + ") table");
}
return Validator.OK;
});
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobPrioritizer.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobPrioritizer.java
index 1f87459..039201b 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobPrioritizer.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobPrioritizer.java
@@ -29,13 +29,13 @@
Comparator.comparingInt(CompactionJob::getPriority)
.thenComparingInt(job -> job.getFiles().size()).reversed();
+ @SuppressWarnings("deprecation")
public static short createPriority(CompactionKind kind, int totalFiles, int compactingFiles) {
int prio = totalFiles + compactingFiles;
switch (kind) {
case USER:
- case CHOP:
// user-initiated compactions will have a positive priority
// based on number of files
if (prio > Short.MAX_VALUE) {
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlannerInitParams.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlannerInitParams.java
index eb28715..86fd8c4 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlannerInitParams.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionPlannerInitParams.java
@@ -23,7 +23,6 @@
import java.util.Map;
import java.util.Set;
-import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.spi.common.ServiceEnvironment;
import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
import org.apache.accumulo.core.spi.compaction.CompactionPlanner;
@@ -38,14 +37,16 @@
private final Set<CompactionExecutorId> requestedExternalExecutors;
private final ServiceEnvironment senv;
private final CompactionServiceId serviceId;
+ private final String prefix;
- public CompactionPlannerInitParams(CompactionServiceId serviceId, Map<String,String> plannerOpts,
- ServiceEnvironment senv) {
+ public CompactionPlannerInitParams(CompactionServiceId serviceId, String prefix,
+ Map<String,String> plannerOpts, ServiceEnvironment senv) {
this.serviceId = serviceId;
this.plannerOpts = plannerOpts;
this.requestedExecutors = new HashMap<>();
this.requestedExternalExecutors = new HashSet<>();
this.senv = senv;
+ this.prefix = prefix;
}
@Override
@@ -60,7 +61,7 @@
@Override
public String getFullyQualifiedOption(String key) {
- return Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + serviceId + ".planner.opts." + key;
+ return prefix + serviceId + ".planner.opts." + key;
}
@Override
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java
index 71d4490..5e2f233 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java
@@ -21,13 +21,14 @@
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
-import java.util.function.Consumer;
+import java.util.stream.Collectors;
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
-import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import com.google.common.collect.Sets;
@@ -35,110 +36,112 @@
* This class serves to configure compaction services from an {@link AccumuloConfiguration} object.
*
* Specifically, compaction service properties (those prefixed by "tserver.compaction.major
- * .service") are used.
+ * .service" or "compaction.service") are used.
*/
public class CompactionServicesConfig {
+ private static final Logger log = LoggerFactory.getLogger(CompactionServicesConfig.class);
private final Map<String,String> planners = new HashMap<>();
+ private final Map<String,String> plannerPrefixes = new HashMap<>();
private final Map<String,Long> rateLimits = new HashMap<>();
private final Map<String,Map<String,String>> options = new HashMap<>();
+ @SuppressWarnings("deprecation")
+ private final Property oldPrefix = Property.TSERV_COMPACTION_SERVICE_PREFIX;
+ private final Property newPrefix = Property.COMPACTION_SERVICE_PREFIX;
long defaultRateLimit;
- private final Consumer<String> deprecationWarningConsumer;
public static final CompactionServiceId DEFAULT_SERVICE = CompactionServiceId.of("default");
- @SuppressWarnings("removal")
- private long getDefaultThroughput(AccumuloConfiguration aconf) {
- if (aconf.isPropertySet(Property.TSERV_MAJC_THROUGHPUT)) {
- return aconf.getAsBytes(Property.TSERV_MAJC_THROUGHPUT);
- }
-
+ @SuppressWarnings("deprecation")
+ private long getDefaultThroughput() {
return ConfigurationTypeHelper
.getMemoryAsBytes(Property.TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT.getDefaultValue());
}
- @SuppressWarnings("removal")
- private Map<String,String> getConfiguration(AccumuloConfiguration aconf) {
+ private Map<String,Map<String,String>> getConfiguration(AccumuloConfiguration aconf) {
+ Map<String,Map<String,String>> properties = new HashMap<>();
- Map<String,String> configs =
- aconf.getAllPropertiesWithPrefix(Property.TSERV_COMPACTION_SERVICE_PREFIX);
+ var newProps = aconf.getAllPropertiesWithPrefixStripped(newPrefix);
+ properties.put(newPrefix.getKey(), newProps);
- // check if deprecated properties for compaction executor are set
- if (aconf.isPropertySet(Property.TSERV_MAJC_MAXCONCURRENT)) {
+ // get all of the services under the new prefix
+ var newServices =
+ newProps.keySet().stream().map(prop -> prop.split("\\.")[0]).collect(Collectors.toSet());
- String defaultServicePrefix =
- Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + DEFAULT_SERVICE.canonical() + ".";
+ Map<String,String> oldServices = new HashMap<>();
- // check if any properties for the default compaction service are set
- boolean defaultServicePropsSet = configs.keySet().stream()
- .filter(key -> key.startsWith(defaultServicePrefix)).map(Property::getPropertyByKey)
- .anyMatch(prop -> prop == null || aconf.isPropertySet(prop));
-
- if (defaultServicePropsSet) {
-
- String warning = "The deprecated property " + Property.TSERV_MAJC_MAXCONCURRENT.getKey()
- + " was set. Properties with the prefix " + defaultServicePrefix
- + " were also set which replace the deprecated properties. The deprecated property "
- + "was therefore ignored.";
-
- deprecationWarningConsumer.accept(warning);
-
+ for (Map.Entry<String,String> entry : aconf.getAllPropertiesWithPrefixStripped(oldPrefix)
+ .entrySet()) {
+ // Discard duplicate service definitions
+ var service = entry.getKey().split("\\.")[0];
+ if (newServices.contains(service)) {
+ log.warn("Duplicate compaction service '{}' definition exists. Ignoring property : '{}'",
+ service, entry.getKey());
} else {
- String numThreads = aconf.get(Property.TSERV_MAJC_MAXCONCURRENT);
-
- // Its possible a user has configured the other compaction services, but not the default
- // service. In this case want to produce a config with the default service configs
- // overridden using deprecated configs.
-
- HashMap<String,String> configsCopy = new HashMap<>(configs);
-
- Map<String,String> defaultServiceConfigs =
- Map.of(defaultServicePrefix + "planner", DefaultCompactionPlanner.class.getName(),
- defaultServicePrefix + "planner.opts.executors",
- "[{'name':'deprecated', 'numThreads':" + numThreads + "}]");
-
- configsCopy.putAll(defaultServiceConfigs);
-
- String warning = "The deprecated property " + Property.TSERV_MAJC_MAXCONCURRENT.getKey()
- + " was set. Properties with the prefix " + defaultServicePrefix
- + " were not set, these should replace the deprecated properties. The old properties "
- + "were automatically mapped to the new properties in process creating : "
- + defaultServiceConfigs + ".";
-
- deprecationWarningConsumer.accept(warning);
-
- configs = Map.copyOf(configsCopy);
+ oldServices.put(entry.getKey(), entry.getValue());
}
}
-
- return configs;
-
+ properties.put(oldPrefix.getKey(), oldServices);
+ // Return unmodifiable map
+ return Map.copyOf(properties);
}
- public CompactionServicesConfig(AccumuloConfiguration aconf,
- Consumer<String> deprecationWarningConsumer) {
- this.deprecationWarningConsumer = deprecationWarningConsumer;
- Map<String,String> configs = getConfiguration(aconf);
+ public CompactionServicesConfig(AccumuloConfiguration aconf) {
+ Map<String,Map<String,String>> configs = getConfiguration(aconf);
- configs.forEach((prop, val) -> {
-
- var suffix = prop.substring(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey().length());
- String[] tokens = suffix.split("\\.");
- if (tokens.length == 4 && tokens[1].equals("planner") && tokens[2].equals("opts")) {
- options.computeIfAbsent(tokens[0], k -> new HashMap<>()).put(tokens[3], val);
- } else if (tokens.length == 2 && tokens[1].equals("planner")) {
- planners.put(tokens[0], val);
- } else if (tokens.length == 3 && tokens[1].equals("rate") && tokens[2].equals("limit")) {
- var eprop = Property.getPropertyByKey(prop);
- if (eprop == null || aconf.isPropertySet(eprop) || !isDeprecatedThroughputSet(aconf)) {
- rateLimits.put(tokens[0], ConfigurationTypeHelper.getFixedMemoryAsBytes(val));
+ // Find compaction planner defs first.
+ configs.forEach((prefix, props) -> {
+ props.forEach((prop, val) -> {
+ String[] tokens = prop.split("\\.");
+ if (tokens.length == 2 && tokens[1].equals("planner")) {
+ if (prefix.equals(oldPrefix.getKey())) {
+ // Log a warning if the old prefix planner is defined by a user.
+ Property userDefined = null;
+ try {
+ userDefined = Property.valueOf(prefix + prop);
+ } catch (IllegalArgumentException e) {
+ log.trace("Property: {} is not set by default configuration", prefix + prop);
+ }
+ boolean isPropSet = true;
+ if (userDefined != null) {
+ isPropSet = aconf.isPropertySet(userDefined);
+ }
+ if (isPropSet) {
+ log.warn(
+ "Found compaction planner '{}' using a deprecated prefix. Please update property to use the '{}' prefix",
+ tokens[0], newPrefix);
+ }
+ }
+ plannerPrefixes.put(tokens[0], prefix);
+ planners.put(tokens[0], val);
}
- } else {
- throw new IllegalArgumentException("Malformed compaction service property " + prop);
- }
+ });
});
- defaultRateLimit = getDefaultThroughput(aconf);
+ // Now find all compaction planner options.
+ configs.forEach((prefix, props) -> {
+ props.forEach((prop, val) -> {
+ String[] tokens = prop.split("\\.");
+ if (!plannerPrefixes.containsKey(tokens[0])) {
+ throw new IllegalArgumentException(
+ "Incomplete compaction service definition, missing planner class: " + prop);
+ }
+ if (tokens.length == 4 && tokens[1].equals("planner") && tokens[2].equals("opts")) {
+ options.computeIfAbsent(tokens[0], k -> new HashMap<>()).put(tokens[3], val);
+ } else if (tokens.length == 3 && tokens[1].equals("rate") && tokens[2].equals("limit")) {
+ var eprop = Property.getPropertyByKey(prop);
+ if (eprop == null || aconf.isPropertySet(eprop)) {
+ rateLimits.put(tokens[0], ConfigurationTypeHelper.getFixedMemoryAsBytes(val));
+ }
+ } else if (tokens.length == 2 && tokens[1].equals("planner")) {
+ return; // moves to next opt
+ } else {
+ throw new IllegalArgumentException(
+ "Malformed compaction service property " + prefix + prop);
+ }
+ });
+ });
+ defaultRateLimit = getDefaultThroughput();
var diff = Sets.difference(options.keySet(), planners.keySet());
@@ -149,11 +152,6 @@
}
- @SuppressWarnings("removal")
- private boolean isDeprecatedThroughputSet(AccumuloConfiguration aconf) {
- return aconf.isPropertySet(Property.TSERV_MAJC_THROUGHPUT);
- }
-
public long getRateLimit(String serviceName) {
return getRateLimits().getOrDefault(serviceName, defaultRateLimit);
}
@@ -178,6 +176,10 @@
return planners;
}
+ public String getPlannerPrefix(String service) {
+ return plannerPrefixes.get(service);
+ }
+
public Map<String,Long> getRateLimits() {
return rateLimits;
}
diff --git a/core/src/main/thrift/trace.thrift b/core/src/main/java/org/apache/accumulo/core/util/compaction/DeprecatedCompactionKind.java
similarity index 68%
rename from core/src/main/thrift/trace.thrift
rename to core/src/main/java/org/apache/accumulo/core/util/compaction/DeprecatedCompactionKind.java
index 2950b34..8b814b2 100644
--- a/core/src/main/thrift/trace.thrift
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/DeprecatedCompactionKind.java
@@ -16,13 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
-namespace java org.apache.accumulo.core.trace.thrift
-namespace cpp org.apache.accumulo.core.trace.thrift
+package org.apache.accumulo.core.util.compaction;
-# OpenTelemetry uses the standards at https://www.w3.org/TR/trace-context/
-# to propagate information across process boundaries.
-struct TInfo {
- // 1:i64 traceId - removed in 2.1.0
- // 2:i64 parentId - removed in 2.1.0
- 3:map<string,string> headers
+import org.apache.accumulo.core.spi.compaction.CompactionKind;
+
+// The purpose of this class is to minimize the amount of code that has to suppress deprecation warnings.
+public class DeprecatedCompactionKind {
+ @SuppressWarnings("deprecation")
+ public static final CompactionKind SELECTOR = CompactionKind.SELECTOR;
}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
index c2e5f81..bbcc3f1 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
@@ -18,8 +18,6 @@
*/
package org.apache.accumulo.core.util.compaction;
-import static java.nio.charset.StandardCharsets.UTF_8;
-
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
@@ -27,6 +25,7 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
+import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
@@ -36,16 +35,16 @@
import org.apache.accumulo.core.clientImpl.ClientContext;
import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
import org.apache.accumulo.core.compaction.thrift.CompactorService;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
import org.apache.accumulo.core.fate.zookeeper.ZooReader;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
import org.apache.accumulo.core.rpc.ThriftUtil;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.threads.ThreadPools;
import org.apache.thrift.TException;
import org.apache.zookeeper.KeeperException;
@@ -53,6 +52,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.net.HostAndPort;
+
public class ExternalCompactionUtil {
private static class RunningCompactionFuture {
@@ -101,25 +102,21 @@
*/
public static Optional<HostAndPort> findCompactionCoordinator(ClientContext context) {
final String lockPath = context.getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
- byte[] address =
- ServiceLock.getLockData(context.getZooCache(), ServiceLock.path(lockPath), new ZcStat());
- if (null == address) {
- return Optional.empty();
- }
- return Optional.of(HostAndPort.fromString(new String(address, UTF_8)));
+ return ServiceLock.getLockData(context.getZooCache(), ServiceLock.path(lockPath), new ZcStat())
+ .map(sld -> sld.getAddress(ThriftService.COORDINATOR));
}
/**
* @return map of queue names to compactor addresses
*/
- public static Map<String,List<HostAndPort>> getCompactorAddrs(ClientContext context) {
+ public static Map<String,Set<HostAndPort>> getCompactorAddrs(ClientContext context) {
try {
- final Map<String,List<HostAndPort>> queuesAndAddresses = new HashMap<>();
+ final Map<String,Set<HostAndPort>> queuesAndAddresses = new HashMap<>();
final String compactorQueuesPath = context.getZooKeeperRoot() + Constants.ZCOMPACTORS;
ZooReader zooReader = context.getZooReader();
List<String> queues = zooReader.getChildren(compactorQueuesPath);
for (String queue : queues) {
- queuesAndAddresses.putIfAbsent(queue, new ArrayList<>());
+ queuesAndAddresses.putIfAbsent(queue, new HashSet<>());
try {
List<String> compactors = zooReader.getChildren(compactorQueuesPath + "/" + queue);
for (String compactor : compactors) {
@@ -139,10 +136,10 @@
return queuesAndAddresses;
} catch (KeeperException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
@@ -242,7 +239,7 @@
results.add(new RunningCompaction(job, compactorAddress, rcf.getQueue()));
}
} catch (InterruptedException | ExecutionException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
});
return results;
@@ -270,7 +267,7 @@
runningIds.add(ceid);
}
} catch (InterruptedException | ExecutionException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
});
diff --git a/core/src/main/java/org/apache/accumulo/core/util/format/BinaryFormatter.java b/core/src/main/java/org/apache/accumulo/core/util/format/BinaryFormatter.java
deleted file mode 100644
index 3dab96c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/format/BinaryFormatter.java
+++ /dev/null
@@ -1,74 +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.util.format;
-
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.ColumnVisibility;
-
-/**
- * @deprecated since 1.8.0; Use {@link DefaultFormatter} providing showLength and printTimestamps
- * via {@link FormatterConfig}.
- */
-@Deprecated(since = "1.8.0")
-public class BinaryFormatter extends DefaultFormatter {
- // this class can probably be replaced by DefaultFormatter since DefaultFormatter has the max
- // length stuff
- @Override
- public String next() {
- checkState(true);
- return formatEntry(getScannerIterator().next(), config.willPrintTimestamps(),
- config.getShownLength());
- }
-
- public static String formatEntry(Entry<Key,Value> entry, boolean printTimestamps,
- int shownLength) {
- StringBuilder sb = new StringBuilder();
-
- Key key = entry.getKey();
-
- // append row
- appendText(sb, key.getRow(), shownLength).append(" ");
-
- // append column family
- appendText(sb, key.getColumnFamily(), shownLength).append(":");
-
- // append column qualifier
- appendText(sb, key.getColumnQualifier(), shownLength).append(" ");
-
- // append visibility expression
- sb.append(new ColumnVisibility(key.getColumnVisibility()));
-
- // append timestamp
- if (printTimestamps) {
- sb.append(" ").append(entry.getKey().getTimestamp());
- }
-
- // append value
- Value value = entry.getValue();
- if (value != null && value.getSize() > 0) {
- sb.append("\t");
- appendValue(sb, value, shownLength);
- }
- return sb.toString();
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/format/DateStringFormatter.java b/core/src/main/java/org/apache/accumulo/core/util/format/DateStringFormatter.java
deleted file mode 100644
index 9383e07..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/format/DateStringFormatter.java
+++ /dev/null
@@ -1,85 +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.util.format;
-
-import java.util.Map.Entry;
-import java.util.TimeZone;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-
-/**
- * This class is <strong>not</strong> recommended because
- * {@link #initialize(Iterable, FormatterConfig)} replaces parameters in {@link FormatterConfig},
- * which could surprise users.
- *
- * This class can be replaced by {@link DefaultFormatter} where FormatterConfig is initialized with
- * a DateFormat set to {@link #DATE_FORMAT}. See
- * {@link DateFormatSupplier#createSimpleFormatSupplier(String, java.util.TimeZone)}.
- *
- * <pre>
- * final DateFormatSupplier dfSupplier = DateFormatSupplier.createSimpleFormatSupplier(
- * DateFormatSupplier.HUMAN_READABLE_FORMAT, TimeZone.getTimeZone("UTC"));
- * final FormatterConfig config =
- * new FormatterConfig().setPrintTimestamps(true).setDateFormatSupplier(dfSupplier);
- * </pre>
- *
- * @deprecated since 1.8.0
- */
-@Deprecated(since = "1.8.0")
-public class DateStringFormatter implements Formatter {
-
- private DefaultFormatter defaultFormatter;
- private TimeZone timeZone;
-
- public static final String DATE_FORMAT = DateFormatSupplier.HUMAN_READABLE_FORMAT;
-
- public DateStringFormatter() {
- this(TimeZone.getDefault());
- }
-
- public DateStringFormatter(TimeZone timeZone) {
- this.defaultFormatter = new DefaultFormatter();
- this.timeZone = timeZone;
- }
-
- @Override
- public void initialize(Iterable<Entry<Key,Value>> scanner, FormatterConfig config) {
- FormatterConfig newConfig = new FormatterConfig(config);
- newConfig.setDateFormatSupplier(
- DateFormatSupplier.createSimpleFormatSupplier(DATE_FORMAT, timeZone));
- defaultFormatter.initialize(scanner, newConfig);
- }
-
- @Override
- public boolean hasNext() {
- return defaultFormatter.hasNext();
- }
-
- @Override
- public String next() {
- return defaultFormatter.next();
- }
-
- @Override
- public void remove() {
- defaultFormatter.remove();
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/format/HexFormatter.java b/core/src/main/java/org/apache/accumulo/core/util/format/HexFormatter.java
deleted file mode 100644
index 1c7e07c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/format/HexFormatter.java
+++ /dev/null
@@ -1,141 +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.util.format;
-
-import java.util.Iterator;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.interpret.ScanInterpreter;
-import org.apache.hadoop.io.Text;
-
-/**
- * A simple formatter that print the row, column family, column qualifier, and value as hex
- */
-@Deprecated(since = "2.1.0")
-public class HexFormatter implements Formatter, ScanInterpreter {
-
- private char[] chars =
- {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'};
- private Iterator<Entry<Key,Value>> iter;
- private FormatterConfig config;
-
- private void toHex(StringBuilder sb, byte[] bin) {
-
- for (int i = 0; i < bin.length; i++) {
- if (i > 0 && i % 2 == 0) {
- sb.append('-');
- }
- sb.append(chars[0x0f & (bin[i] >>> 4)]);
- sb.append(chars[0x0f & bin[i]]);
- }
- }
-
- private int fromChar(char b) {
- if (b >= '0' && b <= '9') {
- return (b - '0');
- } else if (b >= 'a' && b <= 'f') {
- return (b - 'a' + 10);
- }
-
- throw new IllegalArgumentException("Bad char " + b);
- }
-
- private byte[] toBinary(String hex) {
- hex = hex.replace("-", "");
-
- byte[] bin = new byte[(hex.length() / 2) + (hex.length() % 2)];
-
- int j = 0;
- for (int i = 0; i < bin.length; i++) {
- bin[i] = (byte) (fromChar(hex.charAt(j++)) << 4);
- if (j >= hex.length()) {
- break;
- }
- bin[i] |= (byte) fromChar(hex.charAt(j++));
- }
-
- return bin;
- }
-
- @Override
- public boolean hasNext() {
- return iter.hasNext();
- }
-
- @Override
- public String next() {
- Entry<Key,Value> entry = iter.next();
-
- StringBuilder sb = new StringBuilder();
-
- toHex(sb, entry.getKey().getRowData().toArray());
- sb.append(" ");
- toHex(sb, entry.getKey().getColumnFamilyData().toArray());
- sb.append(" ");
- toHex(sb, entry.getKey().getColumnQualifierData().toArray());
- sb.append(" [");
- sb.append(entry.getKey().getColumnVisibilityData());
- sb.append("] ");
- if (config.willPrintTimestamps()) {
- sb.append(entry.getKey().getTimestamp());
- sb.append(" ");
- }
- toHex(sb, entry.getValue().get());
-
- return sb.toString();
- }
-
- @Override
- public void remove() {
- iter.remove();
- }
-
- @Override
- public void initialize(Iterable<Entry<Key,Value>> scanner, FormatterConfig config) {
- this.iter = scanner.iterator();
- this.config = new FormatterConfig(config);
- }
-
- @Override
- public Text interpretRow(Text row) {
- return new Text(toBinary(row.toString()));
- }
-
- @Override
- public Text interpretBeginRow(Text row) {
- return interpretRow(row);
- }
-
- @Override
- public Text interpretEndRow(Text row) {
- return interpretRow(row);
- }
-
- @Override
- public Text interpretColumnFamily(Text cf) {
- return interpretRow(cf);
- }
-
- @Override
- public Text interpretColumnQualifier(Text cq) {
- return interpretRow(cq);
- }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/interpret/DefaultScanInterpreter.java b/core/src/main/java/org/apache/accumulo/core/util/interpret/DefaultScanInterpreter.java
deleted file mode 100644
index 25086ff..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/interpret/DefaultScanInterpreter.java
+++ /dev/null
@@ -1,54 +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.util.interpret;
-
-import org.apache.hadoop.io.Text;
-
-/**
- * @deprecated since 2.1.0 This will be removed in a future version in favor of JShell
- */
-@Deprecated(since = "2.1.0")
-public class DefaultScanInterpreter implements ScanInterpreter {
-
- @Override
- public Text interpretRow(Text row) {
- return row;
- }
-
- @Override
- public Text interpretBeginRow(Text row) {
- return row;
- }
-
- @Override
- public Text interpretEndRow(Text row) {
- return row;
- }
-
- @Override
- public Text interpretColumnFamily(Text cf) {
- return cf;
- }
-
- @Override
- public Text interpretColumnQualifier(Text cq) {
- return cq;
- }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/interpret/HexScanInterpreter.java b/core/src/main/java/org/apache/accumulo/core/util/interpret/HexScanInterpreter.java
deleted file mode 100644
index 09a0400..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/interpret/HexScanInterpreter.java
+++ /dev/null
@@ -1,31 +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.util.interpret;
-
-import org.apache.accumulo.core.util.format.HexFormatter;
-
-/**
- * As simple scan interpreter that converts hex to binary. IT supports translating the output of
- * {@link HexFormatter} back to binary. The hex input can contain dashes (because
- * {@link HexFormatter} outputs dashes) which are ignored.
- */
-@Deprecated(since = "2.1.0")
-public class HexScanInterpreter extends HexFormatter {
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/interpret/ScanInterpreter.java b/core/src/main/java/org/apache/accumulo/core/util/interpret/ScanInterpreter.java
deleted file mode 100644
index 7cac40a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/interpret/ScanInterpreter.java
+++ /dev/null
@@ -1,41 +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.util.interpret;
-
-import org.apache.hadoop.io.Text;
-
-/**
- * A simple interface for creating shell plugins that translate the range and column arguments for
- * the shell's scan command.
- *
- * @deprecated since 2.1.0 This will be removed in a future version in favor of JShell
- */
-@Deprecated(since = "2.1.0")
-public interface ScanInterpreter {
-
- Text interpretRow(Text row);
-
- Text interpretBeginRow(Text row);
-
- Text interpretEndRow(Text row);
-
- Text interpretColumnFamily(Text cf);
-
- Text interpretColumnQualifier(Text cq);
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/json/ByteArrayToBase64TypeAdapter.java b/core/src/main/java/org/apache/accumulo/core/util/json/ByteArrayToBase64TypeAdapter.java
new file mode 100644
index 0000000..2499d78
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/json/ByteArrayToBase64TypeAdapter.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.util.json;
+
+import java.lang.reflect.Type;
+import java.util.Base64;
+import java.util.Base64.Decoder;
+import java.util.Base64.Encoder;
+import java.util.Objects;
+
+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;
+
+/**
+ * Gson adapter to handle serializing and deserializing byte arrays using Base64 encoding.
+ */
+public class ByteArrayToBase64TypeAdapter
+ implements JsonSerializer<byte[]>, JsonDeserializer<byte[]> {
+
+ private static final Decoder decoder = Base64.getUrlDecoder();
+ private static final 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));
+ }
+
+ /**
+ * Creates a new Gson instance that registers {@link ByteArrayToBase64TypeAdapter} for handling
+ * serializing/deserializing byte[] types as Base64 encoded
+ *
+ * @return Gson instance
+ */
+ public static Gson createBase64Gson() {
+ return registerBase64TypeAdapter(new GsonBuilder()).create();
+ }
+
+ /**
+ * Register {@link ByteArrayToBase64TypeAdapter} for handling byte[] types on an existing
+ * GsonBuilder
+ *
+ * @param gsonBuilder existing GsonBuilder
+ * @return GsonBuilder
+ */
+ public static GsonBuilder registerBase64TypeAdapter(final GsonBuilder gsonBuilder) {
+ return Objects.requireNonNull(gsonBuilder).disableHtmlEscaping()
+ .registerTypeHierarchyAdapter(byte[].class, new ByteArrayToBase64TypeAdapter());
+ }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/tables/TableZooHelper.java b/core/src/main/java/org/apache/accumulo/core/util/tables/TableZooHelper.java
index de53d2b..7d8cb85 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/tables/TableZooHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/tables/TableZooHelper.java
@@ -25,7 +25,6 @@
import java.util.List;
import java.util.Objects;
-import java.util.concurrent.ExecutionException;
import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.client.NamespaceNotFoundException;
@@ -37,11 +36,10 @@
import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
public class TableZooHelper implements AutoCloseable {
@@ -49,7 +47,7 @@
// Per instance cache will expire after 10 minutes in case we
// encounter an instance not used frequently
private final Cache<TableZooHelper,TableMap> instanceToMapCache =
- CacheBuilder.newBuilder().expireAfterAccess(10, MINUTES).build();
+ Caffeine.newBuilder().expireAfterAccess(10, MINUTES).build();
public TableZooHelper(ClientContext context) {
this.context = Objects.requireNonNull(context);
@@ -62,11 +60,10 @@
* getCause() of NamespaceNotFoundException
*/
public TableId getTableId(String tableName) throws TableNotFoundException {
- if (MetadataTable.NAME.equals(tableName)) {
- return MetadataTable.ID;
- }
- if (RootTable.NAME.equals(tableName)) {
- return RootTable.ID;
+ for (AccumuloTable systemTable : AccumuloTable.values()) {
+ if (systemTable.tableName().equals(tableName)) {
+ return systemTable.tableId();
+ }
}
try {
return _getTableIdDetectNamespaceNotFound(EXISTING_TABLE_NAME.validate(tableName));
@@ -99,11 +96,10 @@
}
public String getTableName(TableId tableId) throws TableNotFoundException {
- if (MetadataTable.ID.equals(tableId)) {
- return MetadataTable.NAME;
- }
- if (RootTable.ID.equals(tableId)) {
- return RootTable.NAME;
+ for (AccumuloTable systemTable : AccumuloTable.values()) {
+ if (systemTable.tableId().equals(tableId)) {
+ return systemTable.tableName();
+ }
}
String tableName = getTableMap().getIdtoNameMap().get(tableId);
if (tableName == null) {
@@ -127,11 +123,7 @@
}
private TableMap getCachedTableMap() {
- try {
- return instanceToMapCache.get(this, () -> new TableMap(context));
- } catch (ExecutionException e) {
- throw new RuntimeException(e);
- }
+ return instanceToMapCache.get(this, k -> new TableMap(context));
}
public boolean tableNodeExists(TableId tableId) {
@@ -201,7 +193,7 @@
checkArgument(context != null, "instance is null");
checkArgument(tableId != null, "tableId is null");
- if (MetadataTable.ID.equals(tableId) || RootTable.ID.equals(tableId)) {
+ if (AccumuloTable.allTableIds().contains(tableId)) {
return Namespace.ACCUMULO.id();
}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
index ea26c56..324fafe 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
@@ -91,27 +91,24 @@
private static final ConcurrentLinkedQueue<ScheduledFuture<?>> NON_CRITICAL_RUNNING_TASKS =
new ConcurrentLinkedQueue<>();
- private static final Runnable TASK_CHECKER = new Runnable() {
- @Override
- public void run() {
- final List<ConcurrentLinkedQueue<ScheduledFuture<?>>> queues =
- List.of(CRITICAL_RUNNING_TASKS, NON_CRITICAL_RUNNING_TASKS);
- while (true) {
- queues.forEach(q -> {
- Iterator<ScheduledFuture<?>> tasks = q.iterator();
- while (tasks.hasNext()) {
- if (checkTaskFailed(tasks.next(), q)) {
- tasks.remove();
- }
+ private static Runnable TASK_CHECKER = () -> {
+ final List<ConcurrentLinkedQueue<ScheduledFuture<?>>> queues =
+ List.of(CRITICAL_RUNNING_TASKS, NON_CRITICAL_RUNNING_TASKS);
+ while (true) {
+ queues.forEach(q -> {
+ Iterator<ScheduledFuture<?>> tasks = q.iterator();
+ while (tasks.hasNext()) {
+ if (checkTaskFailed(tasks.next(), q)) {
+ tasks.remove();
}
- });
- try {
- MINUTES.sleep(1);
- } catch (InterruptedException ie) {
- // This thread was interrupted by something while sleeping. We don't want to exit
- // this thread, so reset the interrupt state on this thread and keep going.
- Thread.interrupted();
}
+ });
+ try {
+ TimeUnit.MINUTES.sleep(1);
+ } catch (InterruptedException ie) {
+ // This thread was interrupted by something while sleeping. We don't want to exit
+ // this thread, so reset the interrupt state on this thread and keep going.
+ Thread.interrupted();
}
}
};
@@ -266,29 +263,13 @@
* @return ExecutorService impl
* @throws IllegalArgumentException if property is not handled
*/
- @SuppressWarnings("deprecation")
public ThreadPoolExecutor createExecutorService(final AccumuloConfiguration conf,
final Property p, boolean emitThreadPoolMetrics) {
ThreadPoolExecutorBuilder builder;
switch (p) {
- case GENERAL_SIMPLETIMER_THREADPOOL_SIZE:
- return createScheduledExecutorService(conf.getCount(p), "SimpleTimer");
case GENERAL_THREADPOOL_SIZE:
return createScheduledExecutorService(conf.getCount(p), "GeneralExecutor",
emitThreadPoolMetrics);
- case MANAGER_BULK_THREADPOOL_SIZE:
- builder = getPoolBuilder("bulk import").numCoreThreads(conf.getCount(p)).withTimeOut(
- conf.getTimeInMillis(Property.MANAGER_BULK_THREADPOOL_TIMEOUT), MILLISECONDS);
- if (emitThreadPoolMetrics) {
- builder.enableThreadPoolMetrics();
- }
- return builder.build();
- case MANAGER_RENAME_THREADS:
- builder = getPoolBuilder("bulk move").numCoreThreads(conf.getCount(p));
- if (emitThreadPoolMetrics) {
- builder.enableThreadPoolMetrics();
- }
- return builder.build();
case MANAGER_FATE_THREADPOOL_SIZE:
builder = getPoolBuilder("Repo Runner").numCoreThreads(conf.getCount(p));
if (emitThreadPoolMetrics) {
@@ -358,13 +339,6 @@
return builder.build();
case GC_DELETE_THREADS:
return getPoolBuilder("deleting").numCoreThreads(conf.getCount(p)).build();
- case REPLICATION_WORKER_THREADS:
- builder = getPoolBuilder("replication task").numCoreThreads(conf.getCount(p));
- if (emitThreadPoolMetrics) {
- builder.enableThreadPoolMetrics();
- }
- return builder.build();
-
default:
throw new IllegalArgumentException("Unhandled thread pool property: " + p);
}
@@ -566,10 +540,8 @@
*/
public ScheduledThreadPoolExecutor
createGeneralScheduledExecutorService(AccumuloConfiguration conf) {
- @SuppressWarnings("deprecation")
- Property oldProp = Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE;
- Property prop = conf.resolve(Property.GENERAL_THREADPOOL_SIZE, oldProp);
- return (ScheduledThreadPoolExecutor) createExecutorService(conf, prop, true);
+ return (ScheduledThreadPoolExecutor) createExecutorService(conf,
+ Property.GENERAL_THREADPOOL_SIZE, true);
}
/**
diff --git a/core/src/main/java/org/apache/accumulo/core/util/time/NanoTime.java b/core/src/main/java/org/apache/accumulo/core/util/time/NanoTime.java
new file mode 100644
index 0000000..f081278
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/time/NanoTime.java
@@ -0,0 +1,104 @@
+/*
+ * 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.util.time;
+
+import java.time.Duration;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class implements a strong type for System.nanoTime() that offers the limited operations that
+ * can be performed on a nanoTime. See the System.nanoTime() javadoc for details - specifically
+ * these values are meaningful only when the difference between two such values, obtained within the
+ * same instance of a Java virtual machine, are computed.
+ */
+public final class NanoTime implements Comparable<NanoTime> {
+ // In the System.nanoTime javadoc it describes the returned value as the "nanoseconds since some
+ // fixed but arbitrary origin time (perhaps in the future, so values may be negative)". This
+ // variable name is derived from that where AO is arbitrary origin.
+ private final long nanosSinceAO;
+
+ // This method should only be called by test inorder to test edge conditions, that is why it is
+ // package private. Calling this outside of test makes it hard to reason about the correctness of
+ // using this class.
+ @VisibleForTesting
+ NanoTime(long ntsao) {
+ this.nanosSinceAO = ntsao;
+ }
+
+ /**
+ * @return this.nanoTime - other.nanoTime as a Duration
+ */
+ public Duration subtract(NanoTime other) {
+ return Duration.ofNanos(nanosSinceAO - other.nanosSinceAO);
+ }
+
+ /**
+ * Determines the amount of time that has elapsed since this object was created relative to the
+ * current nanoTime.
+ *
+ * @return System.nanoTime() - this.nanoTime
+ */
+ public Duration elapsed() {
+ return Duration.ofNanos(System.nanoTime() - nanosSinceAO);
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (other instanceof NanoTime) {
+ return nanosSinceAO == ((NanoTime) other).nanosSinceAO;
+ }
+
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return Long.hashCode(nanosSinceAO);
+ }
+
+ @Override
+ public int compareTo(NanoTime other) {
+ // All operations w/ nanoTimes must use differences, can not directly compare. This is because a
+ // nano time of Long.MAX_VALUE -10 is considered less than Long.MAX_VALUE +10
+ long diff = nanosSinceAO - other.nanosSinceAO;
+
+ if (diff < 0) {
+ return -1;
+ } else if (diff > 0) {
+ return 1;
+ } else {
+ return 0;
+ }
+ }
+
+ /**
+ * @return a NanoTime created using System.nanoTime()
+ */
+ public static NanoTime now() {
+ return new NanoTime(System.nanoTime());
+ }
+
+ /**
+ * @return a NanoTime created using System.nanoTime() + duration.toNanos()
+ */
+ public static NanoTime nowPlus(Duration duration) {
+ return new NanoTime(System.nanoTime() + duration.toNanos());
+ }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/time/SteadyTime.java b/core/src/main/java/org/apache/accumulo/core/util/time/SteadyTime.java
new file mode 100644
index 0000000..d16f15c
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/time/SteadyTime.java
@@ -0,0 +1,98 @@
+/*
+ * 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.util.time;
+
+import java.time.Duration;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * SteadyTime represents an approximation of the total duration of time this cluster has had a
+ * Manager. Because this represents an elapsed time it is guaranteed to not be negative. SteadyTime
+ * is not expected to represent real world date times, its main use is for computing deltas similar
+ * System.nanoTime but across JVM processes.
+ */
+public class SteadyTime implements Comparable<SteadyTime> {
+
+ private final Duration time;
+
+ private SteadyTime(Duration time) {
+ Preconditions.checkArgument(!time.isNegative(), "SteadyTime '%s' should not be negative.",
+ time.toNanos());
+ this.time = time;
+ }
+
+ public long getMillis() {
+ return time.toMillis();
+ }
+
+ public long getNanos() {
+ return time.toNanos();
+ }
+
+ public Duration getDuration() {
+ return time;
+ }
+
+ public Duration minus(SteadyTime other) {
+ return time.minus(other.getDuration());
+ }
+
+ public SteadyTime plus(Duration other) {
+ return SteadyTime.from(time.plus(other));
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ SteadyTime that = (SteadyTime) o;
+ return Objects.equals(time, that.time);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hashCode(time);
+ }
+
+ @Override
+ public int compareTo(SteadyTime other) {
+ return time.compareTo(other.time);
+ }
+
+ @Override
+ public String toString() {
+ return "SteadyTime[" + time + "]";
+ }
+
+ public static SteadyTime from(long time, TimeUnit unit) {
+ return new SteadyTime(Duration.of(time, unit.toChronoUnit()));
+ }
+
+ public static SteadyTime from(Duration time) {
+ return new SteadyTime(time);
+ }
+
+}
diff --git a/core/src/main/scripts/generate-thrift.sh b/core/src/main/scripts/generate-thrift.sh
index 37929ee..0ad5911 100755
--- a/core/src/main/scripts/generate-thrift.sh
+++ b/core/src/main/scripts/generate-thrift.sh
@@ -32,7 +32,7 @@
[[ -z $REQUIRED_THRIFT_VERSION ]] && REQUIRED_THRIFT_VERSION='0.17.0'
[[ -z $INCLUDED_MODULES ]] && INCLUDED_MODULES=()
[[ -z $BASE_OUTPUT_PACKAGE ]] && BASE_OUTPUT_PACKAGE='org.apache.accumulo.core'
-[[ -z $PACKAGES_TO_GENERATE ]] && PACKAGES_TO_GENERATE=(gc master manager tabletserver securityImpl clientImpl dataImpl replication trace compaction)
+[[ -z $PACKAGES_TO_GENERATE ]] && PACKAGES_TO_GENERATE=(gc manager tabletserver securityImpl clientImpl dataImpl compaction tabletingest tablet tabletscan)
[[ -z $BUILD_DIR ]] && BUILD_DIR='target'
[[ -z $LANGUAGES_TO_GENERATE ]] && LANGUAGES_TO_GENERATE=(java)
[[ -z $FINAL_DIR ]] && FINAL_DIR='src/main'
diff --git a/core/src/main/spotbugs/exclude-filter.xml b/core/src/main/spotbugs/exclude-filter.xml
index cc0bd6a..ded70a9 100644
--- a/core/src/main/spotbugs/exclude-filter.xml
+++ b/core/src/main/spotbugs/exclude-filter.xml
@@ -22,6 +22,7 @@
<!--
DO NOT exclude anything other than generated files here. Other files
can be excluded inline by adding the @SuppressFBWarnings annotation.
+ Exceptions can be made if the bug is particularly spammy or trivial.
-->
<Match>
<!-- ignore thrift-generated files -->
@@ -31,9 +32,10 @@
<Package name="org.apache.accumulo.core.dataImpl.thrift" />
<Package name="org.apache.accumulo.core.gc.thrift" />
<Package name="org.apache.accumulo.core.manager.thrift" />
- <Package name="org.apache.accumulo.core.master.thrift" />
- <Package name="org.apache.accumulo.core.replication.thrift" />
<Package name="org.apache.accumulo.core.securityImpl.thrift" />
+ <Package name="org.apache.accumulo.core.tablet.thrift" />
+ <Package name="org.apache.accumulo.core.tabletingest.thrift" />
+ <Package name="org.apache.accumulo.core.tabletscan.thrift" />
<Package name="org.apache.accumulo.core.tabletserver.thrift" />
<Package name="org.apache.accumulo.core.trace.thrift" />
</Or>
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java
index 91b78f9..64713af 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/ClientService.java
@@ -35,71 +35,69 @@
public java.lang.String getZooKeepers() throws org.apache.thrift.TException;
- public java.util.List<java.lang.String> bulkImportFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
-
- public boolean isActive(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid) throws org.apache.thrift.TException;
+ public boolean isActive(TInfo tinfo, long tid) throws org.apache.thrift.TException;
public void ping(org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
public java.util.List<TDiskUsage> getDiskUsage(java.util.Set<java.lang.String> tables, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
- public java.util.Set<java.lang.String> listLocalUsers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
+ public java.util.Set<java.lang.String> listLocalUsers(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
- public void createLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException;
+ public void createLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException;
- public void dropLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException;
+ public void dropLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException;
- public void changeLocalUserPassword(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException;
+ public void changeLocalUserPassword(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException;
- public boolean authenticate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
+ public boolean authenticate(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
- public boolean authenticateUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws ThriftSecurityException, org.apache.thrift.TException;
+ public boolean authenticateUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws ThriftSecurityException, org.apache.thrift.TException;
- public void changeAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws ThriftSecurityException, org.apache.thrift.TException;
+ public void changeAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws ThriftSecurityException, org.apache.thrift.TException;
- public java.util.List<java.nio.ByteBuffer> getUserAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException;
+ public java.util.List<java.nio.ByteBuffer> getUserAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException;
- public boolean hasSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws ThriftSecurityException, org.apache.thrift.TException;
+ public boolean hasSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws ThriftSecurityException, org.apache.thrift.TException;
- public boolean hasTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+ public boolean hasTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
- public boolean hasNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+ public boolean hasNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
- public void grantSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
+ public void grantSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
- public void revokeSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
+ public void revokeSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
- public void grantTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+ public void grantTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
- public void revokeTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+ public void revokeTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
- public void grantNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+ public void grantNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
- public void revokeNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+ public void revokeNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
- public java.util.Map<java.lang.String,java.lang.String> getConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type) throws ThriftSecurityException, org.apache.thrift.TException;
+ public java.util.Map<java.lang.String,java.lang.String> getConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type) throws ThriftSecurityException, org.apache.thrift.TException;
- public java.util.Map<java.lang.String,java.lang.String> getSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
+ public java.util.Map<java.lang.String,java.lang.String> getSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
- public TVersionedProperties getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
+ public TVersionedProperties getVersionedSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException;
- public java.util.Map<java.lang.String,java.lang.String> getTableConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
+ public java.util.Map<java.lang.String,java.lang.String> getTableConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
- public java.util.Map<java.lang.String,java.lang.String> getTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
+ public java.util.Map<java.lang.String,java.lang.String> getTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
- public TVersionedProperties getVersionedTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
+ public TVersionedProperties getVersionedTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
- public java.util.Map<java.lang.String,java.lang.String> getNamespaceConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
+ public java.util.Map<java.lang.String,java.lang.String> getNamespaceConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
- public java.util.Map<java.lang.String,java.lang.String> getNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
+ public java.util.Map<java.lang.String,java.lang.String> getNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
- public TVersionedProperties getVersionedNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
+ public TVersionedProperties getVersionedNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException;
- public boolean checkClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException;
+ public boolean checkClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException;
- public boolean checkTableClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+ public boolean checkTableClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
- public boolean checkNamespaceClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+ public boolean checkNamespaceClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
}
@@ -111,71 +109,69 @@
public void getZooKeepers(org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
- public void bulkImportFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
-
- public void isActive(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void isActive(TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
public void ping(org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
public void getDiskUsage(java.util.Set<java.lang.String> tables, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TDiskUsage>> resultHandler) throws org.apache.thrift.TException;
- public void listLocalUsers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+ public void listLocalUsers(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
- public void createLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void createLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void dropLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void dropLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void changeLocalUserPassword(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void changeLocalUserPassword(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void authenticate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void authenticate(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
- public void authenticateUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void authenticateUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
- public void changeAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void changeAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void getUserAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler) throws org.apache.thrift.TException;
+ public void getUserAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler) throws org.apache.thrift.TException;
- public void hasSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void hasSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
- public void hasTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void hasTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
- public void hasNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void hasNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
- public void grantSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void grantSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void revokeSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void revokeSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void grantTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void grantTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void revokeTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void revokeTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void grantNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void grantNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void revokeNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void revokeNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void getConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+ public void getConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
- public void getSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+ public void getSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
- public void getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException;
+ public void getVersionedSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException;
- public void getTableConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+ public void getTableConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
- public void getTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+ public void getTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
- public void getVersionedTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException;
+ public void getVersionedTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException;
- public void getNamespaceConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+ public void getNamespaceConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
- public void getNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+ public void getNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException;
- public void getVersionedNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException;
+ public void getVersionedNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException;
- public void checkClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void checkClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
- public void checkTableClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void checkTableClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
- public void checkNamespaceClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void checkNamespaceClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
}
@@ -271,49 +267,13 @@
}
@Override
- public java.util.List<java.lang.String> bulkImportFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
- {
- send_bulkImportFiles(tinfo, credentials, tid, tableId, files, errorDir, setTime);
- return recv_bulkImportFiles();
- }
-
- public void send_bulkImportFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime) throws org.apache.thrift.TException
- {
- bulkImportFiles_args args = new bulkImportFiles_args();
- args.setTinfo(tinfo);
- args.setCredentials(credentials);
- args.setTid(tid);
- args.setTableId(tableId);
- args.setFiles(files);
- args.setErrorDir(errorDir);
- args.setSetTime(setTime);
- sendBase("bulkImportFiles", args);
- }
-
- public java.util.List<java.lang.String> recv_bulkImportFiles() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
- {
- bulkImportFiles_result result = new bulkImportFiles_result();
- receiveBase(result, "bulkImportFiles");
- if (result.isSetSuccess()) {
- return result.success;
- }
- if (result.sec != null) {
- throw result.sec;
- }
- if (result.tope != null) {
- throw result.tope;
- }
- throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "bulkImportFiles failed: unknown result");
- }
-
- @Override
- public boolean isActive(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid) throws org.apache.thrift.TException
+ public boolean isActive(TInfo tinfo, long tid) throws org.apache.thrift.TException
{
send_isActive(tinfo, tid);
return recv_isActive();
}
- public void send_isActive(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid) throws org.apache.thrift.TException
+ public void send_isActive(TInfo tinfo, long tid) throws org.apache.thrift.TException
{
isActive_args args = new isActive_args();
args.setTinfo(tinfo);
@@ -387,13 +347,13 @@
}
@Override
- public java.util.Set<java.lang.String> listLocalUsers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
+ public java.util.Set<java.lang.String> listLocalUsers(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
{
send_listLocalUsers(tinfo, credentials);
return recv_listLocalUsers();
}
- public void send_listLocalUsers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_listLocalUsers(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
listLocalUsers_args args = new listLocalUsers_args();
args.setTinfo(tinfo);
@@ -415,13 +375,13 @@
}
@Override
- public void createLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException
+ public void createLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException
{
send_createLocalUser(tinfo, credentials, principal, password);
recv_createLocalUser();
}
- public void send_createLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws org.apache.thrift.TException
+ public void send_createLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws org.apache.thrift.TException
{
createLocalUser_args args = new createLocalUser_args();
args.setTinfo(tinfo);
@@ -442,13 +402,13 @@
}
@Override
- public void dropLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException
+ public void dropLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException
{
send_dropLocalUser(tinfo, credentials, principal);
recv_dropLocalUser();
}
- public void send_dropLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws org.apache.thrift.TException
+ public void send_dropLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws org.apache.thrift.TException
{
dropLocalUser_args args = new dropLocalUser_args();
args.setTinfo(tinfo);
@@ -468,13 +428,13 @@
}
@Override
- public void changeLocalUserPassword(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException
+ public void changeLocalUserPassword(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws ThriftSecurityException, org.apache.thrift.TException
{
send_changeLocalUserPassword(tinfo, credentials, principal, password);
recv_changeLocalUserPassword();
}
- public void send_changeLocalUserPassword(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws org.apache.thrift.TException
+ public void send_changeLocalUserPassword(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password) throws org.apache.thrift.TException
{
changeLocalUserPassword_args args = new changeLocalUserPassword_args();
args.setTinfo(tinfo);
@@ -495,13 +455,13 @@
}
@Override
- public boolean authenticate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
+ public boolean authenticate(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
{
send_authenticate(tinfo, credentials);
return recv_authenticate();
}
- public void send_authenticate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_authenticate(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
authenticate_args args = new authenticate_args();
args.setTinfo(tinfo);
@@ -523,13 +483,13 @@
}
@Override
- public boolean authenticateUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws ThriftSecurityException, org.apache.thrift.TException
+ public boolean authenticateUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws ThriftSecurityException, org.apache.thrift.TException
{
send_authenticateUser(tinfo, credentials, toAuth);
return recv_authenticateUser();
}
- public void send_authenticateUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws org.apache.thrift.TException
+ public void send_authenticateUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth) throws org.apache.thrift.TException
{
authenticateUser_args args = new authenticateUser_args();
args.setTinfo(tinfo);
@@ -552,13 +512,13 @@
}
@Override
- public void changeAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws ThriftSecurityException, org.apache.thrift.TException
+ public void changeAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws ThriftSecurityException, org.apache.thrift.TException
{
send_changeAuthorizations(tinfo, credentials, principal, authorizations);
recv_changeAuthorizations();
}
- public void send_changeAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws org.apache.thrift.TException
+ public void send_changeAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations) throws org.apache.thrift.TException
{
changeAuthorizations_args args = new changeAuthorizations_args();
args.setTinfo(tinfo);
@@ -579,13 +539,13 @@
}
@Override
- public java.util.List<java.nio.ByteBuffer> getUserAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException
+ public java.util.List<java.nio.ByteBuffer> getUserAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws ThriftSecurityException, org.apache.thrift.TException
{
send_getUserAuthorizations(tinfo, credentials, principal);
return recv_getUserAuthorizations();
}
- public void send_getUserAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws org.apache.thrift.TException
+ public void send_getUserAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal) throws org.apache.thrift.TException
{
getUserAuthorizations_args args = new getUserAuthorizations_args();
args.setTinfo(tinfo);
@@ -608,13 +568,13 @@
}
@Override
- public boolean hasSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws ThriftSecurityException, org.apache.thrift.TException
+ public boolean hasSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws ThriftSecurityException, org.apache.thrift.TException
{
send_hasSystemPermission(tinfo, credentials, principal, sysPerm);
return recv_hasSystemPermission();
}
- public void send_hasSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws org.apache.thrift.TException
+ public void send_hasSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm) throws org.apache.thrift.TException
{
hasSystemPermission_args args = new hasSystemPermission_args();
args.setTinfo(tinfo);
@@ -638,13 +598,13 @@
}
@Override
- public boolean hasTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+ public boolean hasTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
{
send_hasTablePermission(tinfo, credentials, principal, tableName, tblPerm);
return recv_hasTablePermission();
}
- public void send_hasTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws org.apache.thrift.TException
+ public void send_hasTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm) throws org.apache.thrift.TException
{
hasTablePermission_args args = new hasTablePermission_args();
args.setTinfo(tinfo);
@@ -672,13 +632,13 @@
}
@Override
- public boolean hasNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+ public boolean hasNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
{
send_hasNamespacePermission(tinfo, credentials, principal, ns, tblNspcPerm);
return recv_hasNamespacePermission();
}
- public void send_hasNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws org.apache.thrift.TException
+ public void send_hasNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm) throws org.apache.thrift.TException
{
hasNamespacePermission_args args = new hasNamespacePermission_args();
args.setTinfo(tinfo);
@@ -706,13 +666,13 @@
}
@Override
- public void grantSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException
+ public void grantSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException
{
send_grantSystemPermission(tinfo, credentials, principal, permission);
recv_grantSystemPermission();
}
- public void send_grantSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws org.apache.thrift.TException
+ public void send_grantSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws org.apache.thrift.TException
{
grantSystemPermission_args args = new grantSystemPermission_args();
args.setTinfo(tinfo);
@@ -733,13 +693,13 @@
}
@Override
- public void revokeSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException
+ public void revokeSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException
{
send_revokeSystemPermission(tinfo, credentials, principal, permission);
recv_revokeSystemPermission();
}
- public void send_revokeSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws org.apache.thrift.TException
+ public void send_revokeSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission) throws org.apache.thrift.TException
{
revokeSystemPermission_args args = new revokeSystemPermission_args();
args.setTinfo(tinfo);
@@ -760,13 +720,13 @@
}
@Override
- public void grantTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+ public void grantTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
{
send_grantTablePermission(tinfo, credentials, principal, tableName, permission);
recv_grantTablePermission();
}
- public void send_grantTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws org.apache.thrift.TException
+ public void send_grantTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws org.apache.thrift.TException
{
grantTablePermission_args args = new grantTablePermission_args();
args.setTinfo(tinfo);
@@ -791,13 +751,13 @@
}
@Override
- public void revokeTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+ public void revokeTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
{
send_revokeTablePermission(tinfo, credentials, principal, tableName, permission);
recv_revokeTablePermission();
}
- public void send_revokeTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws org.apache.thrift.TException
+ public void send_revokeTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission) throws org.apache.thrift.TException
{
revokeTablePermission_args args = new revokeTablePermission_args();
args.setTinfo(tinfo);
@@ -822,13 +782,13 @@
}
@Override
- public void grantNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+ public void grantNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
{
send_grantNamespacePermission(tinfo, credentials, principal, ns, permission);
recv_grantNamespacePermission();
}
- public void send_grantNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws org.apache.thrift.TException
+ public void send_grantNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws org.apache.thrift.TException
{
grantNamespacePermission_args args = new grantNamespacePermission_args();
args.setTinfo(tinfo);
@@ -853,13 +813,13 @@
}
@Override
- public void revokeNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+ public void revokeNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
{
send_revokeNamespacePermission(tinfo, credentials, principal, ns, permission);
recv_revokeNamespacePermission();
}
- public void send_revokeNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws org.apache.thrift.TException
+ public void send_revokeNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission) throws org.apache.thrift.TException
{
revokeNamespacePermission_args args = new revokeNamespacePermission_args();
args.setTinfo(tinfo);
@@ -884,13 +844,13 @@
}
@Override
- public java.util.Map<java.lang.String,java.lang.String> getConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type) throws ThriftSecurityException, org.apache.thrift.TException
+ public java.util.Map<java.lang.String,java.lang.String> getConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type) throws ThriftSecurityException, org.apache.thrift.TException
{
send_getConfiguration(tinfo, credentials, type);
return recv_getConfiguration();
}
- public void send_getConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type) throws org.apache.thrift.TException
+ public void send_getConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type) throws org.apache.thrift.TException
{
getConfiguration_args args = new getConfiguration_args();
args.setTinfo(tinfo);
@@ -913,13 +873,13 @@
}
@Override
- public java.util.Map<java.lang.String,java.lang.String> getSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
+ public java.util.Map<java.lang.String,java.lang.String> getSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
{
send_getSystemProperties(tinfo, credentials);
return recv_getSystemProperties();
}
- public void send_getSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_getSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
getSystemProperties_args args = new getSystemProperties_args();
args.setTinfo(tinfo);
@@ -941,13 +901,13 @@
}
@Override
- public TVersionedProperties getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
+ public TVersionedProperties getVersionedSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ThriftSecurityException, org.apache.thrift.TException
{
send_getVersionedSystemProperties(tinfo, credentials);
return recv_getVersionedSystemProperties();
}
- public void send_getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_getVersionedSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
getVersionedSystemProperties_args args = new getVersionedSystemProperties_args();
args.setTinfo(tinfo);
@@ -969,13 +929,13 @@
}
@Override
- public java.util.Map<java.lang.String,java.lang.String> getTableConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
+ public java.util.Map<java.lang.String,java.lang.String> getTableConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
{
send_getTableConfiguration(tinfo, credentials, tableName);
return recv_getTableConfiguration();
}
- public void send_getTableConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
+ public void send_getTableConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
{
getTableConfiguration_args args = new getTableConfiguration_args();
args.setTinfo(tinfo);
@@ -1001,13 +961,13 @@
}
@Override
- public java.util.Map<java.lang.String,java.lang.String> getTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
+ public java.util.Map<java.lang.String,java.lang.String> getTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
{
send_getTableProperties(tinfo, credentials, tableName);
return recv_getTableProperties();
}
- public void send_getTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
+ public void send_getTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
{
getTableProperties_args args = new getTableProperties_args();
args.setTinfo(tinfo);
@@ -1033,13 +993,13 @@
}
@Override
- public TVersionedProperties getVersionedTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
+ public TVersionedProperties getVersionedTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
{
send_getVersionedTableProperties(tinfo, credentials, tableName);
return recv_getVersionedTableProperties();
}
- public void send_getVersionedTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
+ public void send_getVersionedTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
{
getVersionedTableProperties_args args = new getVersionedTableProperties_args();
args.setTinfo(tinfo);
@@ -1065,13 +1025,13 @@
}
@Override
- public java.util.Map<java.lang.String,java.lang.String> getNamespaceConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
+ public java.util.Map<java.lang.String,java.lang.String> getNamespaceConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
{
send_getNamespaceConfiguration(tinfo, credentials, ns);
return recv_getNamespaceConfiguration();
}
- public void send_getNamespaceConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws org.apache.thrift.TException
+ public void send_getNamespaceConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws org.apache.thrift.TException
{
getNamespaceConfiguration_args args = new getNamespaceConfiguration_args();
args.setTinfo(tinfo);
@@ -1097,13 +1057,13 @@
}
@Override
- public java.util.Map<java.lang.String,java.lang.String> getNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
+ public java.util.Map<java.lang.String,java.lang.String> getNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
{
send_getNamespaceProperties(tinfo, credentials, ns);
return recv_getNamespaceProperties();
}
- public void send_getNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws org.apache.thrift.TException
+ public void send_getNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws org.apache.thrift.TException
{
getNamespaceProperties_args args = new getNamespaceProperties_args();
args.setTinfo(tinfo);
@@ -1129,13 +1089,13 @@
}
@Override
- public TVersionedProperties getVersionedNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
+ public TVersionedProperties getVersionedNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws ThriftTableOperationException, ThriftSecurityException, org.apache.thrift.TException
{
send_getVersionedNamespaceProperties(tinfo, credentials, ns);
return recv_getVersionedNamespaceProperties();
}
- public void send_getVersionedNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws org.apache.thrift.TException
+ public void send_getVersionedNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns) throws org.apache.thrift.TException
{
getVersionedNamespaceProperties_args args = new getVersionedNamespaceProperties_args();
args.setTinfo(tinfo);
@@ -1161,13 +1121,13 @@
}
@Override
- public boolean checkClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
+ public boolean checkClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
{
send_checkClass(tinfo, credentials, className, interfaceMatch);
return recv_checkClass();
}
- public void send_checkClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
+ public void send_checkClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
{
checkClass_args args = new checkClass_args();
args.setTinfo(tinfo);
@@ -1188,13 +1148,13 @@
}
@Override
- public boolean checkTableClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+ public boolean checkTableClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
{
send_checkTableClass(tinfo, credentials, tableId, className, interfaceMatch);
return recv_checkTableClass();
}
- public void send_checkTableClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
+ public void send_checkTableClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
{
checkTableClass_args args = new checkTableClass_args();
args.setTinfo(tinfo);
@@ -1222,13 +1182,13 @@
}
@Override
- public boolean checkNamespaceClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+ public boolean checkNamespaceClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
{
send_checkNamespaceClass(tinfo, credentials, namespaceId, className, interfaceMatch);
return recv_checkNamespaceClass();
}
- public void send_checkNamespaceClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
+ public void send_checkNamespaceClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch) throws org.apache.thrift.TException
{
checkNamespaceClass_args args = new checkNamespaceClass_args();
args.setTinfo(tinfo);
@@ -1371,60 +1331,7 @@
}
@Override
- public void bulkImportFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
- checkReady();
- bulkImportFiles_call method_call = new bulkImportFiles_call(tinfo, credentials, tid, tableId, files, errorDir, setTime, resultHandler, this, ___protocolFactory, ___transport);
- this.___currentMethod = method_call;
- ___manager.call(method_call);
- }
-
- public static class bulkImportFiles_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<java.lang.String>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
- private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- private long tid;
- private java.lang.String tableId;
- private java.util.List<java.lang.String> files;
- private java.lang.String errorDir;
- private boolean setTime;
- public bulkImportFiles_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String tableId, java.util.List<java.lang.String> files, java.lang.String errorDir, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
- super(client, protocolFactory, transport, resultHandler, false);
- this.tinfo = tinfo;
- this.credentials = credentials;
- this.tid = tid;
- this.tableId = tableId;
- this.files = files;
- this.errorDir = errorDir;
- this.setTime = setTime;
- }
-
- @Override
- public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
- prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("bulkImportFiles", org.apache.thrift.protocol.TMessageType.CALL, 0));
- bulkImportFiles_args args = new bulkImportFiles_args();
- args.setTinfo(tinfo);
- args.setCredentials(credentials);
- args.setTid(tid);
- args.setTableId(tableId);
- args.setFiles(files);
- args.setErrorDir(errorDir);
- args.setSetTime(setTime);
- args.write(prot);
- prot.writeMessageEnd();
- }
-
- @Override
- public java.util.List<java.lang.String> getResult() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException {
- if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
- throw new java.lang.IllegalStateException("Method call not finished!");
- }
- org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
- org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
- return (new Client(prot)).recv_bulkImportFiles();
- }
- }
-
- @Override
- public void isActive(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ public void isActive(TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
checkReady();
isActive_call method_call = new isActive_call(tinfo, tid, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1432,9 +1339,9 @@
}
public static class isActive_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private long tid;
- public isActive_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public isActive_call(TInfo tinfo, long tid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.tid = tid;
@@ -1536,7 +1443,7 @@
}
@Override
- public void listLocalUsers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+ public void listLocalUsers(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
checkReady();
listLocalUsers_call method_call = new listLocalUsers_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1544,9 +1451,9 @@
}
public static class listLocalUsers_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Set<java.lang.String>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public listLocalUsers_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public listLocalUsers_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Set<java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1574,7 +1481,7 @@
}
@Override
- public void createLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void createLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
createLocalUser_call method_call = new createLocalUser_call(tinfo, credentials, principal, password, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1582,11 +1489,11 @@
}
public static class createLocalUser_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private java.nio.ByteBuffer password;
- public createLocalUser_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public createLocalUser_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1619,7 +1526,7 @@
}
@Override
- public void dropLocalUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void dropLocalUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
dropLocalUser_call method_call = new dropLocalUser_call(tinfo, credentials, principal, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1627,10 +1534,10 @@
}
public static class dropLocalUser_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
- public dropLocalUser_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public dropLocalUser_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1661,7 +1568,7 @@
}
@Override
- public void changeLocalUserPassword(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void changeLocalUserPassword(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
changeLocalUserPassword_call method_call = new changeLocalUserPassword_call(tinfo, credentials, principal, password, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1669,11 +1576,11 @@
}
public static class changeLocalUserPassword_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private java.nio.ByteBuffer password;
- public changeLocalUserPassword_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public changeLocalUserPassword_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.nio.ByteBuffer password, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1706,7 +1613,7 @@
}
@Override
- public void authenticate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ public void authenticate(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
checkReady();
authenticate_call method_call = new authenticate_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1714,9 +1621,9 @@
}
public static class authenticate_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public authenticate_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public authenticate_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1744,7 +1651,7 @@
}
@Override
- public void authenticateUser(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ public void authenticateUser(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
checkReady();
authenticateUser_call method_call = new authenticateUser_call(tinfo, credentials, toAuth, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1752,10 +1659,10 @@
}
public static class authenticateUser_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth;
- public authenticateUser_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public authenticateUser_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1785,7 +1692,7 @@
}
@Override
- public void changeAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void changeAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
changeAuthorizations_call method_call = new changeAuthorizations_call(tinfo, credentials, principal, authorizations, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1793,11 +1700,11 @@
}
public static class changeAuthorizations_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private java.util.List<java.nio.ByteBuffer> authorizations;
- public changeAuthorizations_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public changeAuthorizations_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.util.List<java.nio.ByteBuffer> authorizations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1830,7 +1737,7 @@
}
@Override
- public void getUserAuthorizations(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler) throws org.apache.thrift.TException {
+ public void getUserAuthorizations(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler) throws org.apache.thrift.TException {
checkReady();
getUserAuthorizations_call method_call = new getUserAuthorizations_call(tinfo, credentials, principal, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1838,10 +1745,10 @@
}
public static class getUserAuthorizations_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<java.nio.ByteBuffer>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
- public getUserAuthorizations_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getUserAuthorizations_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.nio.ByteBuffer>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1871,7 +1778,7 @@
}
@Override
- public void hasSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ public void hasSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
checkReady();
hasSystemPermission_call method_call = new hasSystemPermission_call(tinfo, credentials, principal, sysPerm, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1879,11 +1786,11 @@
}
public static class hasSystemPermission_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private byte sysPerm;
- public hasSystemPermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public hasSystemPermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte sysPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1915,7 +1822,7 @@
}
@Override
- public void hasTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ public void hasTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
checkReady();
hasTablePermission_call method_call = new hasTablePermission_call(tinfo, credentials, principal, tableName, tblPerm, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1923,12 +1830,12 @@
}
public static class hasTablePermission_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private java.lang.String tableName;
private byte tblPerm;
- public hasTablePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public hasTablePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1962,7 +1869,7 @@
}
@Override
- public void hasNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ public void hasNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
checkReady();
hasNamespacePermission_call method_call = new hasNamespacePermission_call(tinfo, credentials, principal, ns, tblNspcPerm, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1970,12 +1877,12 @@
}
public static class hasNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private java.lang.String ns;
private byte tblNspcPerm;
- public hasNamespacePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public hasNamespacePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2009,7 +1916,7 @@
}
@Override
- public void grantSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void grantSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
grantSystemPermission_call method_call = new grantSystemPermission_call(tinfo, credentials, principal, permission, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2017,11 +1924,11 @@
}
public static class grantSystemPermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private byte permission;
- public grantSystemPermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public grantSystemPermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2054,7 +1961,7 @@
}
@Override
- public void revokeSystemPermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void revokeSystemPermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
revokeSystemPermission_call method_call = new revokeSystemPermission_call(tinfo, credentials, principal, permission, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2062,11 +1969,11 @@
}
public static class revokeSystemPermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private byte permission;
- public revokeSystemPermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public revokeSystemPermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2099,7 +2006,7 @@
}
@Override
- public void grantTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void grantTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
grantTablePermission_call method_call = new grantTablePermission_call(tinfo, credentials, principal, tableName, permission, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2107,12 +2014,12 @@
}
public static class grantTablePermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private java.lang.String tableName;
private byte permission;
- public grantTablePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public grantTablePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2147,7 +2054,7 @@
}
@Override
- public void revokeTablePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void revokeTablePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
revokeTablePermission_call method_call = new revokeTablePermission_call(tinfo, credentials, principal, tableName, permission, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2155,12 +2062,12 @@
}
public static class revokeTablePermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private java.lang.String tableName;
private byte permission;
- public revokeTablePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public revokeTablePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2195,7 +2102,7 @@
}
@Override
- public void grantNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void grantNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
grantNamespacePermission_call method_call = new grantNamespacePermission_call(tinfo, credentials, principal, ns, permission, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2203,12 +2110,12 @@
}
public static class grantNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private java.lang.String ns;
private byte permission;
- public grantNamespacePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public grantNamespacePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2243,7 +2150,7 @@
}
@Override
- public void revokeNamespacePermission(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void revokeNamespacePermission(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
revokeNamespacePermission_call method_call = new revokeNamespacePermission_call(tinfo, credentials, principal, ns, permission, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2251,12 +2158,12 @@
}
public static class revokeNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String principal;
private java.lang.String ns;
private byte permission;
- public revokeNamespacePermission_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public revokeNamespacePermission_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String principal, java.lang.String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2291,7 +2198,7 @@
}
@Override
- public void getConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+ public void getConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
checkReady();
getConfiguration_call method_call = new getConfiguration_call(tinfo, credentials, type, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2299,10 +2206,10 @@
}
public static class getConfiguration_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Map<java.lang.String,java.lang.String>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private ConfigurationType type;
- public getConfiguration_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getConfiguration_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ConfigurationType type, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2332,7 +2239,7 @@
}
@Override
- public void getSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+ public void getSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
checkReady();
getSystemProperties_call method_call = new getSystemProperties_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2340,9 +2247,9 @@
}
public static class getSystemProperties_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Map<java.lang.String,java.lang.String>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getSystemProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getSystemProperties_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2370,7 +2277,7 @@
}
@Override
- public void getVersionedSystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
+ public void getVersionedSystemProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
checkReady();
getVersionedSystemProperties_call method_call = new getVersionedSystemProperties_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2378,9 +2285,9 @@
}
public static class getVersionedSystemProperties_call extends org.apache.thrift.async.TAsyncMethodCall<TVersionedProperties> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getVersionedSystemProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getVersionedSystemProperties_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2408,7 +2315,7 @@
}
@Override
- public void getTableConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+ public void getTableConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
checkReady();
getTableConfiguration_call method_call = new getTableConfiguration_call(tinfo, credentials, tableName, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2416,10 +2323,10 @@
}
public static class getTableConfiguration_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Map<java.lang.String,java.lang.String>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String tableName;
- public getTableConfiguration_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getTableConfiguration_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2449,7 +2356,7 @@
}
@Override
- public void getTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+ public void getTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
checkReady();
getTableProperties_call method_call = new getTableProperties_call(tinfo, credentials, tableName, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2457,10 +2364,10 @@
}
public static class getTableProperties_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Map<java.lang.String,java.lang.String>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String tableName;
- public getTableProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getTableProperties_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2490,7 +2397,7 @@
}
@Override
- public void getVersionedTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
+ public void getVersionedTableProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
checkReady();
getVersionedTableProperties_call method_call = new getVersionedTableProperties_call(tinfo, credentials, tableName, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2498,10 +2405,10 @@
}
public static class getVersionedTableProperties_call extends org.apache.thrift.async.TAsyncMethodCall<TVersionedProperties> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String tableName;
- public getVersionedTableProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getVersionedTableProperties_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2531,7 +2438,7 @@
}
@Override
- public void getNamespaceConfiguration(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+ public void getNamespaceConfiguration(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
checkReady();
getNamespaceConfiguration_call method_call = new getNamespaceConfiguration_call(tinfo, credentials, ns, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2539,10 +2446,10 @@
}
public static class getNamespaceConfiguration_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Map<java.lang.String,java.lang.String>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String ns;
- public getNamespaceConfiguration_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getNamespaceConfiguration_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2572,7 +2479,7 @@
}
@Override
- public void getNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+ public void getNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler) throws org.apache.thrift.TException {
checkReady();
getNamespaceProperties_call method_call = new getNamespaceProperties_call(tinfo, credentials, ns, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2580,10 +2487,10 @@
}
public static class getNamespaceProperties_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.Map<java.lang.String,java.lang.String>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String ns;
- public getNamespaceProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getNamespaceProperties_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<java.util.Map<java.lang.String,java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2613,7 +2520,7 @@
}
@Override
- public void getVersionedNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
+ public void getVersionedNamespaceProperties(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler) throws org.apache.thrift.TException {
checkReady();
getVersionedNamespaceProperties_call method_call = new getVersionedNamespaceProperties_call(tinfo, credentials, ns, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2621,10 +2528,10 @@
}
public static class getVersionedNamespaceProperties_call extends org.apache.thrift.async.TAsyncMethodCall<TVersionedProperties> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String ns;
- public getVersionedNamespaceProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getVersionedNamespaceProperties_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.thrift.async.AsyncMethodCallback<TVersionedProperties> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2654,7 +2561,7 @@
}
@Override
- public void checkClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ public void checkClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
checkReady();
checkClass_call method_call = new checkClass_call(tinfo, credentials, className, interfaceMatch, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2662,11 +2569,11 @@
}
public static class checkClass_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String className;
private java.lang.String interfaceMatch;
- public checkClass_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public checkClass_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2698,7 +2605,7 @@
}
@Override
- public void checkTableClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ public void checkTableClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
checkReady();
checkTableClass_call method_call = new checkTableClass_call(tinfo, credentials, tableId, className, interfaceMatch, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2706,12 +2613,12 @@
}
public static class checkTableClass_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String tableId;
private java.lang.String className;
private java.lang.String interfaceMatch;
- public checkTableClass_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public checkTableClass_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2745,7 +2652,7 @@
}
@Override
- public void checkNamespaceClass(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ public void checkNamespaceClass(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
checkReady();
checkNamespaceClass_call method_call = new checkNamespaceClass_call(tinfo, credentials, namespaceId, className, interfaceMatch, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -2753,12 +2660,12 @@
}
public static class checkNamespaceClass_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String namespaceId;
private java.lang.String className;
private java.lang.String interfaceMatch;
- public checkNamespaceClass_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public checkNamespaceClass_call(TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String namespaceId, java.lang.String className, java.lang.String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -2807,7 +2714,6 @@
processMap.put("getRootTabletLocation", new getRootTabletLocation());
processMap.put("getInstanceId", new getInstanceId());
processMap.put("getZooKeepers", new getZooKeepers());
- processMap.put("bulkImportFiles", new bulkImportFiles());
processMap.put("isActive", new isActive());
processMap.put("ping", new ping());
processMap.put("getDiskUsage", new getDiskUsage());
@@ -2927,40 +2833,6 @@
}
}
- public static class bulkImportFiles<I extends Iface> extends org.apache.thrift.ProcessFunction<I, bulkImportFiles_args> {
- public bulkImportFiles() {
- super("bulkImportFiles");
- }
-
- @Override
- public bulkImportFiles_args getEmptyArgsInstance() {
- return new bulkImportFiles_args();
- }
-
- @Override
- protected boolean isOneway() {
- return false;
- }
-
- @Override
- protected boolean rethrowUnhandledExceptions() {
- return false;
- }
-
- @Override
- public bulkImportFiles_result getResult(I iface, bulkImportFiles_args args) throws org.apache.thrift.TException {
- bulkImportFiles_result result = new bulkImportFiles_result();
- try {
- result.success = iface.bulkImportFiles(args.tinfo, args.credentials, args.tid, args.tableId, args.files, args.errorDir, args.setTime);
- } catch (ThriftSecurityException sec) {
- result.sec = sec;
- } catch (ThriftTableOperationException tope) {
- result.tope = tope;
- }
- return result;
- }
- }
-
public static class isActive<I extends Iface> extends org.apache.thrift.ProcessFunction<I, isActive_args> {
public isActive() {
super("isActive");
@@ -4032,7 +3904,6 @@
processMap.put("getRootTabletLocation", new getRootTabletLocation());
processMap.put("getInstanceId", new getInstanceId());
processMap.put("getZooKeepers", new getZooKeepers());
- processMap.put("bulkImportFiles", new bulkImportFiles());
processMap.put("isActive", new isActive());
processMap.put("ping", new ping());
processMap.put("getDiskUsage", new getDiskUsage());
@@ -4269,81 +4140,6 @@
}
}
- public static class bulkImportFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, bulkImportFiles_args, java.util.List<java.lang.String>> {
- public bulkImportFiles() {
- super("bulkImportFiles");
- }
-
- @Override
- public bulkImportFiles_args getEmptyArgsInstance() {
- return new bulkImportFiles_args();
- }
-
- @Override
- public org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
- final org.apache.thrift.AsyncProcessFunction fcall = this;
- return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>>() {
- @Override
- public void onComplete(java.util.List<java.lang.String> o) {
- bulkImportFiles_result result = new bulkImportFiles_result();
- result.success = o;
- try {
- fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
- } catch (org.apache.thrift.transport.TTransportException e) {
- _LOGGER.error("TTransportException writing to internal frame buffer", e);
- fb.close();
- } catch (java.lang.Exception e) {
- _LOGGER.error("Exception writing to internal frame buffer", e);
- onError(e);
- }
- }
- @Override
- public void onError(java.lang.Exception e) {
- byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
- org.apache.thrift.TSerializable msg;
- bulkImportFiles_result result = new bulkImportFiles_result();
- if (e instanceof ThriftSecurityException) {
- result.sec = (ThriftSecurityException) e;
- result.setSecIsSet(true);
- msg = result;
- } else if (e instanceof ThriftTableOperationException) {
- result.tope = (ThriftTableOperationException) e;
- result.setTopeIsSet(true);
- msg = result;
- } else if (e instanceof org.apache.thrift.transport.TTransportException) {
- _LOGGER.error("TTransportException inside handler", e);
- fb.close();
- return;
- } else if (e instanceof org.apache.thrift.TApplicationException) {
- _LOGGER.error("TApplicationException inside handler", e);
- msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
- msg = (org.apache.thrift.TApplicationException)e;
- } else {
- _LOGGER.error("Exception inside handler", e);
- msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
- msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
- }
- try {
- fcall.sendResponse(fb,msg,msgType,seqid);
- } catch (java.lang.Exception ex) {
- _LOGGER.error("Exception writing to internal frame buffer", ex);
- fb.close();
- }
- }
- };
- }
-
- @Override
- protected boolean isOneway() {
- return false;
- }
-
- @Override
- public void start(I iface, bulkImportFiles_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
- iface.bulkImportFiles(args.tinfo, args.credentials, args.tid, args.tableId, args.files, args.errorDir, args.setTime,resultHandler);
- }
- }
-
public static class isActive<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, isActive_args, java.lang.Boolean> {
public isActive() {
super("isActive");
@@ -8601,1703 +8397,6 @@
}
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
- public static class bulkImportFiles_args implements org.apache.thrift.TBase<bulkImportFiles_args, bulkImportFiles_args._Fields>, java.io.Serializable, Cloneable, Comparable<bulkImportFiles_args> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("bulkImportFiles_args");
-
- private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
- private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)8);
- private static final org.apache.thrift.protocol.TField TID_FIELD_DESC = new org.apache.thrift.protocol.TField("tid", org.apache.thrift.protocol.TType.I64, (short)3);
- private static final org.apache.thrift.protocol.TField TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableId", org.apache.thrift.protocol.TType.STRING, (short)4);
- private static final org.apache.thrift.protocol.TField FILES_FIELD_DESC = new org.apache.thrift.protocol.TField("files", org.apache.thrift.protocol.TType.LIST, (short)5);
- private static final org.apache.thrift.protocol.TField ERROR_DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("errorDir", org.apache.thrift.protocol.TType.STRING, (short)6);
- private static final org.apache.thrift.protocol.TField SET_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("setTime", org.apache.thrift.protocol.TType.BOOL, (short)7);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new bulkImportFiles_argsStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new bulkImportFiles_argsTupleSchemeFactory();
-
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
- public long tid; // required
- public @org.apache.thrift.annotation.Nullable java.lang.String tableId; // required
- public @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> files; // required
- public @org.apache.thrift.annotation.Nullable java.lang.String errorDir; // required
- public boolean setTime; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- TINFO((short)1, "tinfo"),
- CREDENTIALS((short)8, "credentials"),
- TID((short)3, "tid"),
- TABLE_ID((short)4, "tableId"),
- FILES((short)5, "files"),
- ERROR_DIR((short)6, "errorDir"),
- SET_TIME((short)7, "setTime");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 1: // TINFO
- return TINFO;
- case 8: // CREDENTIALS
- return CREDENTIALS;
- case 3: // TID
- return TID;
- case 4: // TABLE_ID
- return TABLE_ID;
- case 5: // FILES
- return FILES;
- case 6: // ERROR_DIR
- return ERROR_DIR;
- case 7: // SET_TIME
- return SET_TIME;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- private static final int __TID_ISSET_ID = 0;
- private static final int __SETTIME_ISSET_ID = 1;
- private byte __isset_bitfield = 0;
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
- tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
- tmpMap.put(_Fields.TID, new org.apache.thrift.meta_data.FieldMetaData("tid", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
- tmpMap.put(_Fields.TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("tableId", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
- tmpMap.put(_Fields.FILES, new org.apache.thrift.meta_data.FieldMetaData("files", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
- tmpMap.put(_Fields.ERROR_DIR, new org.apache.thrift.meta_data.FieldMetaData("errorDir", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
- tmpMap.put(_Fields.SET_TIME, new org.apache.thrift.meta_data.FieldMetaData("setTime", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(bulkImportFiles_args.class, metaDataMap);
- }
-
- public bulkImportFiles_args() {
- }
-
- public bulkImportFiles_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
- org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
- long tid,
- java.lang.String tableId,
- java.util.List<java.lang.String> files,
- java.lang.String errorDir,
- boolean setTime)
- {
- this();
- this.tinfo = tinfo;
- this.credentials = credentials;
- this.tid = tid;
- setTidIsSet(true);
- this.tableId = tableId;
- this.files = files;
- this.errorDir = errorDir;
- this.setTime = setTime;
- setSetTimeIsSet(true);
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public bulkImportFiles_args(bulkImportFiles_args other) {
- __isset_bitfield = other.__isset_bitfield;
- if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
- }
- if (other.isSetCredentials()) {
- this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
- }
- this.tid = other.tid;
- if (other.isSetTableId()) {
- this.tableId = other.tableId;
- }
- if (other.isSetFiles()) {
- java.util.List<java.lang.String> __this__files = new java.util.ArrayList<java.lang.String>(other.files);
- this.files = __this__files;
- }
- if (other.isSetErrorDir()) {
- this.errorDir = other.errorDir;
- }
- this.setTime = other.setTime;
- }
-
- @Override
- public bulkImportFiles_args deepCopy() {
- return new bulkImportFiles_args(this);
- }
-
- @Override
- public void clear() {
- this.tinfo = null;
- this.credentials = null;
- setTidIsSet(false);
- this.tid = 0;
- this.tableId = null;
- this.files = null;
- this.errorDir = null;
- setSetTimeIsSet(false);
- this.setTime = false;
- }
-
- @org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
- return this.tinfo;
- }
-
- public bulkImportFiles_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
- this.tinfo = tinfo;
- return this;
- }
-
- public void unsetTinfo() {
- this.tinfo = null;
- }
-
- /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
- public boolean isSetTinfo() {
- return this.tinfo != null;
- }
-
- public void setTinfoIsSet(boolean value) {
- if (!value) {
- this.tinfo = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
- return this.credentials;
- }
-
- public bulkImportFiles_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
- this.credentials = credentials;
- return this;
- }
-
- public void unsetCredentials() {
- this.credentials = null;
- }
-
- /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
- public boolean isSetCredentials() {
- return this.credentials != null;
- }
-
- public void setCredentialsIsSet(boolean value) {
- if (!value) {
- this.credentials = null;
- }
- }
-
- public long getTid() {
- return this.tid;
- }
-
- public bulkImportFiles_args setTid(long tid) {
- this.tid = tid;
- setTidIsSet(true);
- return this;
- }
-
- public void unsetTid() {
- __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TID_ISSET_ID);
- }
-
- /** Returns true if field tid is set (has been assigned a value) and false otherwise */
- public boolean isSetTid() {
- return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TID_ISSET_ID);
- }
-
- public void setTidIsSet(boolean value) {
- __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TID_ISSET_ID, value);
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.lang.String getTableId() {
- return this.tableId;
- }
-
- public bulkImportFiles_args setTableId(@org.apache.thrift.annotation.Nullable java.lang.String tableId) {
- this.tableId = tableId;
- return this;
- }
-
- public void unsetTableId() {
- this.tableId = null;
- }
-
- /** Returns true if field tableId is set (has been assigned a value) and false otherwise */
- public boolean isSetTableId() {
- return this.tableId != null;
- }
-
- public void setTableIdIsSet(boolean value) {
- if (!value) {
- this.tableId = null;
- }
- }
-
- public int getFilesSize() {
- return (this.files == null) ? 0 : this.files.size();
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.util.Iterator<java.lang.String> getFilesIterator() {
- return (this.files == null) ? null : this.files.iterator();
- }
-
- public void addToFiles(java.lang.String elem) {
- if (this.files == null) {
- this.files = new java.util.ArrayList<java.lang.String>();
- }
- this.files.add(elem);
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.util.List<java.lang.String> getFiles() {
- return this.files;
- }
-
- public bulkImportFiles_args setFiles(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> files) {
- this.files = files;
- return this;
- }
-
- public void unsetFiles() {
- this.files = null;
- }
-
- /** Returns true if field files is set (has been assigned a value) and false otherwise */
- public boolean isSetFiles() {
- return this.files != null;
- }
-
- public void setFilesIsSet(boolean value) {
- if (!value) {
- this.files = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.lang.String getErrorDir() {
- return this.errorDir;
- }
-
- public bulkImportFiles_args setErrorDir(@org.apache.thrift.annotation.Nullable java.lang.String errorDir) {
- this.errorDir = errorDir;
- return this;
- }
-
- public void unsetErrorDir() {
- this.errorDir = null;
- }
-
- /** Returns true if field errorDir is set (has been assigned a value) and false otherwise */
- public boolean isSetErrorDir() {
- return this.errorDir != null;
- }
-
- public void setErrorDirIsSet(boolean value) {
- if (!value) {
- this.errorDir = null;
- }
- }
-
- public boolean isSetTime() {
- return this.setTime;
- }
-
- public bulkImportFiles_args setSetTime(boolean setTime) {
- this.setTime = setTime;
- setSetTimeIsSet(true);
- return this;
- }
-
- public void unsetSetTime() {
- __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SETTIME_ISSET_ID);
- }
-
- /** Returns true if field setTime is set (has been assigned a value) and false otherwise */
- public boolean isSetSetTime() {
- return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SETTIME_ISSET_ID);
- }
-
- public void setSetTimeIsSet(boolean value) {
- __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SETTIME_ISSET_ID, value);
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case TINFO:
- if (value == null) {
- unsetTinfo();
- } else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
- }
- break;
-
- case CREDENTIALS:
- if (value == null) {
- unsetCredentials();
- } else {
- setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
- }
- break;
-
- case TID:
- if (value == null) {
- unsetTid();
- } else {
- setTid((java.lang.Long)value);
- }
- break;
-
- case TABLE_ID:
- if (value == null) {
- unsetTableId();
- } else {
- setTableId((java.lang.String)value);
- }
- break;
-
- case FILES:
- if (value == null) {
- unsetFiles();
- } else {
- setFiles((java.util.List<java.lang.String>)value);
- }
- break;
-
- case ERROR_DIR:
- if (value == null) {
- unsetErrorDir();
- } else {
- setErrorDir((java.lang.String)value);
- }
- break;
-
- case SET_TIME:
- if (value == null) {
- unsetSetTime();
- } else {
- setSetTime((java.lang.Boolean)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case TINFO:
- return getTinfo();
-
- case CREDENTIALS:
- return getCredentials();
-
- case TID:
- return getTid();
-
- case TABLE_ID:
- return getTableId();
-
- case FILES:
- return getFiles();
-
- case ERROR_DIR:
- return getErrorDir();
-
- case SET_TIME:
- return isSetTime();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case TINFO:
- return isSetTinfo();
- case CREDENTIALS:
- return isSetCredentials();
- case TID:
- return isSetTid();
- case TABLE_ID:
- return isSetTableId();
- case FILES:
- return isSetFiles();
- case ERROR_DIR:
- return isSetErrorDir();
- case SET_TIME:
- return isSetSetTime();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof bulkImportFiles_args)
- return this.equals((bulkImportFiles_args)that);
- return false;
- }
-
- public boolean equals(bulkImportFiles_args that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_tinfo = true && this.isSetTinfo();
- boolean that_present_tinfo = true && that.isSetTinfo();
- if (this_present_tinfo || that_present_tinfo) {
- if (!(this_present_tinfo && that_present_tinfo))
- return false;
- if (!this.tinfo.equals(that.tinfo))
- return false;
- }
-
- boolean this_present_credentials = true && this.isSetCredentials();
- boolean that_present_credentials = true && that.isSetCredentials();
- if (this_present_credentials || that_present_credentials) {
- if (!(this_present_credentials && that_present_credentials))
- return false;
- if (!this.credentials.equals(that.credentials))
- return false;
- }
-
- boolean this_present_tid = true;
- boolean that_present_tid = true;
- if (this_present_tid || that_present_tid) {
- if (!(this_present_tid && that_present_tid))
- return false;
- if (this.tid != that.tid)
- return false;
- }
-
- boolean this_present_tableId = true && this.isSetTableId();
- boolean that_present_tableId = true && that.isSetTableId();
- if (this_present_tableId || that_present_tableId) {
- if (!(this_present_tableId && that_present_tableId))
- return false;
- if (!this.tableId.equals(that.tableId))
- return false;
- }
-
- boolean this_present_files = true && this.isSetFiles();
- boolean that_present_files = true && that.isSetFiles();
- if (this_present_files || that_present_files) {
- if (!(this_present_files && that_present_files))
- return false;
- if (!this.files.equals(that.files))
- return false;
- }
-
- boolean this_present_errorDir = true && this.isSetErrorDir();
- boolean that_present_errorDir = true && that.isSetErrorDir();
- if (this_present_errorDir || that_present_errorDir) {
- if (!(this_present_errorDir && that_present_errorDir))
- return false;
- if (!this.errorDir.equals(that.errorDir))
- return false;
- }
-
- boolean this_present_setTime = true;
- boolean that_present_setTime = true;
- if (this_present_setTime || that_present_setTime) {
- if (!(this_present_setTime && that_present_setTime))
- return false;
- if (this.setTime != that.setTime)
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
- if (isSetTinfo())
- hashCode = hashCode * 8191 + tinfo.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
- if (isSetCredentials())
- hashCode = hashCode * 8191 + credentials.hashCode();
-
- hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(tid);
-
- hashCode = hashCode * 8191 + ((isSetTableId()) ? 131071 : 524287);
- if (isSetTableId())
- hashCode = hashCode * 8191 + tableId.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetFiles()) ? 131071 : 524287);
- if (isSetFiles())
- hashCode = hashCode * 8191 + files.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetErrorDir()) ? 131071 : 524287);
- if (isSetErrorDir())
- hashCode = hashCode * 8191 + errorDir.hashCode();
-
- hashCode = hashCode * 8191 + ((setTime) ? 131071 : 524287);
-
- return hashCode;
- }
-
- @Override
- public int compareTo(bulkImportFiles_args other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetTinfo()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetCredentials()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetTid(), other.isSetTid());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetTid()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tid, other.tid);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetTableId(), other.isSetTableId());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetTableId()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableId, other.tableId);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetFiles(), other.isSetFiles());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetFiles()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.files, other.files);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetErrorDir(), other.isSetErrorDir());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetErrorDir()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorDir, other.errorDir);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetSetTime(), other.isSetSetTime());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetSetTime()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.setTime, other.setTime);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("bulkImportFiles_args(");
- boolean first = true;
-
- sb.append("tinfo:");
- if (this.tinfo == null) {
- sb.append("null");
- } else {
- sb.append(this.tinfo);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("credentials:");
- if (this.credentials == null) {
- sb.append("null");
- } else {
- sb.append(this.credentials);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("tid:");
- sb.append(this.tid);
- first = false;
- if (!first) sb.append(", ");
- sb.append("tableId:");
- if (this.tableId == null) {
- sb.append("null");
- } else {
- sb.append(this.tableId);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("files:");
- if (this.files == null) {
- sb.append("null");
- } else {
- sb.append(this.files);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("errorDir:");
- if (this.errorDir == null) {
- sb.append("null");
- } else {
- sb.append(this.errorDir);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("setTime:");
- sb.append(this.setTime);
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- if (tinfo != null) {
- tinfo.validate();
- }
- if (credentials != null) {
- credentials.validate();
- }
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
- __isset_bitfield = 0;
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class bulkImportFiles_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public bulkImportFiles_argsStandardScheme getScheme() {
- return new bulkImportFiles_argsStandardScheme();
- }
- }
-
- private static class bulkImportFiles_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<bulkImportFiles_args> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, bulkImportFiles_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 1: // TINFO
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
- struct.tinfo.read(iprot);
- struct.setTinfoIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 8: // CREDENTIALS
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
- struct.credentials.read(iprot);
- struct.setCredentialsIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 3: // TID
- if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
- struct.tid = iprot.readI64();
- struct.setTidIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 4: // TABLE_ID
- if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
- struct.tableId = iprot.readString();
- struct.setTableIdIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 5: // FILES
- if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
- {
- org.apache.thrift.protocol.TList _list18 = iprot.readListBegin();
- struct.files = new java.util.ArrayList<java.lang.String>(_list18.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem19;
- for (int _i20 = 0; _i20 < _list18.size; ++_i20)
- {
- _elem19 = iprot.readString();
- struct.files.add(_elem19);
- }
- iprot.readListEnd();
- }
- struct.setFilesIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 6: // ERROR_DIR
- if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
- struct.errorDir = iprot.readString();
- struct.setErrorDirIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 7: // SET_TIME
- if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
- struct.setTime = iprot.readBool();
- struct.setSetTimeIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, bulkImportFiles_args struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.tinfo != null) {
- oprot.writeFieldBegin(TINFO_FIELD_DESC);
- struct.tinfo.write(oprot);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldBegin(TID_FIELD_DESC);
- oprot.writeI64(struct.tid);
- oprot.writeFieldEnd();
- if (struct.tableId != null) {
- oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
- oprot.writeString(struct.tableId);
- oprot.writeFieldEnd();
- }
- if (struct.files != null) {
- oprot.writeFieldBegin(FILES_FIELD_DESC);
- {
- oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.files.size()));
- for (java.lang.String _iter21 : struct.files)
- {
- oprot.writeString(_iter21);
- }
- oprot.writeListEnd();
- }
- oprot.writeFieldEnd();
- }
- if (struct.errorDir != null) {
- oprot.writeFieldBegin(ERROR_DIR_FIELD_DESC);
- oprot.writeString(struct.errorDir);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldBegin(SET_TIME_FIELD_DESC);
- oprot.writeBool(struct.setTime);
- oprot.writeFieldEnd();
- if (struct.credentials != null) {
- oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
- struct.credentials.write(oprot);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class bulkImportFiles_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public bulkImportFiles_argsTupleScheme getScheme() {
- return new bulkImportFiles_argsTupleScheme();
- }
- }
-
- private static class bulkImportFiles_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<bulkImportFiles_args> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, bulkImportFiles_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetTinfo()) {
- optionals.set(0);
- }
- if (struct.isSetCredentials()) {
- optionals.set(1);
- }
- if (struct.isSetTid()) {
- optionals.set(2);
- }
- if (struct.isSetTableId()) {
- optionals.set(3);
- }
- if (struct.isSetFiles()) {
- optionals.set(4);
- }
- if (struct.isSetErrorDir()) {
- optionals.set(5);
- }
- if (struct.isSetSetTime()) {
- optionals.set(6);
- }
- oprot.writeBitSet(optionals, 7);
- if (struct.isSetTinfo()) {
- struct.tinfo.write(oprot);
- }
- if (struct.isSetCredentials()) {
- struct.credentials.write(oprot);
- }
- if (struct.isSetTid()) {
- oprot.writeI64(struct.tid);
- }
- if (struct.isSetTableId()) {
- oprot.writeString(struct.tableId);
- }
- if (struct.isSetFiles()) {
- {
- oprot.writeI32(struct.files.size());
- for (java.lang.String _iter22 : struct.files)
- {
- oprot.writeString(_iter22);
- }
- }
- }
- if (struct.isSetErrorDir()) {
- oprot.writeString(struct.errorDir);
- }
- if (struct.isSetSetTime()) {
- oprot.writeBool(struct.setTime);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, bulkImportFiles_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(7);
- if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
- struct.tinfo.read(iprot);
- struct.setTinfoIsSet(true);
- }
- if (incoming.get(1)) {
- struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
- struct.credentials.read(iprot);
- struct.setCredentialsIsSet(true);
- }
- if (incoming.get(2)) {
- struct.tid = iprot.readI64();
- struct.setTidIsSet(true);
- }
- if (incoming.get(3)) {
- struct.tableId = iprot.readString();
- struct.setTableIdIsSet(true);
- }
- if (incoming.get(4)) {
- {
- org.apache.thrift.protocol.TList _list23 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
- struct.files = new java.util.ArrayList<java.lang.String>(_list23.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem24;
- for (int _i25 = 0; _i25 < _list23.size; ++_i25)
- {
- _elem24 = iprot.readString();
- struct.files.add(_elem24);
- }
- }
- struct.setFilesIsSet(true);
- }
- if (incoming.get(5)) {
- struct.errorDir = iprot.readString();
- struct.setErrorDirIsSet(true);
- }
- if (incoming.get(6)) {
- struct.setTime = iprot.readBool();
- struct.setSetTimeIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- }
-
- @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
- public static class bulkImportFiles_result implements org.apache.thrift.TBase<bulkImportFiles_result, bulkImportFiles_result._Fields>, java.io.Serializable, Cloneable, Comparable<bulkImportFiles_result> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("bulkImportFiles_result");
-
- private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
- private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
- private static final org.apache.thrift.protocol.TField TOPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tope", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new bulkImportFiles_resultStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new bulkImportFiles_resultTupleSchemeFactory();
-
- public @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> success; // required
- public @org.apache.thrift.annotation.Nullable ThriftSecurityException sec; // required
- public @org.apache.thrift.annotation.Nullable ThriftTableOperationException tope; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- SUCCESS((short)0, "success"),
- SEC((short)1, "sec"),
- TOPE((short)2, "tope");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 0: // SUCCESS
- return SUCCESS;
- case 1: // SEC
- return SEC;
- case 2: // TOPE
- return TOPE;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
- tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftSecurityException.class)));
- tmpMap.put(_Fields.TOPE, new org.apache.thrift.meta_data.FieldMetaData("tope", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftTableOperationException.class)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(bulkImportFiles_result.class, metaDataMap);
- }
-
- public bulkImportFiles_result() {
- }
-
- public bulkImportFiles_result(
- java.util.List<java.lang.String> success,
- ThriftSecurityException sec,
- ThriftTableOperationException tope)
- {
- this();
- this.success = success;
- this.sec = sec;
- this.tope = tope;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public bulkImportFiles_result(bulkImportFiles_result other) {
- if (other.isSetSuccess()) {
- java.util.List<java.lang.String> __this__success = new java.util.ArrayList<java.lang.String>(other.success);
- this.success = __this__success;
- }
- if (other.isSetSec()) {
- this.sec = new ThriftSecurityException(other.sec);
- }
- if (other.isSetTope()) {
- this.tope = new ThriftTableOperationException(other.tope);
- }
- }
-
- @Override
- public bulkImportFiles_result deepCopy() {
- return new bulkImportFiles_result(this);
- }
-
- @Override
- public void clear() {
- this.success = null;
- this.sec = null;
- this.tope = null;
- }
-
- public int getSuccessSize() {
- return (this.success == null) ? 0 : this.success.size();
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.util.Iterator<java.lang.String> getSuccessIterator() {
- return (this.success == null) ? null : this.success.iterator();
- }
-
- public void addToSuccess(java.lang.String elem) {
- if (this.success == null) {
- this.success = new java.util.ArrayList<java.lang.String>();
- }
- this.success.add(elem);
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.util.List<java.lang.String> getSuccess() {
- return this.success;
- }
-
- public bulkImportFiles_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> success) {
- this.success = success;
- return this;
- }
-
- public void unsetSuccess() {
- this.success = null;
- }
-
- /** Returns true if field success is set (has been assigned a value) and false otherwise */
- public boolean isSetSuccess() {
- return this.success != null;
- }
-
- public void setSuccessIsSet(boolean value) {
- if (!value) {
- this.success = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public ThriftSecurityException getSec() {
- return this.sec;
- }
-
- public bulkImportFiles_result setSec(@org.apache.thrift.annotation.Nullable ThriftSecurityException sec) {
- this.sec = sec;
- return this;
- }
-
- public void unsetSec() {
- this.sec = null;
- }
-
- /** Returns true if field sec is set (has been assigned a value) and false otherwise */
- public boolean isSetSec() {
- return this.sec != null;
- }
-
- public void setSecIsSet(boolean value) {
- if (!value) {
- this.sec = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public ThriftTableOperationException getTope() {
- return this.tope;
- }
-
- public bulkImportFiles_result setTope(@org.apache.thrift.annotation.Nullable ThriftTableOperationException tope) {
- this.tope = tope;
- return this;
- }
-
- public void unsetTope() {
- this.tope = null;
- }
-
- /** Returns true if field tope is set (has been assigned a value) and false otherwise */
- public boolean isSetTope() {
- return this.tope != null;
- }
-
- public void setTopeIsSet(boolean value) {
- if (!value) {
- this.tope = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case SUCCESS:
- if (value == null) {
- unsetSuccess();
- } else {
- setSuccess((java.util.List<java.lang.String>)value);
- }
- break;
-
- case SEC:
- if (value == null) {
- unsetSec();
- } else {
- setSec((ThriftSecurityException)value);
- }
- break;
-
- case TOPE:
- if (value == null) {
- unsetTope();
- } else {
- setTope((ThriftTableOperationException)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case SUCCESS:
- return getSuccess();
-
- case SEC:
- return getSec();
-
- case TOPE:
- return getTope();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case SUCCESS:
- return isSetSuccess();
- case SEC:
- return isSetSec();
- case TOPE:
- return isSetTope();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof bulkImportFiles_result)
- return this.equals((bulkImportFiles_result)that);
- return false;
- }
-
- public boolean equals(bulkImportFiles_result that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_success = true && this.isSetSuccess();
- boolean that_present_success = true && that.isSetSuccess();
- if (this_present_success || that_present_success) {
- if (!(this_present_success && that_present_success))
- return false;
- if (!this.success.equals(that.success))
- return false;
- }
-
- boolean this_present_sec = true && this.isSetSec();
- boolean that_present_sec = true && that.isSetSec();
- if (this_present_sec || that_present_sec) {
- if (!(this_present_sec && that_present_sec))
- return false;
- if (!this.sec.equals(that.sec))
- return false;
- }
-
- boolean this_present_tope = true && this.isSetTope();
- boolean that_present_tope = true && that.isSetTope();
- if (this_present_tope || that_present_tope) {
- if (!(this_present_tope && that_present_tope))
- return false;
- if (!this.tope.equals(that.tope))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
- if (isSetSuccess())
- hashCode = hashCode * 8191 + success.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
- if (isSetSec())
- hashCode = hashCode * 8191 + sec.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetTope()) ? 131071 : 524287);
- if (isSetTope())
- hashCode = hashCode * 8191 + tope.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(bulkImportFiles_result other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetSuccess()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetSec(), other.isSetSec());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetSec()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetTope(), other.isSetTope());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetTope()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tope, other.tope);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("bulkImportFiles_result(");
- boolean first = true;
-
- sb.append("success:");
- if (this.success == null) {
- sb.append("null");
- } else {
- sb.append(this.success);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("sec:");
- if (this.sec == null) {
- sb.append("null");
- } else {
- sb.append(this.sec);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("tope:");
- if (this.tope == null) {
- sb.append("null");
- } else {
- sb.append(this.tope);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class bulkImportFiles_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public bulkImportFiles_resultStandardScheme getScheme() {
- return new bulkImportFiles_resultStandardScheme();
- }
- }
-
- private static class bulkImportFiles_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<bulkImportFiles_result> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, bulkImportFiles_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 0: // SUCCESS
- if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
- {
- org.apache.thrift.protocol.TList _list26 = iprot.readListBegin();
- struct.success = new java.util.ArrayList<java.lang.String>(_list26.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem27;
- for (int _i28 = 0; _i28 < _list26.size; ++_i28)
- {
- _elem27 = iprot.readString();
- struct.success.add(_elem27);
- }
- iprot.readListEnd();
- }
- struct.setSuccessIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 1: // SEC
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.sec = new ThriftSecurityException();
- struct.sec.read(iprot);
- struct.setSecIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 2: // TOPE
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tope = new ThriftTableOperationException();
- struct.tope.read(iprot);
- struct.setTopeIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, bulkImportFiles_result struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.success != null) {
- oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
- {
- oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
- for (java.lang.String _iter29 : struct.success)
- {
- oprot.writeString(_iter29);
- }
- oprot.writeListEnd();
- }
- oprot.writeFieldEnd();
- }
- if (struct.sec != null) {
- oprot.writeFieldBegin(SEC_FIELD_DESC);
- struct.sec.write(oprot);
- oprot.writeFieldEnd();
- }
- if (struct.tope != null) {
- oprot.writeFieldBegin(TOPE_FIELD_DESC);
- struct.tope.write(oprot);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class bulkImportFiles_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public bulkImportFiles_resultTupleScheme getScheme() {
- return new bulkImportFiles_resultTupleScheme();
- }
- }
-
- private static class bulkImportFiles_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<bulkImportFiles_result> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, bulkImportFiles_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetSuccess()) {
- optionals.set(0);
- }
- if (struct.isSetSec()) {
- optionals.set(1);
- }
- if (struct.isSetTope()) {
- optionals.set(2);
- }
- oprot.writeBitSet(optionals, 3);
- if (struct.isSetSuccess()) {
- {
- oprot.writeI32(struct.success.size());
- for (java.lang.String _iter30 : struct.success)
- {
- oprot.writeString(_iter30);
- }
- }
- }
- if (struct.isSetSec()) {
- struct.sec.write(oprot);
- }
- if (struct.isSetTope()) {
- struct.tope.write(oprot);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, bulkImportFiles_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(3);
- if (incoming.get(0)) {
- {
- org.apache.thrift.protocol.TList _list31 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
- struct.success = new java.util.ArrayList<java.lang.String>(_list31.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem32;
- for (int _i33 = 0; _i33 < _list31.size; ++_i33)
- {
- _elem32 = iprot.readString();
- struct.success.add(_elem32);
- }
- }
- struct.setSuccessIsSet(true);
- }
- if (incoming.get(1)) {
- struct.sec = new ThriftSecurityException();
- struct.sec.read(iprot);
- struct.setSecIsSet(true);
- }
- if (incoming.get(2)) {
- struct.tope = new ThriftTableOperationException();
- struct.tope.read(iprot);
- struct.setTopeIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- }
-
- @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
public static class isActive_args implements org.apache.thrift.TBase<isActive_args, isActive_args._Fields>, java.io.Serializable, Cloneable, Comparable<isActive_args> {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isActive_args");
@@ -10307,7 +8406,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new isActive_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new isActive_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public long tid; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -10382,7 +8481,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.TID, new org.apache.thrift.meta_data.FieldMetaData("tid", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -10393,7 +8492,7 @@
}
public isActive_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
long tid)
{
this();
@@ -10408,7 +8507,7 @@
public isActive_args(isActive_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
this.tid = other.tid;
}
@@ -10426,11 +8525,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public isActive_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public isActive_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -10480,7 +8579,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -10687,7 +8786,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -10765,7 +8864,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -12332,13 +10431,13 @@
case 2: // TABLES
if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
{
- org.apache.thrift.protocol.TSet _set34 = iprot.readSetBegin();
- struct.tables = new java.util.HashSet<java.lang.String>(2*_set34.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem35;
- for (int _i36 = 0; _i36 < _set34.size; ++_i36)
+ org.apache.thrift.protocol.TSet _set28 = iprot.readSetBegin();
+ struct.tables = new java.util.HashSet<java.lang.String>(2*_set28.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _elem29;
+ for (int _i30 = 0; _i30 < _set28.size; ++_i30)
{
- _elem35 = iprot.readString();
- struct.tables.add(_elem35);
+ _elem29 = iprot.readString();
+ struct.tables.add(_elem29);
}
iprot.readSetEnd();
}
@@ -12381,9 +10480,9 @@
oprot.writeFieldBegin(TABLES_FIELD_DESC);
{
oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tables.size()));
- for (java.lang.String _iter37 : struct.tables)
+ for (java.lang.String _iter31 : struct.tables)
{
- oprot.writeString(_iter37);
+ oprot.writeString(_iter31);
}
oprot.writeSetEnd();
}
@@ -12418,9 +10517,9 @@
if (struct.isSetTables()) {
{
oprot.writeI32(struct.tables.size());
- for (java.lang.String _iter38 : struct.tables)
+ for (java.lang.String _iter32 : struct.tables)
{
- oprot.writeString(_iter38);
+ oprot.writeString(_iter32);
}
}
}
@@ -12435,13 +10534,13 @@
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TSet _set39 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
- struct.tables = new java.util.HashSet<java.lang.String>(2*_set39.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem40;
- for (int _i41 = 0; _i41 < _set39.size; ++_i41)
+ org.apache.thrift.protocol.TSet _set33 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
+ struct.tables = new java.util.HashSet<java.lang.String>(2*_set33.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _elem34;
+ for (int _i35 = 0; _i35 < _set33.size; ++_i35)
{
- _elem40 = iprot.readString();
- struct.tables.add(_elem40);
+ _elem34 = iprot.readString();
+ struct.tables.add(_elem34);
}
}
struct.setTablesIsSet(true);
@@ -12952,14 +11051,14 @@
case 0: // SUCCESS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
- org.apache.thrift.protocol.TList _list42 = iprot.readListBegin();
- struct.success = new java.util.ArrayList<TDiskUsage>(_list42.size);
- @org.apache.thrift.annotation.Nullable TDiskUsage _elem43;
- for (int _i44 = 0; _i44 < _list42.size; ++_i44)
+ org.apache.thrift.protocol.TList _list36 = iprot.readListBegin();
+ struct.success = new java.util.ArrayList<TDiskUsage>(_list36.size);
+ @org.apache.thrift.annotation.Nullable TDiskUsage _elem37;
+ for (int _i38 = 0; _i38 < _list36.size; ++_i38)
{
- _elem43 = new TDiskUsage();
- _elem43.read(iprot);
- struct.success.add(_elem43);
+ _elem37 = new TDiskUsage();
+ _elem37.read(iprot);
+ struct.success.add(_elem37);
}
iprot.readListEnd();
}
@@ -13006,9 +11105,9 @@
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
- for (TDiskUsage _iter45 : struct.success)
+ for (TDiskUsage _iter39 : struct.success)
{
- _iter45.write(oprot);
+ _iter39.write(oprot);
}
oprot.writeListEnd();
}
@@ -13056,9 +11155,9 @@
if (struct.isSetSuccess()) {
{
oprot.writeI32(struct.success.size());
- for (TDiskUsage _iter46 : struct.success)
+ for (TDiskUsage _iter40 : struct.success)
{
- _iter46.write(oprot);
+ _iter40.write(oprot);
}
}
}
@@ -13076,14 +11175,14 @@
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TList _list47 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
- struct.success = new java.util.ArrayList<TDiskUsage>(_list47.size);
- @org.apache.thrift.annotation.Nullable TDiskUsage _elem48;
- for (int _i49 = 0; _i49 < _list47.size; ++_i49)
+ org.apache.thrift.protocol.TList _list41 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+ struct.success = new java.util.ArrayList<TDiskUsage>(_list41.size);
+ @org.apache.thrift.annotation.Nullable TDiskUsage _elem42;
+ for (int _i43 = 0; _i43 < _list41.size; ++_i43)
{
- _elem48 = new TDiskUsage();
- _elem48.read(iprot);
- struct.success.add(_elem48);
+ _elem42 = new TDiskUsage();
+ _elem42.read(iprot);
+ struct.success.add(_elem42);
}
}
struct.setSuccessIsSet(true);
@@ -13116,7 +11215,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new listLocalUsers_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new listLocalUsers_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -13189,7 +11288,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -13200,7 +11299,7 @@
}
public listLocalUsers_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -13213,7 +11312,7 @@
*/
public listLocalUsers_args(listLocalUsers_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -13232,11 +11331,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public listLocalUsers_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public listLocalUsers_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -13288,7 +11387,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -13502,7 +11601,7 @@
switch (schemeField.id) {
case 2: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -13583,7 +11682,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -14008,13 +12107,13 @@
case 0: // SUCCESS
if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
{
- org.apache.thrift.protocol.TSet _set50 = iprot.readSetBegin();
- struct.success = new java.util.HashSet<java.lang.String>(2*_set50.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem51;
- for (int _i52 = 0; _i52 < _set50.size; ++_i52)
+ org.apache.thrift.protocol.TSet _set44 = iprot.readSetBegin();
+ struct.success = new java.util.HashSet<java.lang.String>(2*_set44.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _elem45;
+ for (int _i46 = 0; _i46 < _set44.size; ++_i46)
{
- _elem51 = iprot.readString();
- struct.success.add(_elem51);
+ _elem45 = iprot.readString();
+ struct.success.add(_elem45);
}
iprot.readSetEnd();
}
@@ -14052,9 +12151,9 @@
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
{
oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
- for (java.lang.String _iter53 : struct.success)
+ for (java.lang.String _iter47 : struct.success)
{
- oprot.writeString(_iter53);
+ oprot.writeString(_iter47);
}
oprot.writeSetEnd();
}
@@ -14094,9 +12193,9 @@
if (struct.isSetSuccess()) {
{
oprot.writeI32(struct.success.size());
- for (java.lang.String _iter54 : struct.success)
+ for (java.lang.String _iter48 : struct.success)
{
- oprot.writeString(_iter54);
+ oprot.writeString(_iter48);
}
}
}
@@ -14111,13 +12210,13 @@
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TSet _set55 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
- struct.success = new java.util.HashSet<java.lang.String>(2*_set55.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem56;
- for (int _i57 = 0; _i57 < _set55.size; ++_i57)
+ org.apache.thrift.protocol.TSet _set49 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
+ struct.success = new java.util.HashSet<java.lang.String>(2*_set49.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _elem50;
+ for (int _i51 = 0; _i51 < _set49.size; ++_i51)
{
- _elem56 = iprot.readString();
- struct.success.add(_elem56);
+ _elem50 = iprot.readString();
+ struct.success.add(_elem50);
}
}
struct.setSuccessIsSet(true);
@@ -14147,7 +12246,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new createLocalUser_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new createLocalUser_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer password; // required
@@ -14228,7 +12327,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -14243,7 +12342,7 @@
}
public createLocalUser_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
java.nio.ByteBuffer password)
@@ -14260,7 +12359,7 @@
*/
public createLocalUser_args(createLocalUser_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -14287,11 +12386,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public createLocalUser_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public createLocalUser_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -14402,7 +12501,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -14708,7 +12807,7 @@
switch (schemeField.id) {
case 5: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -14827,7 +12926,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -15243,7 +13342,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new dropLocalUser_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new dropLocalUser_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
@@ -15320,7 +13419,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -15333,7 +13432,7 @@
}
public dropLocalUser_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal)
{
@@ -15348,7 +13447,7 @@
*/
public dropLocalUser_args(dropLocalUser_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -15371,11 +13470,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public dropLocalUser_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public dropLocalUser_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -15452,7 +13551,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -15710,7 +13809,7 @@
switch (schemeField.id) {
case 3: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -15810,7 +13909,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -16223,7 +14322,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new changeLocalUserPassword_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new changeLocalUserPassword_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer password; // required
@@ -16304,7 +14403,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -16319,7 +14418,7 @@
}
public changeLocalUserPassword_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
java.nio.ByteBuffer password)
@@ -16336,7 +14435,7 @@
*/
public changeLocalUserPassword_args(changeLocalUserPassword_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -16363,11 +14462,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public changeLocalUserPassword_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public changeLocalUserPassword_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -16478,7 +14577,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -16784,7 +14883,7 @@
switch (schemeField.id) {
case 4: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -16903,7 +15002,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -17318,7 +15417,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new authenticate_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new authenticate_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -17391,7 +15490,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -17402,7 +15501,7 @@
}
public authenticate_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -17415,7 +15514,7 @@
*/
public authenticate_args(authenticate_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -17434,11 +15533,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public authenticate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public authenticate_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -17490,7 +15589,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -17704,7 +15803,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -17785,7 +15884,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -18295,7 +16394,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new authenticateUser_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new authenticateUser_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth; // required
@@ -18372,7 +16471,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TO_AUTH, new org.apache.thrift.meta_data.FieldMetaData("toAuth", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -18385,7 +16484,7 @@
}
public authenticateUser_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
org.apache.accumulo.core.securityImpl.thrift.TCredentials toAuth)
{
@@ -18400,7 +16499,7 @@
*/
public authenticateUser_args(authenticateUser_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -18423,11 +16522,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public authenticateUser_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public authenticateUser_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -18504,7 +16603,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -18765,7 +16864,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -18866,7 +16965,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -19382,7 +17481,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new changeAuthorizations_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new changeAuthorizations_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public @org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> authorizations; // required
@@ -19463,7 +17562,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -19479,7 +17578,7 @@
}
public changeAuthorizations_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
java.util.List<java.nio.ByteBuffer> authorizations)
@@ -19496,7 +17595,7 @@
*/
public changeAuthorizations_args(changeAuthorizations_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -19524,11 +17623,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public changeAuthorizations_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public changeAuthorizations_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -19646,7 +17745,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -19948,7 +18047,7 @@
switch (schemeField.id) {
case 4: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -19975,13 +18074,13 @@
case 3: // AUTHORIZATIONS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
- org.apache.thrift.protocol.TList _list58 = iprot.readListBegin();
- struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list58.size);
- @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem59;
- for (int _i60 = 0; _i60 < _list58.size; ++_i60)
+ org.apache.thrift.protocol.TList _list52 = iprot.readListBegin();
+ struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list52.size);
+ @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem53;
+ for (int _i54 = 0; _i54 < _list52.size; ++_i54)
{
- _elem59 = iprot.readBinary();
- struct.authorizations.add(_elem59);
+ _elem53 = iprot.readBinary();
+ struct.authorizations.add(_elem53);
}
iprot.readListEnd();
}
@@ -20015,9 +18114,9 @@
oprot.writeFieldBegin(AUTHORIZATIONS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.authorizations.size()));
- for (java.nio.ByteBuffer _iter61 : struct.authorizations)
+ for (java.nio.ByteBuffer _iter55 : struct.authorizations)
{
- oprot.writeBinary(_iter61);
+ oprot.writeBinary(_iter55);
}
oprot.writeListEnd();
}
@@ -20077,9 +18176,9 @@
if (struct.isSetAuthorizations()) {
{
oprot.writeI32(struct.authorizations.size());
- for (java.nio.ByteBuffer _iter62 : struct.authorizations)
+ for (java.nio.ByteBuffer _iter56 : struct.authorizations)
{
- oprot.writeBinary(_iter62);
+ oprot.writeBinary(_iter56);
}
}
}
@@ -20090,7 +18189,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -20105,13 +18204,13 @@
}
if (incoming.get(3)) {
{
- org.apache.thrift.protocol.TList _list63 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
- struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list63.size);
- @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem64;
- for (int _i65 = 0; _i65 < _list63.size; ++_i65)
+ org.apache.thrift.protocol.TList _list57 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+ struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list57.size);
+ @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem58;
+ for (int _i59 = 0; _i59 < _list57.size; ++_i59)
{
- _elem64 = iprot.readBinary();
- struct.authorizations.add(_elem64);
+ _elem58 = iprot.readBinary();
+ struct.authorizations.add(_elem58);
}
}
struct.setAuthorizationsIsSet(true);
@@ -20515,7 +18614,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getUserAuthorizations_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getUserAuthorizations_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
@@ -20592,7 +18691,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -20605,7 +18704,7 @@
}
public getUserAuthorizations_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal)
{
@@ -20620,7 +18719,7 @@
*/
public getUserAuthorizations_args(getUserAuthorizations_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -20643,11 +18742,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public getUserAuthorizations_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getUserAuthorizations_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -20724,7 +18823,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -20982,7 +19081,7 @@
switch (schemeField.id) {
case 3: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -21082,7 +19181,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -21511,13 +19610,13 @@
case 0: // SUCCESS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
- org.apache.thrift.protocol.TList _list66 = iprot.readListBegin();
- struct.success = new java.util.ArrayList<java.nio.ByteBuffer>(_list66.size);
- @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem67;
- for (int _i68 = 0; _i68 < _list66.size; ++_i68)
+ org.apache.thrift.protocol.TList _list60 = iprot.readListBegin();
+ struct.success = new java.util.ArrayList<java.nio.ByteBuffer>(_list60.size);
+ @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem61;
+ for (int _i62 = 0; _i62 < _list60.size; ++_i62)
{
- _elem67 = iprot.readBinary();
- struct.success.add(_elem67);
+ _elem61 = iprot.readBinary();
+ struct.success.add(_elem61);
}
iprot.readListEnd();
}
@@ -21555,9 +19654,9 @@
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
- for (java.nio.ByteBuffer _iter69 : struct.success)
+ for (java.nio.ByteBuffer _iter63 : struct.success)
{
- oprot.writeBinary(_iter69);
+ oprot.writeBinary(_iter63);
}
oprot.writeListEnd();
}
@@ -21597,9 +19696,9 @@
if (struct.isSetSuccess()) {
{
oprot.writeI32(struct.success.size());
- for (java.nio.ByteBuffer _iter70 : struct.success)
+ for (java.nio.ByteBuffer _iter64 : struct.success)
{
- oprot.writeBinary(_iter70);
+ oprot.writeBinary(_iter64);
}
}
}
@@ -21614,13 +19713,13 @@
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TList _list71 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
- struct.success = new java.util.ArrayList<java.nio.ByteBuffer>(_list71.size);
- @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem72;
- for (int _i73 = 0; _i73 < _list71.size; ++_i73)
+ org.apache.thrift.protocol.TList _list65 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+ struct.success = new java.util.ArrayList<java.nio.ByteBuffer>(_list65.size);
+ @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem66;
+ for (int _i67 = 0; _i67 < _list65.size; ++_i67)
{
- _elem72 = iprot.readBinary();
- struct.success.add(_elem72);
+ _elem66 = iprot.readBinary();
+ struct.success.add(_elem66);
}
}
struct.setSuccessIsSet(true);
@@ -21650,7 +19749,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new hasSystemPermission_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new hasSystemPermission_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public byte sysPerm; // required
@@ -21733,7 +19832,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -21748,7 +19847,7 @@
}
public hasSystemPermission_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
byte sysPerm)
@@ -21767,7 +19866,7 @@
public hasSystemPermission_args(hasSystemPermission_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -21793,11 +19892,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public hasSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public hasSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -21897,7 +19996,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -22195,7 +20294,7 @@
switch (schemeField.id) {
case 4: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -22312,7 +20411,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -22832,7 +20931,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new hasTablePermission_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new hasTablePermission_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
@@ -22919,7 +21018,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -22936,7 +21035,7 @@
}
public hasTablePermission_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
java.lang.String tableName,
@@ -22957,7 +21056,7 @@
public hasTablePermission_args(hasTablePermission_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -22987,11 +21086,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public hasTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public hasTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -23116,7 +21215,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -23458,7 +21557,7 @@
switch (schemeField.id) {
case 5: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -23594,7 +21693,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -24225,7 +22324,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new hasNamespacePermission_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new hasNamespacePermission_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
@@ -24312,7 +22411,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -24329,7 +22428,7 @@
}
public hasNamespacePermission_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
java.lang.String ns,
@@ -24350,7 +22449,7 @@
public hasNamespacePermission_args(hasNamespacePermission_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -24380,11 +22479,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public hasNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public hasNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -24509,7 +22608,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -24851,7 +22950,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -24987,7 +23086,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -25617,7 +23716,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new grantSystemPermission_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new grantSystemPermission_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public byte permission; // required
@@ -25700,7 +23799,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -25715,7 +23814,7 @@
}
public grantSystemPermission_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
byte permission)
@@ -25734,7 +23833,7 @@
public grantSystemPermission_args(grantSystemPermission_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -25760,11 +23859,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public grantSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public grantSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -25864,7 +23963,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -26162,7 +24261,7 @@
switch (schemeField.id) {
case 4: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -26279,7 +24378,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -26696,7 +24795,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new revokeSystemPermission_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new revokeSystemPermission_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public byte permission; // required
@@ -26779,7 +24878,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -26794,7 +24893,7 @@
}
public revokeSystemPermission_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
byte permission)
@@ -26813,7 +24912,7 @@
public revokeSystemPermission_args(revokeSystemPermission_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -26839,11 +24938,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public revokeSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public revokeSystemPermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -26943,7 +25042,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -27241,7 +25340,7 @@
switch (schemeField.id) {
case 4: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -27358,7 +25457,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -27776,7 +25875,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new grantTablePermission_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new grantTablePermission_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
@@ -27863,7 +25962,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -27880,7 +25979,7 @@
}
public grantTablePermission_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
java.lang.String tableName,
@@ -27901,7 +26000,7 @@
public grantTablePermission_args(grantTablePermission_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -27931,11 +26030,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public grantTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public grantTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -28060,7 +26159,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -28402,7 +26501,7 @@
switch (schemeField.id) {
case 5: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -28538,7 +26637,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -29067,7 +27166,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new revokeTablePermission_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new revokeTablePermission_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
@@ -29154,7 +27253,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -29171,7 +27270,7 @@
}
public revokeTablePermission_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
java.lang.String tableName,
@@ -29192,7 +27291,7 @@
public revokeTablePermission_args(revokeTablePermission_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -29222,11 +27321,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public revokeTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public revokeTablePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -29351,7 +27450,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -29693,7 +27792,7 @@
switch (schemeField.id) {
case 5: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -29829,7 +27928,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -30358,7 +28457,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new grantNamespacePermission_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new grantNamespacePermission_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
@@ -30445,7 +28544,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -30462,7 +28561,7 @@
}
public grantNamespacePermission_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
java.lang.String ns,
@@ -30483,7 +28582,7 @@
public grantNamespacePermission_args(grantNamespacePermission_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -30513,11 +28612,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public grantNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public grantNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -30642,7 +28741,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -30984,7 +29083,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -31120,7 +29219,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -31649,7 +29748,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new revokeNamespacePermission_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new revokeNamespacePermission_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String principal; // required
public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
@@ -31736,7 +29835,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -31753,7 +29852,7 @@
}
public revokeNamespacePermission_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String principal,
java.lang.String ns,
@@ -31774,7 +29873,7 @@
public revokeNamespacePermission_args(revokeNamespacePermission_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -31804,11 +29903,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public revokeNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public revokeNamespacePermission_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -31933,7 +30032,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -32275,7 +30374,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -32411,7 +30510,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -32938,7 +31037,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getConfiguration_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getConfiguration_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/**
*
@@ -33023,7 +31122,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -33036,7 +31135,7 @@
}
public getConfiguration_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
ConfigurationType type)
{
@@ -33051,7 +31150,7 @@
*/
public getConfiguration_args(getConfiguration_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -33074,11 +31173,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public getConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -33163,7 +31262,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -33421,7 +31520,7 @@
switch (schemeField.id) {
case 2: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -33521,7 +31620,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -33946,15 +32045,15 @@
case 0: // SUCCESS
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map74 = iprot.readMapBegin();
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map74.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key75;
- @org.apache.thrift.annotation.Nullable java.lang.String _val76;
- for (int _i77 = 0; _i77 < _map74.size; ++_i77)
+ org.apache.thrift.protocol.TMap _map68 = iprot.readMapBegin();
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map68.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key69;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val70;
+ for (int _i71 = 0; _i71 < _map68.size; ++_i71)
{
- _key75 = iprot.readString();
- _val76 = iprot.readString();
- struct.success.put(_key75, _val76);
+ _key69 = iprot.readString();
+ _val70 = iprot.readString();
+ struct.success.put(_key69, _val70);
}
iprot.readMapEnd();
}
@@ -33992,10 +32091,10 @@
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter78 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter72 : struct.success.entrySet())
{
- oprot.writeString(_iter78.getKey());
- oprot.writeString(_iter78.getValue());
+ oprot.writeString(_iter72.getKey());
+ oprot.writeString(_iter72.getValue());
}
oprot.writeMapEnd();
}
@@ -34035,10 +32134,10 @@
if (struct.isSetSuccess()) {
{
oprot.writeI32(struct.success.size());
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter79 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter73 : struct.success.entrySet())
{
- oprot.writeString(_iter79.getKey());
- oprot.writeString(_iter79.getValue());
+ oprot.writeString(_iter73.getKey());
+ oprot.writeString(_iter73.getValue());
}
}
}
@@ -34053,15 +32152,15 @@
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TMap _map80 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map80.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key81;
- @org.apache.thrift.annotation.Nullable java.lang.String _val82;
- for (int _i83 = 0; _i83 < _map80.size; ++_i83)
+ org.apache.thrift.protocol.TMap _map74 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map74.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key75;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val76;
+ for (int _i77 = 0; _i77 < _map74.size; ++_i77)
{
- _key81 = iprot.readString();
- _val82 = iprot.readString();
- struct.success.put(_key81, _val82);
+ _key75 = iprot.readString();
+ _val76 = iprot.readString();
+ struct.success.put(_key75, _val76);
}
}
struct.setSuccessIsSet(true);
@@ -34089,7 +32188,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getSystemProperties_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getSystemProperties_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -34162,7 +32261,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -34173,7 +32272,7 @@
}
public getSystemProperties_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -34186,7 +32285,7 @@
*/
public getSystemProperties_args(getSystemProperties_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -34205,11 +32304,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public getSystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getSystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -34261,7 +32360,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -34475,7 +32574,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -34556,7 +32655,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -34977,15 +33076,15 @@
case 0: // SUCCESS
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map84 = iprot.readMapBegin();
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map84.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key85;
- @org.apache.thrift.annotation.Nullable java.lang.String _val86;
- for (int _i87 = 0; _i87 < _map84.size; ++_i87)
+ org.apache.thrift.protocol.TMap _map78 = iprot.readMapBegin();
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map78.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key79;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val80;
+ for (int _i81 = 0; _i81 < _map78.size; ++_i81)
{
- _key85 = iprot.readString();
- _val86 = iprot.readString();
- struct.success.put(_key85, _val86);
+ _key79 = iprot.readString();
+ _val80 = iprot.readString();
+ struct.success.put(_key79, _val80);
}
iprot.readMapEnd();
}
@@ -35023,10 +33122,10 @@
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter88 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter82 : struct.success.entrySet())
{
- oprot.writeString(_iter88.getKey());
- oprot.writeString(_iter88.getValue());
+ oprot.writeString(_iter82.getKey());
+ oprot.writeString(_iter82.getValue());
}
oprot.writeMapEnd();
}
@@ -35066,10 +33165,10 @@
if (struct.isSetSuccess()) {
{
oprot.writeI32(struct.success.size());
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter89 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter83 : struct.success.entrySet())
{
- oprot.writeString(_iter89.getKey());
- oprot.writeString(_iter89.getValue());
+ oprot.writeString(_iter83.getKey());
+ oprot.writeString(_iter83.getValue());
}
}
}
@@ -35084,15 +33183,15 @@
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TMap _map90 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map90.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key91;
- @org.apache.thrift.annotation.Nullable java.lang.String _val92;
- for (int _i93 = 0; _i93 < _map90.size; ++_i93)
+ org.apache.thrift.protocol.TMap _map84 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map84.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key85;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val86;
+ for (int _i87 = 0; _i87 < _map84.size; ++_i87)
{
- _key91 = iprot.readString();
- _val92 = iprot.readString();
- struct.success.put(_key91, _val92);
+ _key85 = iprot.readString();
+ _val86 = iprot.readString();
+ struct.success.put(_key85, _val86);
}
}
struct.setSuccessIsSet(true);
@@ -35120,7 +33219,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getVersionedSystemProperties_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getVersionedSystemProperties_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -35193,7 +33292,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -35204,7 +33303,7 @@
}
public getVersionedSystemProperties_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -35217,7 +33316,7 @@
*/
public getVersionedSystemProperties_args(getVersionedSystemProperties_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -35236,11 +33335,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public getVersionedSystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getVersionedSystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -35292,7 +33391,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -35506,7 +33605,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -35587,7 +33686,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -36105,7 +34204,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTableConfiguration_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTableConfiguration_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
@@ -36182,7 +34281,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -36195,7 +34294,7 @@
}
public getTableConfiguration_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String tableName)
{
@@ -36210,7 +34309,7 @@
*/
public getTableConfiguration_args(getTableConfiguration_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -36233,11 +34332,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public getTableConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getTableConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -36314,7 +34413,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -36572,7 +34671,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -36672,7 +34771,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -37179,15 +35278,15 @@
case 0: // SUCCESS
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map94 = iprot.readMapBegin();
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map94.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key95;
- @org.apache.thrift.annotation.Nullable java.lang.String _val96;
- for (int _i97 = 0; _i97 < _map94.size; ++_i97)
+ org.apache.thrift.protocol.TMap _map88 = iprot.readMapBegin();
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map88.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key89;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val90;
+ for (int _i91 = 0; _i91 < _map88.size; ++_i91)
{
- _key95 = iprot.readString();
- _val96 = iprot.readString();
- struct.success.put(_key95, _val96);
+ _key89 = iprot.readString();
+ _val90 = iprot.readString();
+ struct.success.put(_key89, _val90);
}
iprot.readMapEnd();
}
@@ -37234,10 +35333,10 @@
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter98 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter92 : struct.success.entrySet())
{
- oprot.writeString(_iter98.getKey());
- oprot.writeString(_iter98.getValue());
+ oprot.writeString(_iter92.getKey());
+ oprot.writeString(_iter92.getValue());
}
oprot.writeMapEnd();
}
@@ -37285,10 +35384,10 @@
if (struct.isSetSuccess()) {
{
oprot.writeI32(struct.success.size());
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter99 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter93 : struct.success.entrySet())
{
- oprot.writeString(_iter99.getKey());
- oprot.writeString(_iter99.getValue());
+ oprot.writeString(_iter93.getKey());
+ oprot.writeString(_iter93.getValue());
}
}
}
@@ -37306,15 +35405,15 @@
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TMap _map100 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map100.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key101;
- @org.apache.thrift.annotation.Nullable java.lang.String _val102;
- for (int _i103 = 0; _i103 < _map100.size; ++_i103)
+ org.apache.thrift.protocol.TMap _map94 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map94.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key95;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val96;
+ for (int _i97 = 0; _i97 < _map94.size; ++_i97)
{
- _key101 = iprot.readString();
- _val102 = iprot.readString();
- struct.success.put(_key101, _val102);
+ _key95 = iprot.readString();
+ _val96 = iprot.readString();
+ struct.success.put(_key95, _val96);
}
}
struct.setSuccessIsSet(true);
@@ -37348,7 +35447,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTableProperties_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTableProperties_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
@@ -37425,7 +35524,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -37438,7 +35537,7 @@
}
public getTableProperties_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String tableName)
{
@@ -37453,7 +35552,7 @@
*/
public getTableProperties_args(getTableProperties_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -37476,11 +35575,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public getTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -37557,7 +35656,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -37815,7 +35914,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -37915,7 +36014,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -38422,15 +36521,15 @@
case 0: // SUCCESS
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map104 = iprot.readMapBegin();
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map104.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key105;
- @org.apache.thrift.annotation.Nullable java.lang.String _val106;
- for (int _i107 = 0; _i107 < _map104.size; ++_i107)
+ org.apache.thrift.protocol.TMap _map98 = iprot.readMapBegin();
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map98.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key99;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val100;
+ for (int _i101 = 0; _i101 < _map98.size; ++_i101)
{
- _key105 = iprot.readString();
- _val106 = iprot.readString();
- struct.success.put(_key105, _val106);
+ _key99 = iprot.readString();
+ _val100 = iprot.readString();
+ struct.success.put(_key99, _val100);
}
iprot.readMapEnd();
}
@@ -38477,10 +36576,10 @@
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter108 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter102 : struct.success.entrySet())
{
- oprot.writeString(_iter108.getKey());
- oprot.writeString(_iter108.getValue());
+ oprot.writeString(_iter102.getKey());
+ oprot.writeString(_iter102.getValue());
}
oprot.writeMapEnd();
}
@@ -38528,10 +36627,10 @@
if (struct.isSetSuccess()) {
{
oprot.writeI32(struct.success.size());
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter109 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter103 : struct.success.entrySet())
{
- oprot.writeString(_iter109.getKey());
- oprot.writeString(_iter109.getValue());
+ oprot.writeString(_iter103.getKey());
+ oprot.writeString(_iter103.getValue());
}
}
}
@@ -38549,15 +36648,15 @@
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TMap _map110 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map110.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key111;
- @org.apache.thrift.annotation.Nullable java.lang.String _val112;
- for (int _i113 = 0; _i113 < _map110.size; ++_i113)
+ org.apache.thrift.protocol.TMap _map104 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map104.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key105;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val106;
+ for (int _i107 = 0; _i107 < _map104.size; ++_i107)
{
- _key111 = iprot.readString();
- _val112 = iprot.readString();
- struct.success.put(_key111, _val112);
+ _key105 = iprot.readString();
+ _val106 = iprot.readString();
+ struct.success.put(_key105, _val106);
}
}
struct.setSuccessIsSet(true);
@@ -38591,7 +36690,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getVersionedTableProperties_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getVersionedTableProperties_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
@@ -38668,7 +36767,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -38681,7 +36780,7 @@
}
public getVersionedTableProperties_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String tableName)
{
@@ -38696,7 +36795,7 @@
*/
public getVersionedTableProperties_args(getVersionedTableProperties_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -38719,11 +36818,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public getVersionedTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getVersionedTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -38800,7 +36899,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -39058,7 +37157,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -39158,7 +37257,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -39787,7 +37886,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceConfiguration_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceConfiguration_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
@@ -39864,7 +37963,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -39877,7 +37976,7 @@
}
public getNamespaceConfiguration_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String ns)
{
@@ -39892,7 +37991,7 @@
*/
public getNamespaceConfiguration_args(getNamespaceConfiguration_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -39915,11 +38014,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public getNamespaceConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getNamespaceConfiguration_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -39996,7 +38095,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -40254,7 +38353,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -40354,7 +38453,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -40861,15 +38960,15 @@
case 0: // SUCCESS
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map114 = iprot.readMapBegin();
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map114.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key115;
- @org.apache.thrift.annotation.Nullable java.lang.String _val116;
- for (int _i117 = 0; _i117 < _map114.size; ++_i117)
+ org.apache.thrift.protocol.TMap _map108 = iprot.readMapBegin();
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map108.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key109;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val110;
+ for (int _i111 = 0; _i111 < _map108.size; ++_i111)
{
- _key115 = iprot.readString();
- _val116 = iprot.readString();
- struct.success.put(_key115, _val116);
+ _key109 = iprot.readString();
+ _val110 = iprot.readString();
+ struct.success.put(_key109, _val110);
}
iprot.readMapEnd();
}
@@ -40916,10 +39015,10 @@
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter118 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter112 : struct.success.entrySet())
{
- oprot.writeString(_iter118.getKey());
- oprot.writeString(_iter118.getValue());
+ oprot.writeString(_iter112.getKey());
+ oprot.writeString(_iter112.getValue());
}
oprot.writeMapEnd();
}
@@ -40967,10 +39066,10 @@
if (struct.isSetSuccess()) {
{
oprot.writeI32(struct.success.size());
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter119 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter113 : struct.success.entrySet())
{
- oprot.writeString(_iter119.getKey());
- oprot.writeString(_iter119.getValue());
+ oprot.writeString(_iter113.getKey());
+ oprot.writeString(_iter113.getValue());
}
}
}
@@ -40988,15 +39087,15 @@
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TMap _map120 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map120.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key121;
- @org.apache.thrift.annotation.Nullable java.lang.String _val122;
- for (int _i123 = 0; _i123 < _map120.size; ++_i123)
+ org.apache.thrift.protocol.TMap _map114 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map114.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key115;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val116;
+ for (int _i117 = 0; _i117 < _map114.size; ++_i117)
{
- _key121 = iprot.readString();
- _val122 = iprot.readString();
- struct.success.put(_key121, _val122);
+ _key115 = iprot.readString();
+ _val116 = iprot.readString();
+ struct.success.put(_key115, _val116);
}
}
struct.setSuccessIsSet(true);
@@ -41030,7 +39129,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getNamespaceProperties_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getNamespaceProperties_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
@@ -41107,7 +39206,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -41120,7 +39219,7 @@
}
public getNamespaceProperties_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String ns)
{
@@ -41135,7 +39234,7 @@
*/
public getNamespaceProperties_args(getNamespaceProperties_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -41158,11 +39257,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public getNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -41239,7 +39338,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -41497,7 +39596,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -41597,7 +39696,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -42104,15 +40203,15 @@
case 0: // SUCCESS
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map124 = iprot.readMapBegin();
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map124.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key125;
- @org.apache.thrift.annotation.Nullable java.lang.String _val126;
- for (int _i127 = 0; _i127 < _map124.size; ++_i127)
+ org.apache.thrift.protocol.TMap _map118 = iprot.readMapBegin();
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map118.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key119;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val120;
+ for (int _i121 = 0; _i121 < _map118.size; ++_i121)
{
- _key125 = iprot.readString();
- _val126 = iprot.readString();
- struct.success.put(_key125, _val126);
+ _key119 = iprot.readString();
+ _val120 = iprot.readString();
+ struct.success.put(_key119, _val120);
}
iprot.readMapEnd();
}
@@ -42159,10 +40258,10 @@
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size()));
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter128 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter122 : struct.success.entrySet())
{
- oprot.writeString(_iter128.getKey());
- oprot.writeString(_iter128.getValue());
+ oprot.writeString(_iter122.getKey());
+ oprot.writeString(_iter122.getValue());
}
oprot.writeMapEnd();
}
@@ -42210,10 +40309,10 @@
if (struct.isSetSuccess()) {
{
oprot.writeI32(struct.success.size());
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter129 : struct.success.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter123 : struct.success.entrySet())
{
- oprot.writeString(_iter129.getKey());
- oprot.writeString(_iter129.getValue());
+ oprot.writeString(_iter123.getKey());
+ oprot.writeString(_iter123.getValue());
}
}
}
@@ -42231,15 +40330,15 @@
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TMap _map130 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
- struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map130.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key131;
- @org.apache.thrift.annotation.Nullable java.lang.String _val132;
- for (int _i133 = 0; _i133 < _map130.size; ++_i133)
+ org.apache.thrift.protocol.TMap _map124 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
+ struct.success = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map124.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key125;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val126;
+ for (int _i127 = 0; _i127 < _map124.size; ++_i127)
{
- _key131 = iprot.readString();
- _val132 = iprot.readString();
- struct.success.put(_key131, _val132);
+ _key125 = iprot.readString();
+ _val126 = iprot.readString();
+ struct.success.put(_key125, _val126);
}
}
struct.setSuccessIsSet(true);
@@ -42273,7 +40372,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getVersionedNamespaceProperties_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getVersionedNamespaceProperties_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
@@ -42350,7 +40449,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -42363,7 +40462,7 @@
}
public getVersionedNamespaceProperties_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String ns)
{
@@ -42378,7 +40477,7 @@
*/
public getVersionedNamespaceProperties_args(getVersionedNamespaceProperties_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -42401,11 +40500,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public getVersionedNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getVersionedNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -42482,7 +40581,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -42740,7 +40839,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -42840,7 +40939,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -43470,7 +41569,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new checkClass_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new checkClass_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String className; // required
public @org.apache.thrift.annotation.Nullable java.lang.String interfaceMatch; // required
@@ -43551,7 +41650,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.CLASS_NAME, new org.apache.thrift.meta_data.FieldMetaData("className", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -43566,7 +41665,7 @@
}
public checkClass_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String className,
java.lang.String interfaceMatch)
@@ -43583,7 +41682,7 @@
*/
public checkClass_args(checkClass_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -43610,11 +41709,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public checkClass_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public checkClass_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -43716,7 +41815,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -44018,7 +42117,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -44137,7 +42236,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -44550,7 +42649,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new checkTableClass_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new checkTableClass_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableId; // required
public @org.apache.thrift.annotation.Nullable java.lang.String className; // required
@@ -44635,7 +42734,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("tableId", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -44652,7 +42751,7 @@
}
public checkTableClass_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String tableId,
java.lang.String className,
@@ -44671,7 +42770,7 @@
*/
public checkTableClass_args(checkTableClass_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -44702,11 +42801,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public checkTableClass_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public checkTableClass_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -44833,7 +42932,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -45179,7 +43278,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -45317,7 +43416,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -45948,7 +44047,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new checkNamespaceClass_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new checkNamespaceClass_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String namespaceId; // required
public @org.apache.thrift.annotation.Nullable java.lang.String className; // required
@@ -46033,7 +44132,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.NAMESPACE_ID, new org.apache.thrift.meta_data.FieldMetaData("namespaceId", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -46050,7 +44149,7 @@
}
public checkNamespaceClass_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String namespaceId,
java.lang.String className,
@@ -46069,7 +44168,7 @@
*/
public checkNamespaceClass_args(checkNamespaceClass_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -46100,11 +44199,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public TInfo getTinfo() {
return this.tinfo;
}
- public checkNamespaceClass_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public checkNamespaceClass_args setTinfo(@org.apache.thrift.annotation.Nullable TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -46231,7 +44330,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((TInfo)value);
}
break;
@@ -46577,7 +44676,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -46715,7 +44814,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/trace/thrift/TInfo.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TInfo.java
similarity index 91%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/trace/thrift/TInfo.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TInfo.java
index e657138..d5b9581 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/trace/thrift/TInfo.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TInfo.java
@@ -22,13 +22,13 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.trace.thrift;
+package org.apache.accumulo.core.clientImpl.thrift;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
public class TInfo implements org.apache.thrift.TBase<TInfo, TInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TInfo> {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TInfo");
- private static final org.apache.thrift.protocol.TField HEADERS_FIELD_DESC = new org.apache.thrift.protocol.TField("headers", org.apache.thrift.protocol.TType.MAP, (short)3);
+ private static final org.apache.thrift.protocol.TField HEADERS_FIELD_DESC = new org.apache.thrift.protocol.TField("headers", org.apache.thrift.protocol.TType.MAP, (short)1);
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TInfoStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TInfoTupleSchemeFactory();
@@ -37,7 +37,7 @@
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- HEADERS((short)3, "headers");
+ HEADERS((short)1, "headers");
private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
@@ -53,7 +53,7 @@
@org.apache.thrift.annotation.Nullable
public static _Fields findByThriftId(int fieldId) {
switch(fieldId) {
- case 3: // HEADERS
+ case 1: // HEADERS
return HEADERS;
default:
return null;
@@ -344,18 +344,18 @@
break;
}
switch (schemeField.id) {
- case 3: // HEADERS
+ case 1: // HEADERS
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin();
- struct.headers = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map0.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key1;
- @org.apache.thrift.annotation.Nullable java.lang.String _val2;
- for (int _i3 = 0; _i3 < _map0.size; ++_i3)
+ org.apache.thrift.protocol.TMap _map18 = iprot.readMapBegin();
+ struct.headers = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map18.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key19;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val20;
+ for (int _i21 = 0; _i21 < _map18.size; ++_i21)
{
- _key1 = iprot.readString();
- _val2 = iprot.readString();
- struct.headers.put(_key1, _val2);
+ _key19 = iprot.readString();
+ _val20 = iprot.readString();
+ struct.headers.put(_key19, _val20);
}
iprot.readMapEnd();
}
@@ -384,10 +384,10 @@
oprot.writeFieldBegin(HEADERS_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.headers.size()));
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter4 : struct.headers.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter22 : struct.headers.entrySet())
{
- oprot.writeString(_iter4.getKey());
- oprot.writeString(_iter4.getValue());
+ oprot.writeString(_iter22.getKey());
+ oprot.writeString(_iter22.getValue());
}
oprot.writeMapEnd();
}
@@ -419,10 +419,10 @@
if (struct.isSetHeaders()) {
{
oprot.writeI32(struct.headers.size());
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter5 : struct.headers.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter23 : struct.headers.entrySet())
{
- oprot.writeString(_iter5.getKey());
- oprot.writeString(_iter5.getValue());
+ oprot.writeString(_iter23.getKey());
+ oprot.writeString(_iter23.getValue());
}
}
}
@@ -434,15 +434,15 @@
java.util.BitSet incoming = iprot.readBitSet(1);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TMap _map6 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
- struct.headers = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map6.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key7;
- @org.apache.thrift.annotation.Nullable java.lang.String _val8;
- for (int _i9 = 0; _i9 < _map6.size; ++_i9)
+ org.apache.thrift.protocol.TMap _map24 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
+ struct.headers = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map24.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key25;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val26;
+ for (int _i27 = 0; _i27 < _map24.size; ++_i27)
{
- _key7 = iprot.readString();
- _val8 = iprot.readString();
- struct.headers.put(_key7, _val8);
+ _key25 = iprot.readString();
+ _val26 = iprot.readString();
+ struct.headers.put(_key25, _val26);
}
}
struct.setHeadersIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TableOperationExceptionType.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TableOperationExceptionType.java
index 4b8ec29..ba4a397 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TableOperationExceptionType.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/clientImpl/thrift/TableOperationExceptionType.java
@@ -30,13 +30,13 @@
NOTFOUND(1),
OFFLINE(2),
BULK_BAD_INPUT_DIRECTORY(3),
- BULK_BAD_ERROR_DIRECTORY(4),
+ OBSOLETE_BULK_BAD_ERROR_DIRECTORY(4),
BAD_RANGE(5),
OTHER(6),
NAMESPACE_EXISTS(7),
NAMESPACE_NOTFOUND(8),
INVALID_NAME(9),
- BULK_BAD_LOAD_MAPPING(10),
+ OBSOLETE_BULK_BAD_LOAD_MAPPING(10),
BULK_CONCURRENT_MERGE(11);
private final int value;
@@ -69,7 +69,7 @@
case 3:
return BULK_BAD_INPUT_DIRECTORY;
case 4:
- return BULK_BAD_ERROR_DIRECTORY;
+ return OBSOLETE_BULK_BAD_ERROR_DIRECTORY;
case 5:
return BAD_RANGE;
case 6:
@@ -81,7 +81,7 @@
case 9:
return INVALID_NAME;
case 10:
- return BULK_BAD_LOAD_MAPPING;
+ return OBSOLETE_BULK_BAD_LOAD_MAPPING;
case 11:
return BULK_CONCURRENT_MERGE;
default:
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java
index ac8e15f..7b8af04 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactionCoordinatorService.java
@@ -29,37 +29,37 @@
public interface Iface {
- public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException;
+ public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException;
- public TNextCompactionJob getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
+ public TNextCompactionJob getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
- public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException;
+ public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException;
- public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException;
+ public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException;
- public TExternalCompactionList getRunningCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
+ public TExternalCompactionList getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
- public TExternalCompactionList getCompletedCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
+ public TExternalCompactionList getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
- public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
+ public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
}
public interface AsyncIface {
- public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler) throws org.apache.thrift.TException;
+ public void getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler) throws org.apache.thrift.TException;
- public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void getRunningCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException;
+ public void getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException;
- public void getCompletedCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException;
+ public void getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException;
- public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
}
@@ -86,13 +86,13 @@
}
@Override
- public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException
+ public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException
{
send_compactionCompleted(tinfo, credentials, externalCompactionId, extent, stats);
recv_compactionCompleted();
}
- public void send_compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException
+ public void send_compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats) throws org.apache.thrift.TException
{
compactionCompleted_args args = new compactionCompleted_args();
args.setTinfo(tinfo);
@@ -111,13 +111,13 @@
}
@Override
- public TNextCompactionJob getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+ public TNextCompactionJob getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
{
send_getCompactionJob(tinfo, credentials, queueName, compactor, externalCompactionId);
return recv_getCompactionJob();
}
- public void send_getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+ public void send_getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
{
getCompactionJob_args args = new getCompactionJob_args();
args.setTinfo(tinfo);
@@ -139,13 +139,13 @@
}
@Override
- public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException
+ public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException
{
send_updateCompactionStatus(tinfo, credentials, externalCompactionId, status, timestamp);
recv_updateCompactionStatus();
}
- public void send_updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException
+ public void send_updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp) throws org.apache.thrift.TException
{
updateCompactionStatus_args args = new updateCompactionStatus_args();
args.setTinfo(tinfo);
@@ -164,13 +164,13 @@
}
@Override
- public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+ public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
{
send_compactionFailed(tinfo, credentials, externalCompactionId, extent);
recv_compactionFailed();
}
- public void send_compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+ public void send_compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
{
compactionFailed_args args = new compactionFailed_args();
args.setTinfo(tinfo);
@@ -188,13 +188,13 @@
}
@Override
- public TExternalCompactionList getRunningCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public TExternalCompactionList getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
send_getRunningCompactions(tinfo, credentials);
return recv_getRunningCompactions();
}
- public void send_getRunningCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
getRunningCompactions_args args = new getRunningCompactions_args();
args.setTinfo(tinfo);
@@ -213,13 +213,13 @@
}
@Override
- public TExternalCompactionList getCompletedCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public TExternalCompactionList getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
send_getCompletedCompactions(tinfo, credentials);
return recv_getCompletedCompactions();
}
- public void send_getCompletedCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
getCompletedCompactions_args args = new getCompletedCompactions_args();
args.setTinfo(tinfo);
@@ -238,13 +238,13 @@
}
@Override
- public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+ public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
{
send_cancel(tinfo, credentials, externalCompactionId);
recv_cancel();
}
- public void send_cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+ public void send_cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
{
cancel_args args = new cancel_args();
args.setTinfo(tinfo);
@@ -280,7 +280,7 @@
}
@Override
- public void compactionCompleted(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void compactionCompleted(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
compactionCompleted_call method_call = new compactionCompleted_call(tinfo, credentials, externalCompactionId, extent, stats, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -288,12 +288,12 @@
}
public static class compactionCompleted_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String externalCompactionId;
private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
private org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats;
- public compactionCompleted_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public compactionCompleted_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.tabletserver.thrift.TCompactionStats stats, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -328,7 +328,7 @@
}
@Override
- public void getCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler) throws org.apache.thrift.TException {
+ public void getCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler) throws org.apache.thrift.TException {
checkReady();
getCompactionJob_call method_call = new getCompactionJob_call(tinfo, credentials, queueName, compactor, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -336,12 +336,12 @@
}
public static class getCompactionJob_call extends org.apache.thrift.async.TAsyncMethodCall<TNextCompactionJob> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String queueName;
private java.lang.String compactor;
private java.lang.String externalCompactionId;
- public getCompactionJob_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getCompactionJob_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TNextCompactionJob> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -375,7 +375,7 @@
}
@Override
- public void updateCompactionStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void updateCompactionStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
updateCompactionStatus_call method_call = new updateCompactionStatus_call(tinfo, credentials, externalCompactionId, status, timestamp, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -383,12 +383,12 @@
}
public static class updateCompactionStatus_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String externalCompactionId;
private TCompactionStatusUpdate status;
private long timestamp;
- public updateCompactionStatus_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public updateCompactionStatus_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, TCompactionStatusUpdate status, long timestamp, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -423,7 +423,7 @@
}
@Override
- public void compactionFailed(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void compactionFailed(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
compactionFailed_call method_call = new compactionFailed_call(tinfo, credentials, externalCompactionId, extent, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -431,11 +431,11 @@
}
public static class compactionFailed_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String externalCompactionId;
private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
- public compactionFailed_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public compactionFailed_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -468,7 +468,7 @@
}
@Override
- public void getRunningCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException {
+ public void getRunningCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException {
checkReady();
getRunningCompactions_call method_call = new getRunningCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -476,9 +476,9 @@
}
public static class getRunningCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<TExternalCompactionList> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getRunningCompactions_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getRunningCompactions_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -506,7 +506,7 @@
}
@Override
- public void getCompletedCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException {
+ public void getCompletedCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler) throws org.apache.thrift.TException {
checkReady();
getCompletedCompactions_call method_call = new getCompletedCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -514,9 +514,9 @@
}
public static class getCompletedCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<TExternalCompactionList> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getCompletedCompactions_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getCompletedCompactions_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionList> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -544,7 +544,7 @@
}
@Override
- public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
cancel_call method_call = new cancel_call(tinfo, credentials, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -552,10 +552,10 @@
}
public static class cancel_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String externalCompactionId;
- public cancel_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public cancel_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1307,7 +1307,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionCompleted_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionCompleted_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
@@ -1392,7 +1392,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -1409,7 +1409,7 @@
}
public compactionCompleted_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String externalCompactionId,
org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
@@ -1428,7 +1428,7 @@
*/
public compactionCompleted_args(compactionCompleted_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -1459,11 +1459,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public compactionCompleted_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public compactionCompleted_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -1590,7 +1590,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -1942,7 +1942,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -2082,7 +2082,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -2391,7 +2391,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionJob_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionJob_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String queueName; // required
public @org.apache.thrift.annotation.Nullable java.lang.String compactor; // required
@@ -2476,7 +2476,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.QUEUE_NAME, new org.apache.thrift.meta_data.FieldMetaData("queueName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -2493,7 +2493,7 @@
}
public getCompactionJob_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String queueName,
java.lang.String compactor,
@@ -2512,7 +2512,7 @@
*/
public getCompactionJob_args(getCompactionJob_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -2543,11 +2543,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public getCompactionJob_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getCompactionJob_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -2674,7 +2674,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -3020,7 +3020,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -3158,7 +3158,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -3583,7 +3583,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new updateCompactionStatus_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new updateCompactionStatus_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // required
public @org.apache.thrift.annotation.Nullable TCompactionStatusUpdate status; // required
@@ -3670,7 +3670,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -3687,7 +3687,7 @@
}
public updateCompactionStatus_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String externalCompactionId,
TCompactionStatusUpdate status,
@@ -3708,7 +3708,7 @@
public updateCompactionStatus_args(updateCompactionStatus_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -3738,11 +3738,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public updateCompactionStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public updateCompactionStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -3867,7 +3867,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -4212,7 +4212,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -4349,7 +4349,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -4656,7 +4656,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionFailed_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionFailed_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
@@ -4737,7 +4737,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -4752,7 +4752,7 @@
}
public compactionFailed_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String externalCompactionId,
org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent)
@@ -4769,7 +4769,7 @@
*/
public compactionFailed_args(compactionFailed_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -4796,11 +4796,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public compactionFailed_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public compactionFailed_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -4902,7 +4902,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -5207,7 +5207,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -5327,7 +5327,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -5628,7 +5628,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningCompactions_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompactions_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -5701,7 +5701,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -5712,7 +5712,7 @@
}
public getRunningCompactions_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -5725,7 +5725,7 @@
*/
public getRunningCompactions_args(getRunningCompactions_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -5744,11 +5744,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public getRunningCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getRunningCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -5800,7 +5800,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -6014,7 +6014,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -6095,7 +6095,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -6505,7 +6505,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompletedCompactions_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompletedCompactions_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -6578,7 +6578,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -6589,7 +6589,7 @@
}
public getCompletedCompactions_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -6602,7 +6602,7 @@
*/
public getCompletedCompactions_args(getCompletedCompactions_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -6621,11 +6621,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public getCompletedCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getCompletedCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -6677,7 +6677,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -6891,7 +6891,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -6972,7 +6972,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -7383,7 +7383,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancel_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancel_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // required
@@ -7460,7 +7460,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -7473,7 +7473,7 @@
}
public cancel_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String externalCompactionId)
{
@@ -7488,7 +7488,7 @@
*/
public cancel_args(cancel_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -7511,11 +7511,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -7592,7 +7592,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -7850,7 +7850,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -7950,7 +7950,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java
index 90996c5..d7224ae 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/compaction/thrift/CompactorService.java
@@ -29,25 +29,25 @@
public interface Iface {
- public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+ public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
- public java.lang.String getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+ public java.lang.String getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
- public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+ public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
- public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
+ public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException;
}
public interface AsyncIface {
- public void getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException;
+ public void getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException;
- public void getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+ public void getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
- public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException;
+ public void getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException;
- public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
}
@@ -74,13 +74,13 @@
}
@Override
- public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+ public org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
{
send_getRunningCompaction(tinfo, credentials);
return recv_getRunningCompaction();
}
- public void send_getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
getRunningCompaction_args args = new getRunningCompaction_args();
args.setTinfo(tinfo);
@@ -102,13 +102,13 @@
}
@Override
- public java.lang.String getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+ public java.lang.String getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
{
send_getRunningCompactionId(tinfo, credentials);
return recv_getRunningCompactionId();
}
- public void send_getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
getRunningCompactionId_args args = new getRunningCompactionId_args();
args.setTinfo(tinfo);
@@ -130,13 +130,13 @@
}
@Override
- public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+ public java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
{
send_getActiveCompactions(tinfo, credentials);
return recv_getActiveCompactions();
}
- public void send_getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
getActiveCompactions_args args = new getActiveCompactions_args();
args.setTinfo(tinfo);
@@ -158,13 +158,13 @@
}
@Override
- public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+ public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
{
send_cancel(tinfo, credentials, externalCompactionId);
recv_cancel();
}
- public void send_cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+ public void send_cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId) throws org.apache.thrift.TException
{
cancel_args args = new cancel_args();
args.setTinfo(tinfo);
@@ -200,7 +200,7 @@
}
@Override
- public void getRunningCompaction(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
+ public void getRunningCompaction(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
checkReady();
getRunningCompaction_call method_call = new getRunningCompaction_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -208,9 +208,9 @@
}
public static class getRunningCompaction_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getRunningCompaction_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getRunningCompaction_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -238,7 +238,7 @@
}
@Override
- public void getRunningCompactionId(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+ public void getRunningCompactionId(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
checkReady();
getRunningCompactionId_call method_call = new getRunningCompactionId_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -246,9 +246,9 @@
}
public static class getRunningCompactionId_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getRunningCompactionId_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getRunningCompactionId_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -276,7 +276,7 @@
}
@Override
- public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
+ public void getActiveCompactions(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
checkReady();
getActiveCompactions_call method_call = new getActiveCompactions_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -284,9 +284,9 @@
}
public static class getActiveCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getActiveCompactions_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getActiveCompactions_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -314,7 +314,7 @@
}
@Override
- public void cancel(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void cancel(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
cancel_call method_call = new cancel_call(tinfo, credentials, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -322,10 +322,10 @@
}
public static class cancel_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String externalCompactionId;
- public cancel_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public cancel_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -810,7 +810,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningCompaction_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompaction_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -883,7 +883,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -894,7 +894,7 @@
}
public getRunningCompaction_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -907,7 +907,7 @@
*/
public getRunningCompaction_args(getRunningCompaction_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -926,11 +926,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public getRunningCompaction_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getRunningCompaction_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -982,7 +982,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -1196,7 +1196,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -1277,7 +1277,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -1794,7 +1794,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getRunningCompactionId_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getRunningCompactionId_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -1867,7 +1867,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -1878,7 +1878,7 @@
}
public getRunningCompactionId_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -1891,7 +1891,7 @@
*/
public getRunningCompactionId_args(getRunningCompactionId_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -1910,11 +1910,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public getRunningCompactionId_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getRunningCompactionId_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -1966,7 +1966,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -2180,7 +2180,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -2261,7 +2261,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -2773,7 +2773,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveCompactions_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveCompactions_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -2846,7 +2846,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -2857,7 +2857,7 @@
}
public getActiveCompactions_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -2870,7 +2870,7 @@
*/
public getActiveCompactions_args(getActiveCompactions_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -2889,11 +2889,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public getActiveCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getActiveCompactions_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -2945,7 +2945,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -3159,7 +3159,7 @@
switch (schemeField.id) {
case 2: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -3240,7 +3240,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -3808,7 +3808,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancel_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancel_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // required
@@ -3885,7 +3885,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -3898,7 +3898,7 @@
}
public cancel_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String externalCompactionId)
{
@@ -3913,7 +3913,7 @@
*/
public cancel_args(cancel_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -3936,11 +3936,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public cancel_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -4017,7 +4017,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -4275,7 +4275,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -4375,7 +4375,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java
index c9f2809..840a5e5 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/gc/thrift/GCMonitorService.java
@@ -29,13 +29,13 @@
public interface Iface {
- public GCStatus getStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+ public GCStatus getStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
}
public interface AsyncIface {
- public void getStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler) throws org.apache.thrift.TException;
+ public void getStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler) throws org.apache.thrift.TException;
}
@@ -62,13 +62,13 @@
}
@Override
- public GCStatus getStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+ public GCStatus getStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
{
send_getStatus(tinfo, credentials);
return recv_getStatus();
}
- public void send_getStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_getStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
getStatus_args args = new getStatus_args();
args.setTinfo(tinfo);
@@ -109,7 +109,7 @@
}
@Override
- public void getStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler) throws org.apache.thrift.TException {
+ public void getStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler) throws org.apache.thrift.TException {
checkReady();
getStatus_call method_call = new getStatus_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -117,9 +117,9 @@
}
public static class getStatus_call extends org.apache.thrift.async.TAsyncMethodCall<GCStatus> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getStatus_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getStatus_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<GCStatus> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -295,7 +295,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getStatus_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getStatus_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -368,7 +368,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -379,7 +379,7 @@
}
public getStatus_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -392,7 +392,7 @@
*/
public getStatus_args(getStatus_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -411,11 +411,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public getStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -467,7 +467,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -681,7 +681,7 @@
switch (schemeField.id) {
case 2: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -762,7 +762,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportState.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportState.java
similarity index 97%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportState.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportState.java
index 0e5f17d..3cb402c 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportState.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportState.java
@@ -22,7 +22,7 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
public enum BulkImportState implements org.apache.thrift.TEnum {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportStatus.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportStatus.java
similarity index 98%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportStatus.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportStatus.java
index 5c92bee..64de85b 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/BulkImportStatus.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/BulkImportStatus.java
@@ -22,7 +22,7 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
public class BulkImportStatus implements org.apache.thrift.TBase<BulkImportStatus, BulkImportStatus._Fields>, java.io.Serializable, Cloneable, Comparable<BulkImportStatus> {
@@ -525,7 +525,7 @@
break;
case 3: // STATE
if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
- struct.state = org.apache.accumulo.core.master.thrift.BulkImportState.findByValue(iprot.readI32());
+ struct.state = org.apache.accumulo.core.manager.thrift.BulkImportState.findByValue(iprot.readI32());
struct.setStateIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -613,7 +613,7 @@
struct.setFilenameIsSet(true);
}
if (incoming.get(2)) {
- struct.state = org.apache.accumulo.core.master.thrift.BulkImportState.findByValue(iprot.readI32());
+ struct.state = org.apache.accumulo.core.manager.thrift.BulkImportState.findByValue(iprot.readI32());
struct.setStateIsSet(true);
}
}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/Compacting.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/Compacting.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/Compacting.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/Compacting.java
index 7e67064..d21cff5 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/Compacting.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/Compacting.java
@@ -22,7 +22,7 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
public class Compacting implements org.apache.thrift.TBase<Compacting, Compacting._Fields>, java.io.Serializable, Cloneable, Comparable<Compacting> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateOperation.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateOperation.java
index f803d85..a93eb39 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateOperation.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateOperation.java
@@ -34,7 +34,7 @@
TABLE_OFFLINE(5),
TABLE_MERGE(6),
TABLE_DELETE_RANGE(7),
- TABLE_BULK_IMPORT(8),
+ OBSOLETE_TABLE_BULK_IMPORT(8),
TABLE_COMPACT(9),
TABLE_IMPORT(10),
TABLE_EXPORT(11),
@@ -82,7 +82,7 @@
case 7:
return TABLE_DELETE_RANGE;
case 8:
- return TABLE_BULK_IMPORT;
+ return OBSOLETE_TABLE_BULK_IMPORT;
case 9:
return TABLE_COMPACT;
case 10:
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java
index c9c0648..89becfd 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java
@@ -29,29 +29,29 @@
public interface Iface {
- public long beginFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public long beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void executeFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public java.lang.String waitForFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void finishFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public boolean cancelFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
}
public interface AsyncIface {
- public void beginFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
+ public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
- public void executeFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void waitForFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+ public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
- public void finishFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void cancelFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
}
@@ -78,13 +78,13 @@
}
@Override
- public long beginFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public long beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_beginFateOperation(tinfo, credentials);
return recv_beginFateOperation();
}
- public void send_beginFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
beginFateOperation_args args = new beginFateOperation_args();
args.setTinfo(tinfo);
@@ -109,13 +109,13 @@
}
@Override
- public void executeFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_executeFateOperation(tinfo, credentials, opid, op, arguments, options, autoClean);
recv_executeFateOperation();
}
- public void send_executeFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.thrift.TException
+ public void send_executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean) throws org.apache.thrift.TException
{
executeFateOperation_args args = new executeFateOperation_args();
args.setTinfo(tinfo);
@@ -145,13 +145,13 @@
}
@Override
- public java.lang.String waitForFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_waitForFateOperation(tinfo, credentials, opid);
return recv_waitForFateOperation();
}
- public void send_waitForFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+ public void send_waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
{
waitForFateOperation_args args = new waitForFateOperation_args();
args.setTinfo(tinfo);
@@ -180,13 +180,13 @@
}
@Override
- public void finishFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_finishFateOperation(tinfo, credentials, opid);
recv_finishFateOperation();
}
- public void send_finishFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+ public void send_finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
{
finishFateOperation_args args = new finishFateOperation_args();
args.setTinfo(tinfo);
@@ -209,13 +209,13 @@
}
@Override
- public boolean cancelFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_cancelFateOperation(tinfo, credentials, opid);
return recv_cancelFateOperation();
}
- public void send_cancelFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+ public void send_cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
{
cancelFateOperation_args args = new cancelFateOperation_args();
args.setTinfo(tinfo);
@@ -260,7 +260,7 @@
}
@Override
- public void beginFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+ public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
checkReady();
beginFateOperation_call method_call = new beginFateOperation_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -268,9 +268,9 @@
}
public static class beginFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public beginFateOperation_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public beginFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -298,7 +298,7 @@
}
@Override
- public void executeFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
executeFateOperation_call method_call = new executeFateOperation_call(tinfo, credentials, opid, op, arguments, options, autoClean, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -306,14 +306,14 @@
}
public static class executeFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private long opid;
private FateOperation op;
private java.util.List<java.nio.ByteBuffer> arguments;
private java.util.Map<java.lang.String,java.lang.String> options;
private boolean autoClean;
- public executeFateOperation_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public executeFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List<java.nio.ByteBuffer> arguments, java.util.Map<java.lang.String,java.lang.String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -352,7 +352,7 @@
}
@Override
- public void waitForFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+ public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
checkReady();
waitForFateOperation_call method_call = new waitForFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -360,10 +360,10 @@
}
public static class waitForFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private long opid;
- public waitForFateOperation_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public waitForFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -393,7 +393,7 @@
}
@Override
- public void finishFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
finishFateOperation_call method_call = new finishFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -401,10 +401,10 @@
}
public static class finishFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private long opid;
- public finishFateOperation_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public finishFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -435,7 +435,7 @@
}
@Override
- public void cancelFateOperation(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
checkReady();
cancelFateOperation_call method_call = new cancelFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -443,10 +443,10 @@
}
public static class cancelFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private long opid;
- public cancelFateOperation_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public cancelFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1088,7 +1088,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new beginFateOperation_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new beginFateOperation_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -1161,7 +1161,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -1172,7 +1172,7 @@
}
public beginFateOperation_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -1185,7 +1185,7 @@
*/
public beginFateOperation_args(beginFateOperation_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -1204,11 +1204,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public beginFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public beginFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -1260,7 +1260,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -1474,7 +1474,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -1555,7 +1555,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -2176,7 +2176,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new executeFateOperation_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new executeFateOperation_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public long opid; // required
/**
@@ -2280,7 +2280,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -2304,7 +2304,7 @@
}
public executeFateOperation_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
long opid,
FateOperation op,
@@ -2330,7 +2330,7 @@
public executeFateOperation_args(executeFateOperation_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -2369,11 +2369,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public executeFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public executeFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -2581,7 +2581,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -3005,7 +3005,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -3040,13 +3040,13 @@
case 5: // ARGUMENTS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
- org.apache.thrift.protocol.TList _list60 = iprot.readListBegin();
- struct.arguments = new java.util.ArrayList<java.nio.ByteBuffer>(_list60.size);
- @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem61;
- for (int _i62 = 0; _i62 < _list60.size; ++_i62)
+ org.apache.thrift.protocol.TList _list86 = iprot.readListBegin();
+ struct.arguments = new java.util.ArrayList<java.nio.ByteBuffer>(_list86.size);
+ @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem87;
+ for (int _i88 = 0; _i88 < _list86.size; ++_i88)
{
- _elem61 = iprot.readBinary();
- struct.arguments.add(_elem61);
+ _elem87 = iprot.readBinary();
+ struct.arguments.add(_elem87);
}
iprot.readListEnd();
}
@@ -3058,15 +3058,15 @@
case 6: // OPTIONS
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map63 = iprot.readMapBegin();
- struct.options = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map63.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key64;
- @org.apache.thrift.annotation.Nullable java.lang.String _val65;
- for (int _i66 = 0; _i66 < _map63.size; ++_i66)
+ org.apache.thrift.protocol.TMap _map89 = iprot.readMapBegin();
+ struct.options = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map89.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key90;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val91;
+ for (int _i92 = 0; _i92 < _map89.size; ++_i92)
{
- _key64 = iprot.readString();
- _val65 = iprot.readString();
- struct.options.put(_key64, _val65);
+ _key90 = iprot.readString();
+ _val91 = iprot.readString();
+ struct.options.put(_key90, _val91);
}
iprot.readMapEnd();
}
@@ -3121,9 +3121,9 @@
oprot.writeFieldBegin(ARGUMENTS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.arguments.size()));
- for (java.nio.ByteBuffer _iter67 : struct.arguments)
+ for (java.nio.ByteBuffer _iter93 : struct.arguments)
{
- oprot.writeBinary(_iter67);
+ oprot.writeBinary(_iter93);
}
oprot.writeListEnd();
}
@@ -3133,10 +3133,10 @@
oprot.writeFieldBegin(OPTIONS_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.options.size()));
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter68 : struct.options.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter94 : struct.options.entrySet())
{
- oprot.writeString(_iter68.getKey());
- oprot.writeString(_iter68.getValue());
+ oprot.writeString(_iter94.getKey());
+ oprot.writeString(_iter94.getValue());
}
oprot.writeMapEnd();
}
@@ -3201,19 +3201,19 @@
if (struct.isSetArguments()) {
{
oprot.writeI32(struct.arguments.size());
- for (java.nio.ByteBuffer _iter69 : struct.arguments)
+ for (java.nio.ByteBuffer _iter95 : struct.arguments)
{
- oprot.writeBinary(_iter69);
+ oprot.writeBinary(_iter95);
}
}
}
if (struct.isSetOptions()) {
{
oprot.writeI32(struct.options.size());
- for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter70 : struct.options.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter96 : struct.options.entrySet())
{
- oprot.writeString(_iter70.getKey());
- oprot.writeString(_iter70.getValue());
+ oprot.writeString(_iter96.getKey());
+ oprot.writeString(_iter96.getValue());
}
}
}
@@ -3227,7 +3227,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(7);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -3246,28 +3246,28 @@
}
if (incoming.get(4)) {
{
- org.apache.thrift.protocol.TList _list71 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
- struct.arguments = new java.util.ArrayList<java.nio.ByteBuffer>(_list71.size);
- @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem72;
- for (int _i73 = 0; _i73 < _list71.size; ++_i73)
+ org.apache.thrift.protocol.TList _list97 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+ struct.arguments = new java.util.ArrayList<java.nio.ByteBuffer>(_list97.size);
+ @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem98;
+ for (int _i99 = 0; _i99 < _list97.size; ++_i99)
{
- _elem72 = iprot.readBinary();
- struct.arguments.add(_elem72);
+ _elem98 = iprot.readBinary();
+ struct.arguments.add(_elem98);
}
}
struct.setArgumentsIsSet(true);
}
if (incoming.get(5)) {
{
- org.apache.thrift.protocol.TMap _map74 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
- struct.options = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map74.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key75;
- @org.apache.thrift.annotation.Nullable java.lang.String _val76;
- for (int _i77 = 0; _i77 < _map74.size; ++_i77)
+ org.apache.thrift.protocol.TMap _map100 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
+ struct.options = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map100.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key101;
+ @org.apache.thrift.annotation.Nullable java.lang.String _val102;
+ for (int _i103 = 0; _i103 < _map100.size; ++_i103)
{
- _key75 = iprot.readString();
- _val76 = iprot.readString();
- struct.options.put(_key75, _val76);
+ _key101 = iprot.readString();
+ _val102 = iprot.readString();
+ struct.options.put(_key101, _val102);
}
}
struct.setOptionsIsSet(true);
@@ -3889,7 +3889,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new waitForFateOperation_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new waitForFateOperation_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public long opid; // required
@@ -3968,7 +3968,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -3981,7 +3981,7 @@
}
public waitForFateOperation_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
long opid)
{
@@ -3998,7 +3998,7 @@
public waitForFateOperation_args(waitForFateOperation_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -4020,11 +4020,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public waitForFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public waitForFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -4099,7 +4099,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -4353,7 +4353,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -4451,7 +4451,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -5182,7 +5182,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new finishFateOperation_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new finishFateOperation_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public long opid; // required
@@ -5261,7 +5261,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -5274,7 +5274,7 @@
}
public finishFateOperation_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
long opid)
{
@@ -5291,7 +5291,7 @@
public finishFateOperation_args(finishFateOperation_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -5313,11 +5313,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public finishFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public finishFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -5392,7 +5392,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -5646,7 +5646,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -5744,7 +5744,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -6263,7 +6263,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelFateOperation_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelFateOperation_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public long opid; // required
@@ -6342,7 +6342,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -6355,7 +6355,7 @@
}
public cancelFateOperation_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
long opid)
{
@@ -6372,7 +6372,7 @@
public cancelFateOperation_args(cancelFateOperation_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -6394,11 +6394,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public cancelFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public cancelFateOperation_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -6473,7 +6473,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -6727,7 +6727,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -6825,7 +6825,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java
index 5e17421..b2c82b5 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java
@@ -29,93 +29,89 @@
public interface Iface {
- public long initiateFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public long initiateFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void waitForFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public void waitForFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void setTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
+ public void setTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
- public void modifyTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException;
+ public void modifyTableProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException;
- public void removeTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public void removeTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void setNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
+ public void setNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
- public void modifyNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, org.apache.thrift.TException;
+ public void modifyNamespaceProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, org.apache.thrift.TException;
- public void removeNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public void removeNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void setManagerGoalState(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public void setManagerGoalState(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void shutdown(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public void shutdown(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public void shutdownTabletServer(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
+ public void setSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException;
- public void modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException;
+ public void modifySystemProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException;
- public void removeSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public void removeSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public ManagerMonitorInfo getManagerStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public ManagerMonitorInfo getManagerStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void waitForBalance(org.apache.accumulo.core.trace.thrift.TInfo tinfo) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public void waitForBalance(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public void reportSplitExtent(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException;
+ public void reportSplitExtent(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException;
- public void reportTabletStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException;
+ public void reportTabletStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException;
- public java.util.List<java.lang.String> getActiveTservers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public java.util.List<java.lang.String> getActiveTservers(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
- public org.apache.accumulo.core.securityImpl.thrift.TDelegationToken getDelegationToken(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
-
- public boolean drainReplicationTable(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
+ public org.apache.accumulo.core.securityImpl.thrift.TDelegationToken getDelegationToken(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException;
}
public interface AsyncIface {
- public void initiateFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
+ public void initiateFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
- public void waitForFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void waitForFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void setTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void setTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void modifyTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void modifyTableProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void removeTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void removeTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void setNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void setNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void modifyNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void modifyNamespaceProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void removeNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void removeNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void setManagerGoalState(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void setManagerGoalState(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void shutdown(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void shutdown(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void shutdownTabletServer(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void setSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void modifySystemProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void removeSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void removeSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void getManagerStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler) throws org.apache.thrift.TException;
+ public void getManagerStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler) throws org.apache.thrift.TException;
- public void waitForBalance(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void waitForBalance(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void reportSplitExtent(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void reportSplitExtent(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void reportTabletStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+ public void reportTabletStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
- public void getActiveTservers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
+ public void getActiveTservers(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException;
- public void getDelegationToken(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler) throws org.apache.thrift.TException;
-
- public void drainReplicationTable(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+ public void getDelegationToken(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler) throws org.apache.thrift.TException;
}
@@ -142,13 +138,13 @@
}
@Override
- public long initiateFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public long initiateFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_initiateFlush(tinfo, credentials, tableName);
return recv_initiateFlush();
}
- public void send_initiateFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
+ public void send_initiateFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName) throws org.apache.thrift.TException
{
initiateFlush_args args = new initiateFlush_args();
args.setTinfo(tinfo);
@@ -177,13 +173,13 @@
}
@Override
- public void waitForFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public void waitForFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_waitForFlush(tinfo, credentials, tableName, startRow, endRow, flushID, maxLoops);
recv_waitForFlush();
}
- public void send_waitForFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.thrift.TException
+ public void send_waitForFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops) throws org.apache.thrift.TException
{
waitForFlush_args args = new waitForFlush_args();
args.setTinfo(tinfo);
@@ -213,13 +209,13 @@
}
@Override
- public void setTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
+ public void setTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
{
send_setTableProperty(tinfo, credentials, tableName, property, value);
recv_setTableProperty();
}
- public void send_setTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
+ public void send_setTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
{
setTableProperty_args args = new setTableProperty_args();
args.setTinfo(tinfo);
@@ -250,13 +246,13 @@
}
@Override
- public void modifyTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException
+ public void modifyTableProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException
{
send_modifyTableProperties(tinfo, credentials, tableName, vProperties);
recv_modifyTableProperties();
}
- public void send_modifyTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
+ public void send_modifyTableProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
{
modifyTableProperties_args args = new modifyTableProperties_args();
args.setTinfo(tinfo);
@@ -289,13 +285,13 @@
}
@Override
- public void removeTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public void removeTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_removeTableProperty(tinfo, credentials, tableName, property);
recv_removeTableProperty();
}
- public void send_removeTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.thrift.TException
+ public void send_removeTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property) throws org.apache.thrift.TException
{
removeTableProperty_args args = new removeTableProperty_args();
args.setTinfo(tinfo);
@@ -322,13 +318,13 @@
}
@Override
- public void setNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
+ public void setNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
{
send_setNamespaceProperty(tinfo, credentials, ns, property, value);
recv_setNamespaceProperty();
}
- public void send_setNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
+ public void send_setNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
{
setNamespaceProperty_args args = new setNamespaceProperty_args();
args.setTinfo(tinfo);
@@ -359,13 +355,13 @@
}
@Override
- public void modifyNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, org.apache.thrift.TException
+ public void modifyNamespaceProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, org.apache.thrift.TException
{
send_modifyNamespaceProperties(tinfo, credentials, ns, vProperties);
recv_modifyNamespaceProperties();
}
- public void send_modifyNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
+ public void send_modifyNamespaceProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
{
modifyNamespaceProperties_args args = new modifyNamespaceProperties_args();
args.setTinfo(tinfo);
@@ -395,13 +391,13 @@
}
@Override
- public void removeNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public void removeNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_removeNamespaceProperty(tinfo, credentials, ns, property);
recv_removeNamespaceProperty();
}
- public void send_removeNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.thrift.TException
+ public void send_removeNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property) throws org.apache.thrift.TException
{
removeNamespaceProperty_args args = new removeNamespaceProperty_args();
args.setTinfo(tinfo);
@@ -428,13 +424,13 @@
}
@Override
- public void setManagerGoalState(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public void setManagerGoalState(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_setManagerGoalState(tinfo, credentials, state);
recv_setManagerGoalState();
}
- public void send_setManagerGoalState(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.thrift.TException
+ public void send_setManagerGoalState(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state) throws org.apache.thrift.TException
{
setManagerGoalState_args args = new setManagerGoalState_args();
args.setTinfo(tinfo);
@@ -457,13 +453,13 @@
}
@Override
- public void shutdown(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public void shutdown(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_shutdown(tinfo, credentials, stopTabletServers);
recv_shutdown();
}
- public void send_shutdown(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.thrift.TException
+ public void send_shutdown(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers) throws org.apache.thrift.TException
{
shutdown_args args = new shutdown_args();
args.setTinfo(tinfo);
@@ -486,13 +482,13 @@
}
@Override
- public void shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public void shutdownTabletServer(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_shutdownTabletServer(tinfo, credentials, tabletServer, force);
recv_shutdownTabletServer();
}
- public void send_shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.thrift.TException
+ public void send_shutdownTabletServer(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force) throws org.apache.thrift.TException
{
shutdownTabletServer_args args = new shutdownTabletServer_args();
args.setTinfo(tinfo);
@@ -516,13 +512,13 @@
}
@Override
- public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
+ public void setSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, ThriftPropertyException, org.apache.thrift.TException
{
send_setSystemProperty(tinfo, credentials, property, value);
recv_setSystemProperty();
}
- public void send_setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
+ public void send_setSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value) throws org.apache.thrift.TException
{
setSystemProperty_args args = new setSystemProperty_args();
args.setTinfo(tinfo);
@@ -549,13 +545,13 @@
}
@Override
- public void modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException
+ public void modifySystemProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.accumulo.core.clientImpl.thrift.ThriftConcurrentModificationException, ThriftPropertyException, org.apache.thrift.TException
{
send_modifySystemProperties(tinfo, credentials, vProperties);
recv_modifySystemProperties();
}
- public void send_modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
+ public void send_modifySystemProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties) throws org.apache.thrift.TException
{
modifySystemProperties_args args = new modifySystemProperties_args();
args.setTinfo(tinfo);
@@ -584,13 +580,13 @@
}
@Override
- public void removeSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public void removeSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_removeSystemProperty(tinfo, credentials, property);
recv_removeSystemProperty();
}
- public void send_removeSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.thrift.TException
+ public void send_removeSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property) throws org.apache.thrift.TException
{
removeSystemProperty_args args = new removeSystemProperty_args();
args.setTinfo(tinfo);
@@ -613,13 +609,13 @@
}
@Override
- public ManagerMonitorInfo getManagerStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public ManagerMonitorInfo getManagerStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_getManagerStats(tinfo, credentials);
return recv_getManagerStats();
}
- public void send_getManagerStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_getManagerStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
getManagerStats_args args = new getManagerStats_args();
args.setTinfo(tinfo);
@@ -644,13 +640,13 @@
}
@Override
- public void waitForBalance(org.apache.accumulo.core.trace.thrift.TInfo tinfo) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public void waitForBalance(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_waitForBalance(tinfo);
recv_waitForBalance();
}
- public void send_waitForBalance(org.apache.accumulo.core.trace.thrift.TInfo tinfo) throws org.apache.thrift.TException
+ public void send_waitForBalance(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) throws org.apache.thrift.TException
{
waitForBalance_args args = new waitForBalance_args();
args.setTinfo(tinfo);
@@ -668,12 +664,12 @@
}
@Override
- public void reportSplitExtent(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException
+ public void reportSplitExtent(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException
{
send_reportSplitExtent(tinfo, credentials, serverName, split);
}
- public void send_reportSplitExtent(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException
+ public void send_reportSplitExtent(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split) throws org.apache.thrift.TException
{
reportSplitExtent_args args = new reportSplitExtent_args();
args.setTinfo(tinfo);
@@ -684,12 +680,12 @@
}
@Override
- public void reportTabletStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException
+ public void reportTabletStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException
{
send_reportTabletStatus(tinfo, credentials, serverName, status, tablet);
}
- public void send_reportTabletStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException
+ public void send_reportTabletStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet) throws org.apache.thrift.TException
{
reportTabletStatus_args args = new reportTabletStatus_args();
args.setTinfo(tinfo);
@@ -701,13 +697,13 @@
}
@Override
- public java.util.List<java.lang.String> getActiveTservers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public java.util.List<java.lang.String> getActiveTservers(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_getActiveTservers(tinfo, credentials);
return recv_getActiveTservers();
}
- public void send_getActiveTservers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+ public void send_getActiveTservers(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
{
getActiveTservers_args args = new getActiveTservers_args();
args.setTinfo(tinfo);
@@ -732,13 +728,13 @@
}
@Override
- public org.apache.accumulo.core.securityImpl.thrift.TDelegationToken getDelegationToken(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
+ public org.apache.accumulo.core.securityImpl.thrift.TDelegationToken getDelegationToken(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
{
send_getDelegationToken(tinfo, credentials, cfg);
return recv_getDelegationToken();
}
- public void send_getDelegationToken(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.thrift.TException
+ public void send_getDelegationToken(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg) throws org.apache.thrift.TException
{
getDelegationToken_args args = new getDelegationToken_args();
args.setTinfo(tinfo);
@@ -763,36 +759,6 @@
throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getDelegationToken failed: unknown result");
}
- @Override
- public boolean drainReplicationTable(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch) throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
- {
- send_drainReplicationTable(tfino, credentials, tableName, logsToWatch);
- return recv_drainReplicationTable();
- }
-
- public void send_drainReplicationTable(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch) throws org.apache.thrift.TException
- {
- drainReplicationTable_args args = new drainReplicationTable_args();
- args.setTfino(tfino);
- args.setCredentials(credentials);
- args.setTableName(tableName);
- args.setLogsToWatch(logsToWatch);
- sendBase("drainReplicationTable", args);
- }
-
- public boolean recv_drainReplicationTable() throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException
- {
- drainReplicationTable_result result = new drainReplicationTable_result();
- receiveBase(result, "drainReplicationTable");
- if (result.isSetSuccess()) {
- return result.success;
- }
- if (result.tnase != null) {
- throw result.tnase;
- }
- throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "drainReplicationTable failed: unknown result");
- }
-
}
public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -813,7 +779,7 @@
}
@Override
- public void initiateFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+ public void initiateFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
checkReady();
initiateFlush_call method_call = new initiateFlush_call(tinfo, credentials, tableName, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -821,10 +787,10 @@
}
public static class initiateFlush_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String tableName;
- public initiateFlush_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public initiateFlush_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -854,7 +820,7 @@
}
@Override
- public void waitForFlush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void waitForFlush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
waitForFlush_call method_call = new waitForFlush_call(tinfo, credentials, tableName, startRow, endRow, flushID, maxLoops, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -862,14 +828,14 @@
}
public static class waitForFlush_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String tableName;
private java.nio.ByteBuffer startRow;
private java.nio.ByteBuffer endRow;
private long flushID;
private long maxLoops;
- public waitForFlush_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public waitForFlush_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, long flushID, long maxLoops, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -908,7 +874,7 @@
}
@Override
- public void setTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void setTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
setTableProperty_call method_call = new setTableProperty_call(tinfo, credentials, tableName, property, value, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -916,12 +882,12 @@
}
public static class setTableProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String tableName;
private java.lang.String property;
private java.lang.String value;
- public setTableProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public setTableProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -956,7 +922,7 @@
}
@Override
- public void modifyTableProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void modifyTableProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
modifyTableProperties_call method_call = new modifyTableProperties_call(tinfo, credentials, tableName, vProperties, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -964,11 +930,11 @@
}
public static class modifyTableProperties_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String tableName;
private org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties;
- public modifyTableProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public modifyTableProperties_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1001,7 +967,7 @@
}
@Override
- public void removeTableProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void removeTableProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
removeTableProperty_call method_call = new removeTableProperty_call(tinfo, credentials, tableName, property, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1009,11 +975,11 @@
}
public static class removeTableProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String tableName;
private java.lang.String property;
- public removeTableProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public removeTableProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1046,7 +1012,7 @@
}
@Override
- public void setNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void setNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
setNamespaceProperty_call method_call = new setNamespaceProperty_call(tinfo, credentials, ns, property, value, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1054,12 +1020,12 @@
}
public static class setNamespaceProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String ns;
private java.lang.String property;
private java.lang.String value;
- public setNamespaceProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public setNamespaceProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1094,7 +1060,7 @@
}
@Override
- public void modifyNamespaceProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void modifyNamespaceProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
modifyNamespaceProperties_call method_call = new modifyNamespaceProperties_call(tinfo, credentials, ns, vProperties, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1102,11 +1068,11 @@
}
public static class modifyNamespaceProperties_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String ns;
private org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties;
- public modifyNamespaceProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public modifyNamespaceProperties_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1139,7 +1105,7 @@
}
@Override
- public void removeNamespaceProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void removeNamespaceProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
removeNamespaceProperty_call method_call = new removeNamespaceProperty_call(tinfo, credentials, ns, property, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1147,11 +1113,11 @@
}
public static class removeNamespaceProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String ns;
private java.lang.String property;
- public removeNamespaceProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public removeNamespaceProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String ns, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1184,7 +1150,7 @@
}
@Override
- public void setManagerGoalState(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void setManagerGoalState(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
setManagerGoalState_call method_call = new setManagerGoalState_call(tinfo, credentials, state, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1192,10 +1158,10 @@
}
public static class setManagerGoalState_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private ManagerGoalState state;
- public setManagerGoalState_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public setManagerGoalState_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, ManagerGoalState state, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1226,7 +1192,7 @@
}
@Override
- public void shutdown(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void shutdown(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
shutdown_call method_call = new shutdown_call(tinfo, credentials, stopTabletServers, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1234,10 +1200,10 @@
}
public static class shutdown_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private boolean stopTabletServers;
- public shutdown_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public shutdown_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, boolean stopTabletServers, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1268,7 +1234,7 @@
}
@Override
- public void shutdownTabletServer(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void shutdownTabletServer(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
shutdownTabletServer_call method_call = new shutdownTabletServer_call(tinfo, credentials, tabletServer, force, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1276,11 +1242,11 @@
}
public static class shutdownTabletServer_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String tabletServer;
private boolean force;
- public shutdownTabletServer_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public shutdownTabletServer_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tabletServer, boolean force, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1313,7 +1279,7 @@
}
@Override
- public void setSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void setSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
setSystemProperty_call method_call = new setSystemProperty_call(tinfo, credentials, property, value, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1321,11 +1287,11 @@
}
public static class setSystemProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String property;
private java.lang.String value;
- public setSystemProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public setSystemProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1358,7 +1324,7 @@
}
@Override
- public void modifySystemProperties(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void modifySystemProperties(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
modifySystemProperties_call method_call = new modifySystemProperties_call(tinfo, credentials, vProperties, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1366,10 +1332,10 @@
}
public static class modifySystemProperties_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties;
- public modifySystemProperties_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public modifySystemProperties_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1400,7 +1366,7 @@
}
@Override
- public void removeSystemProperty(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void removeSystemProperty(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
removeSystemProperty_call method_call = new removeSystemProperty_call(tinfo, credentials, property, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1408,10 +1374,10 @@
}
public static class removeSystemProperty_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String property;
- public removeSystemProperty_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public removeSystemProperty_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String property, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1442,7 +1408,7 @@
}
@Override
- public void getManagerStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler) throws org.apache.thrift.TException {
+ public void getManagerStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler) throws org.apache.thrift.TException {
checkReady();
getManagerStats_call method_call = new getManagerStats_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1450,9 +1416,9 @@
}
public static class getManagerStats_call extends org.apache.thrift.async.TAsyncMethodCall<ManagerMonitorInfo> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getManagerStats_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getManagerStats_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<ManagerMonitorInfo> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1480,7 +1446,7 @@
}
@Override
- public void waitForBalance(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void waitForBalance(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
waitForBalance_call method_call = new waitForBalance_call(tinfo, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1488,8 +1454,8 @@
}
public static class waitForBalance_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
- public waitForBalance_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ public waitForBalance_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
}
@@ -1516,7 +1482,7 @@
}
@Override
- public void reportSplitExtent(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void reportSplitExtent(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
reportSplitExtent_call method_call = new reportSplitExtent_call(tinfo, credentials, serverName, split, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1524,11 +1490,11 @@
}
public static class reportSplitExtent_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String serverName;
private TabletSplit split;
- public reportSplitExtent_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public reportSplitExtent_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletSplit split, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, true);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1560,7 +1526,7 @@
}
@Override
- public void reportTabletStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ public void reportTabletStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
checkReady();
reportTabletStatus_call method_call = new reportTabletStatus_call(tinfo, credentials, serverName, status, tablet, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1568,12 +1534,12 @@
}
public static class reportTabletStatus_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private java.lang.String serverName;
private TabletLoadState status;
private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet;
- public reportTabletStatus_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public reportTabletStatus_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String serverName, TabletLoadState status, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent tablet, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, true);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1607,7 +1573,7 @@
}
@Override
- public void getActiveTservers(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+ public void getActiveTservers(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
checkReady();
getActiveTservers_call method_call = new getActiveTservers_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1615,9 +1581,9 @@
}
public static class getActiveTservers_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<java.lang.String>> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getActiveTservers_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getActiveTservers_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1645,7 +1611,7 @@
}
@Override
- public void getDelegationToken(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler) throws org.apache.thrift.TException {
+ public void getDelegationToken(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler) throws org.apache.thrift.TException {
checkReady();
getDelegationToken_call method_call = new getDelegationToken_call(tinfo, credentials, cfg, resultHandler, this, ___protocolFactory, ___transport);
this.___currentMethod = method_call;
@@ -1653,10 +1619,10 @@
}
public static class getDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> {
- private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
private org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg;
- public getDelegationToken_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ public getDelegationToken_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.securityImpl.thrift.TDelegationToken> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
super(client, protocolFactory, transport, resultHandler, false);
this.tinfo = tinfo;
this.credentials = credentials;
@@ -1685,50 +1651,6 @@
}
}
- @Override
- public void drainReplicationTable(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
- checkReady();
- drainReplicationTable_call method_call = new drainReplicationTable_call(tfino, credentials, tableName, logsToWatch, resultHandler, this, ___protocolFactory, ___transport);
- this.___currentMethod = method_call;
- ___manager.call(method_call);
- }
-
- public static class drainReplicationTable_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
- private org.apache.accumulo.core.trace.thrift.TInfo tfino;
- private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- private java.lang.String tableName;
- private java.util.Set<java.lang.String> logsToWatch;
- public drainReplicationTable_call(org.apache.accumulo.core.trace.thrift.TInfo tfino, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Set<java.lang.String> logsToWatch, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
- super(client, protocolFactory, transport, resultHandler, false);
- this.tfino = tfino;
- this.credentials = credentials;
- this.tableName = tableName;
- this.logsToWatch = logsToWatch;
- }
-
- @Override
- public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
- prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("drainReplicationTable", org.apache.thrift.protocol.TMessageType.CALL, 0));
- drainReplicationTable_args args = new drainReplicationTable_args();
- args.setTfino(tfino);
- args.setCredentials(credentials);
- args.setTableName(tableName);
- args.setLogsToWatch(logsToWatch);
- args.write(prot);
- prot.writeMessageEnd();
- }
-
- @Override
- public java.lang.Boolean getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException {
- if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
- throw new java.lang.IllegalStateException("Method call not finished!");
- }
- org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
- org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
- return (new Client(prot)).recv_drainReplicationTable();
- }
- }
-
}
public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -1762,7 +1684,6 @@
processMap.put("reportTabletStatus", new reportTabletStatus());
processMap.put("getActiveTservers", new getActiveTservers());
processMap.put("getDelegationToken", new getDelegationToken());
- processMap.put("drainReplicationTable", new drainReplicationTable());
return processMap;
}
@@ -2463,39 +2384,6 @@
}
}
- public static class drainReplicationTable<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drainReplicationTable_args> {
- public drainReplicationTable() {
- super("drainReplicationTable");
- }
-
- @Override
- public drainReplicationTable_args getEmptyArgsInstance() {
- return new drainReplicationTable_args();
- }
-
- @Override
- protected boolean isOneway() {
- return false;
- }
-
- @Override
- protected boolean rethrowUnhandledExceptions() {
- return false;
- }
-
- @Override
- public drainReplicationTable_result getResult(I iface, drainReplicationTable_args args) throws org.apache.thrift.TException {
- drainReplicationTable_result result = new drainReplicationTable_result();
- try {
- result.success = iface.drainReplicationTable(args.tfino, args.credentials, args.tableName, args.logsToWatch);
- result.setSuccessIsSet(true);
- } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) {
- result.tnase = tnase;
- }
- return result;
- }
- }
-
}
public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
@@ -2529,7 +2417,6 @@
processMap.put("reportTabletStatus", new reportTabletStatus());
processMap.put("getActiveTservers", new getActiveTservers());
processMap.put("getDelegationToken", new getDelegationToken());
- processMap.put("drainReplicationTable", new drainReplicationTable());
return processMap;
}
@@ -4010,78 +3897,6 @@
}
}
- public static class drainReplicationTable<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drainReplicationTable_args, java.lang.Boolean> {
- public drainReplicationTable() {
- super("drainReplicationTable");
- }
-
- @Override
- public drainReplicationTable_args getEmptyArgsInstance() {
- return new drainReplicationTable_args();
- }
-
- @Override
- public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
- final org.apache.thrift.AsyncProcessFunction fcall = this;
- return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() {
- @Override
- public void onComplete(java.lang.Boolean o) {
- drainReplicationTable_result result = new drainReplicationTable_result();
- result.success = o;
- result.setSuccessIsSet(true);
- try {
- fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
- } catch (org.apache.thrift.transport.TTransportException e) {
- _LOGGER.error("TTransportException writing to internal frame buffer", e);
- fb.close();
- } catch (java.lang.Exception e) {
- _LOGGER.error("Exception writing to internal frame buffer", e);
- onError(e);
- }
- }
- @Override
- public void onError(java.lang.Exception e) {
- byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
- org.apache.thrift.TSerializable msg;
- drainReplicationTable_result result = new drainReplicationTable_result();
- if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException) {
- result.tnase = (org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException) e;
- result.setTnaseIsSet(true);
- msg = result;
- } else if (e instanceof org.apache.thrift.transport.TTransportException) {
- _LOGGER.error("TTransportException inside handler", e);
- fb.close();
- return;
- } else if (e instanceof org.apache.thrift.TApplicationException) {
- _LOGGER.error("TApplicationException inside handler", e);
- msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
- msg = (org.apache.thrift.TApplicationException)e;
- } else {
- _LOGGER.error("Exception inside handler", e);
- msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
- msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
- }
- try {
- fcall.sendResponse(fb,msg,msgType,seqid);
- } catch (java.lang.Exception ex) {
- _LOGGER.error("Exception writing to internal frame buffer", ex);
- fb.close();
- }
- }
- };
- }
-
- @Override
- protected boolean isOneway() {
- return false;
- }
-
- @Override
- public void start(I iface, drainReplicationTable_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
- iface.drainReplicationTable(args.tfino, args.credentials, args.tableName, args.logsToWatch,resultHandler);
- }
- }
-
}
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
@@ -4095,7 +3910,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new initiateFlush_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new initiateFlush_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
@@ -4172,7 +3987,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -4185,7 +4000,7 @@
}
public initiateFlush_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String tableName)
{
@@ -4200,7 +4015,7 @@
*/
public initiateFlush_args(initiateFlush_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -4223,11 +4038,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public initiateFlush_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public initiateFlush_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -4304,7 +4119,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -4562,7 +4377,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -4662,7 +4477,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -5394,7 +5209,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new waitForFlush_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new waitForFlush_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer startRow; // required
@@ -5490,7 +5305,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -5511,7 +5326,7 @@
}
public waitForFlush_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String tableName,
java.nio.ByteBuffer startRow,
@@ -5537,7 +5352,7 @@
public waitForFlush_args(waitForFlush_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -5574,11 +5389,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public waitForFlush_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public waitForFlush_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -5769,7 +5584,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -6201,7 +6016,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -6373,7 +6188,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(7);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -7017,7 +6832,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new setTableProperty_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new setTableProperty_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
public @org.apache.thrift.annotation.Nullable java.lang.String property; // required
@@ -7102,7 +6917,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -7119,7 +6934,7 @@
}
public setTableProperty_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String tableName,
java.lang.String property,
@@ -7138,7 +6953,7 @@
*/
public setTableProperty_args(setTableProperty_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -7169,11 +6984,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public setTableProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public setTableProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -7300,7 +7115,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -7646,7 +7461,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -7784,7 +7599,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -8526,7 +8341,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyTableProperties_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyTableProperties_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties; // required
@@ -8607,7 +8422,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -8622,7 +8437,7 @@
}
public modifyTableProperties_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String tableName,
org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties)
@@ -8639,7 +8454,7 @@
*/
public modifyTableProperties_args(modifyTableProperties_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -8666,11 +8481,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public modifyTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public modifyTableProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -8772,7 +8587,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -9077,7 +8892,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -9197,7 +9012,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -10043,7 +9858,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new removeTableProperty_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new removeTableProperty_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
public @org.apache.thrift.annotation.Nullable java.lang.String property; // required
@@ -10124,7 +9939,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -10139,7 +9954,7 @@
}
public removeTableProperty_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String tableName,
java.lang.String property)
@@ -10156,7 +9971,7 @@
*/
public removeTableProperty_args(removeTableProperty_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -10183,11 +9998,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public removeTableProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public removeTableProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -10289,7 +10104,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -10591,7 +10406,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -10710,7 +10525,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -11342,7 +11157,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new setNamespaceProperty_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new setNamespaceProperty_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
public @org.apache.thrift.annotation.Nullable java.lang.String property; // required
@@ -11427,7 +11242,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -11444,7 +11259,7 @@
}
public setNamespaceProperty_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String ns,
java.lang.String property,
@@ -11463,7 +11278,7 @@
*/
public setNamespaceProperty_args(setNamespaceProperty_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -11494,11 +11309,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public setNamespaceProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public setNamespaceProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -11625,7 +11440,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -11971,7 +11786,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -12109,7 +11924,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -12851,7 +12666,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifyNamespaceProperties_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifyNamespaceProperties_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties; // required
@@ -12932,7 +12747,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -12947,7 +12762,7 @@
}
public modifyNamespaceProperties_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String ns,
org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties)
@@ -12964,7 +12779,7 @@
*/
public modifyNamespaceProperties_args(modifyNamespaceProperties_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -12991,11 +12806,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public modifyNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public modifyNamespaceProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -13097,7 +12912,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -13402,7 +13217,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -13522,7 +13337,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -14261,7 +14076,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new removeNamespaceProperty_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new removeNamespaceProperty_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String ns; // required
public @org.apache.thrift.annotation.Nullable java.lang.String property; // required
@@ -14342,7 +14157,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -14357,7 +14172,7 @@
}
public removeNamespaceProperty_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String ns,
java.lang.String property)
@@ -14374,7 +14189,7 @@
*/
public removeNamespaceProperty_args(removeNamespaceProperty_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -14401,11 +14216,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public removeNamespaceProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public removeNamespaceProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -14507,7 +14322,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -14809,7 +14624,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -14928,7 +14743,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -15558,7 +15373,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new setManagerGoalState_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new setManagerGoalState_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/**
*
@@ -15643,7 +15458,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -15656,7 +15471,7 @@
}
public setManagerGoalState_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
ManagerGoalState state)
{
@@ -15671,7 +15486,7 @@
*/
public setManagerGoalState_args(setManagerGoalState_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -15694,11 +15509,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public setManagerGoalState_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public setManagerGoalState_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -15783,7 +15598,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -16041,7 +15856,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -16141,7 +15956,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -16660,7 +16475,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new shutdown_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new shutdown_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public boolean stopTabletServers; // required
@@ -16739,7 +16554,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.STOP_TABLET_SERVERS, new org.apache.thrift.meta_data.FieldMetaData("stopTabletServers", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -16752,7 +16567,7 @@
}
public shutdown_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
boolean stopTabletServers)
{
@@ -16769,7 +16584,7 @@
public shutdown_args(shutdown_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -16791,11 +16606,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public shutdown_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public shutdown_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -16870,7 +16685,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -17124,7 +16939,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -17222,7 +17037,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -17742,7 +17557,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new shutdownTabletServer_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new shutdownTabletServer_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String tabletServer; // required
public boolean force; // required
@@ -17825,7 +17640,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.TABLET_SERVER, new org.apache.thrift.meta_data.FieldMetaData("tabletServer", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -17840,7 +17655,7 @@
}
public shutdownTabletServer_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String tabletServer,
boolean force)
@@ -17859,7 +17674,7 @@
public shutdownTabletServer_args(shutdownTabletServer_args other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -17885,11 +17700,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public shutdownTabletServer_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public shutdownTabletServer_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -17989,7 +17804,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -18287,7 +18102,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -18404,7 +18219,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -18928,7 +18743,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new setSystemProperty_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new setSystemProperty_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String property; // required
public @org.apache.thrift.annotation.Nullable java.lang.String value; // required
@@ -19009,7 +18824,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PROPERTY, new org.apache.thrift.meta_data.FieldMetaData("property", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -19024,7 +18839,7 @@
}
public setSystemProperty_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String property,
java.lang.String value)
@@ -19041,7 +18856,7 @@
*/
public setSystemProperty_args(setSystemProperty_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -19068,11 +18883,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public setSystemProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public setSystemProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -19174,7 +18989,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -19476,7 +19291,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -19595,7 +19410,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -20225,7 +20040,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new modifySystemProperties_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new modifySystemProperties_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties; // required
@@ -20302,7 +20117,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.V_PROPERTIES, new org.apache.thrift.meta_data.FieldMetaData("vProperties", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -20315,7 +20130,7 @@
}
public modifySystemProperties_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties vProperties)
{
@@ -20330,7 +20145,7 @@
*/
public modifySystemProperties_args(modifySystemProperties_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -20353,11 +20168,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public modifySystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public modifySystemProperties_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -20434,7 +20249,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -20695,7 +20510,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -20796,7 +20611,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -21530,7 +21345,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new removeSystemProperty_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new removeSystemProperty_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String property; // required
@@ -21607,7 +21422,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.PROPERTY, new org.apache.thrift.meta_data.FieldMetaData("property", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -21620,7 +21435,7 @@
}
public removeSystemProperty_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String property)
{
@@ -21635,7 +21450,7 @@
*/
public removeSystemProperty_args(removeSystemProperty_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -21658,11 +21473,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public removeSystemProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public removeSystemProperty_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -21739,7 +21554,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -21997,7 +21812,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -22097,7 +21912,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -22615,7 +22430,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getManagerStats_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getManagerStats_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -22688,7 +22503,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -22699,7 +22514,7 @@
}
public getManagerStats_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -22712,7 +22527,7 @@
*/
public getManagerStats_args(getManagerStats_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -22731,11 +22546,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public getManagerStats_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getManagerStats_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -22787,7 +22602,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -23001,7 +22816,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -23082,7 +22897,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -23705,7 +23520,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new waitForBalance_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new waitForBalance_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -23774,7 +23589,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(waitForBalance_args.class, metaDataMap);
}
@@ -23783,7 +23598,7 @@
}
public waitForBalance_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo)
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo)
{
this();
this.tinfo = tinfo;
@@ -23794,7 +23609,7 @@
*/
public waitForBalance_args(waitForBalance_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
}
@@ -23809,11 +23624,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public waitForBalance_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public waitForBalance_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -23840,7 +23655,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -24007,7 +23822,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -24068,7 +23883,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(1);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -24472,7 +24287,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new reportSplitExtent_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new reportSplitExtent_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String serverName; // required
public @org.apache.thrift.annotation.Nullable TabletSplit split; // required
@@ -24553,7 +24368,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.SERVER_NAME, new org.apache.thrift.meta_data.FieldMetaData("serverName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -24568,7 +24383,7 @@
}
public reportSplitExtent_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String serverName,
TabletSplit split)
@@ -24585,7 +24400,7 @@
*/
public reportSplitExtent_args(reportSplitExtent_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -24612,11 +24427,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public reportSplitExtent_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public reportSplitExtent_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -24718,7 +24533,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -25023,7 +24838,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -25143,7 +24958,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -25182,7 +24997,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new reportTabletStatus_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new reportTabletStatus_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable java.lang.String serverName; // required
/**
@@ -25275,7 +25090,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.SERVER_NAME, new org.apache.thrift.meta_data.FieldMetaData("serverName", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -25292,7 +25107,7 @@
}
public reportTabletStatus_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
java.lang.String serverName,
TabletLoadState status,
@@ -25311,7 +25126,7 @@
*/
public reportTabletStatus_args(reportTabletStatus_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -25342,11 +25157,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public reportTabletStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public reportTabletStatus_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -25481,7 +25296,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -25830,7 +25645,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -25969,7 +25784,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(5);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -26009,7 +25824,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveTservers_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveTservers_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -26082,7 +25897,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -26093,7 +25908,7 @@
}
public getActiveTservers_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
{
this();
@@ -26106,7 +25921,7 @@
*/
public getActiveTservers_args(getActiveTservers_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -26125,11 +25940,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public getActiveTservers_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getActiveTservers_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -26181,7 +25996,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -26395,7 +26210,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -26476,7 +26291,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -26983,13 +26798,13 @@
case 0: // SUCCESS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
- org.apache.thrift.protocol.TList _list78 = iprot.readListBegin();
- struct.success = new java.util.ArrayList<java.lang.String>(_list78.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem79;
- for (int _i80 = 0; _i80 < _list78.size; ++_i80)
+ org.apache.thrift.protocol.TList _list104 = iprot.readListBegin();
+ struct.success = new java.util.ArrayList<java.lang.String>(_list104.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _elem105;
+ for (int _i106 = 0; _i106 < _list104.size; ++_i106)
{
- _elem79 = iprot.readString();
- struct.success.add(_elem79);
+ _elem105 = iprot.readString();
+ struct.success.add(_elem105);
}
iprot.readListEnd();
}
@@ -27036,9 +26851,9 @@
oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
- for (java.lang.String _iter81 : struct.success)
+ for (java.lang.String _iter107 : struct.success)
{
- oprot.writeString(_iter81);
+ oprot.writeString(_iter107);
}
oprot.writeListEnd();
}
@@ -27086,9 +26901,9 @@
if (struct.isSetSuccess()) {
{
oprot.writeI32(struct.success.size());
- for (java.lang.String _iter82 : struct.success)
+ for (java.lang.String _iter108 : struct.success)
{
- oprot.writeString(_iter82);
+ oprot.writeString(_iter108);
}
}
}
@@ -27106,13 +26921,13 @@
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TList _list83 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
- struct.success = new java.util.ArrayList<java.lang.String>(_list83.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem84;
- for (int _i85 = 0; _i85 < _list83.size; ++_i85)
+ org.apache.thrift.protocol.TList _list109 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+ struct.success = new java.util.ArrayList<java.lang.String>(_list109.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _elem110;
+ for (int _i111 = 0; _i111 < _list109.size; ++_i111)
{
- _elem84 = iprot.readString();
- struct.success.add(_elem84);
+ _elem110 = iprot.readString();
+ struct.success.add(_elem110);
}
}
struct.setSuccessIsSet(true);
@@ -27146,7 +26961,7 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getDelegationToken_argsStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getDelegationToken_argsTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg; // required
@@ -27223,7 +27038,7 @@
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
tmpMap.put(_Fields.CFG, new org.apache.thrift.meta_data.FieldMetaData("cfg", org.apache.thrift.TFieldRequirementType.DEFAULT,
@@ -27236,7 +27051,7 @@
}
public getDelegationToken_args(
- org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig cfg)
{
@@ -27251,7 +27066,7 @@
*/
public getDelegationToken_args(getDelegationToken_args other) {
if (other.isSetTinfo()) {
- this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
}
if (other.isSetCredentials()) {
this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
@@ -27274,11 +27089,11 @@
}
@org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
return this.tinfo;
}
- public getDelegationToken_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+ public getDelegationToken_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
this.tinfo = tinfo;
return this;
}
@@ -27355,7 +27170,7 @@
if (value == null) {
unsetTinfo();
} else {
- setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
}
break;
@@ -27616,7 +27431,7 @@
switch (schemeField.id) {
case 1: // TINFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
} else {
@@ -27717,7 +27532,7 @@
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
- struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
struct.tinfo.read(iprot);
struct.setTinfoIsSet(true);
}
@@ -28336,1241 +28151,5 @@
}
}
- @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
- public static class drainReplicationTable_args implements org.apache.thrift.TBase<drainReplicationTable_args, drainReplicationTable_args._Fields>, java.io.Serializable, Cloneable, Comparable<drainReplicationTable_args> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("drainReplicationTable_args");
-
- private static final org.apache.thrift.protocol.TField TFINO_FIELD_DESC = new org.apache.thrift.protocol.TField("tfino", org.apache.thrift.protocol.TType.STRUCT, (short)1);
- private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
- private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)3);
- private static final org.apache.thrift.protocol.TField LOGS_TO_WATCH_FIELD_DESC = new org.apache.thrift.protocol.TField("logsToWatch", org.apache.thrift.protocol.TType.SET, (short)4);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new drainReplicationTable_argsStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new drainReplicationTable_argsTupleSchemeFactory();
-
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tfino; // required
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
- public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required
- public @org.apache.thrift.annotation.Nullable java.util.Set<java.lang.String> logsToWatch; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- TFINO((short)1, "tfino"),
- CREDENTIALS((short)2, "credentials"),
- TABLE_NAME((short)3, "tableName"),
- LOGS_TO_WATCH((short)4, "logsToWatch");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 1: // TFINO
- return TFINO;
- case 2: // CREDENTIALS
- return CREDENTIALS;
- case 3: // TABLE_NAME
- return TABLE_NAME;
- case 4: // LOGS_TO_WATCH
- return LOGS_TO_WATCH;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.TFINO, new org.apache.thrift.meta_data.FieldMetaData("tfino", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
- tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
- tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
- tmpMap.put(_Fields.LOGS_TO_WATCH, new org.apache.thrift.meta_data.FieldMetaData("logsToWatch", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(drainReplicationTable_args.class, metaDataMap);
- }
-
- public drainReplicationTable_args() {
- }
-
- public drainReplicationTable_args(
- org.apache.accumulo.core.trace.thrift.TInfo tfino,
- org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
- java.lang.String tableName,
- java.util.Set<java.lang.String> logsToWatch)
- {
- this();
- this.tfino = tfino;
- this.credentials = credentials;
- this.tableName = tableName;
- this.logsToWatch = logsToWatch;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public drainReplicationTable_args(drainReplicationTable_args other) {
- if (other.isSetTfino()) {
- this.tfino = new org.apache.accumulo.core.trace.thrift.TInfo(other.tfino);
- }
- if (other.isSetCredentials()) {
- this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
- }
- if (other.isSetTableName()) {
- this.tableName = other.tableName;
- }
- if (other.isSetLogsToWatch()) {
- java.util.Set<java.lang.String> __this__logsToWatch = new java.util.HashSet<java.lang.String>(other.logsToWatch);
- this.logsToWatch = __this__logsToWatch;
- }
- }
-
- @Override
- public drainReplicationTable_args deepCopy() {
- return new drainReplicationTable_args(this);
- }
-
- @Override
- public void clear() {
- this.tfino = null;
- this.credentials = null;
- this.tableName = null;
- this.logsToWatch = null;
- }
-
- @org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.trace.thrift.TInfo getTfino() {
- return this.tfino;
- }
-
- public drainReplicationTable_args setTfino(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tfino) {
- this.tfino = tfino;
- return this;
- }
-
- public void unsetTfino() {
- this.tfino = null;
- }
-
- /** Returns true if field tfino is set (has been assigned a value) and false otherwise */
- public boolean isSetTfino() {
- return this.tfino != null;
- }
-
- public void setTfinoIsSet(boolean value) {
- if (!value) {
- this.tfino = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
- return this.credentials;
- }
-
- public drainReplicationTable_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
- this.credentials = credentials;
- return this;
- }
-
- public void unsetCredentials() {
- this.credentials = null;
- }
-
- /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
- public boolean isSetCredentials() {
- return this.credentials != null;
- }
-
- public void setCredentialsIsSet(boolean value) {
- if (!value) {
- this.credentials = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.lang.String getTableName() {
- return this.tableName;
- }
-
- public drainReplicationTable_args setTableName(@org.apache.thrift.annotation.Nullable java.lang.String tableName) {
- this.tableName = tableName;
- return this;
- }
-
- public void unsetTableName() {
- this.tableName = null;
- }
-
- /** Returns true if field tableName is set (has been assigned a value) and false otherwise */
- public boolean isSetTableName() {
- return this.tableName != null;
- }
-
- public void setTableNameIsSet(boolean value) {
- if (!value) {
- this.tableName = null;
- }
- }
-
- public int getLogsToWatchSize() {
- return (this.logsToWatch == null) ? 0 : this.logsToWatch.size();
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.util.Iterator<java.lang.String> getLogsToWatchIterator() {
- return (this.logsToWatch == null) ? null : this.logsToWatch.iterator();
- }
-
- public void addToLogsToWatch(java.lang.String elem) {
- if (this.logsToWatch == null) {
- this.logsToWatch = new java.util.HashSet<java.lang.String>();
- }
- this.logsToWatch.add(elem);
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.util.Set<java.lang.String> getLogsToWatch() {
- return this.logsToWatch;
- }
-
- public drainReplicationTable_args setLogsToWatch(@org.apache.thrift.annotation.Nullable java.util.Set<java.lang.String> logsToWatch) {
- this.logsToWatch = logsToWatch;
- return this;
- }
-
- public void unsetLogsToWatch() {
- this.logsToWatch = null;
- }
-
- /** Returns true if field logsToWatch is set (has been assigned a value) and false otherwise */
- public boolean isSetLogsToWatch() {
- return this.logsToWatch != null;
- }
-
- public void setLogsToWatchIsSet(boolean value) {
- if (!value) {
- this.logsToWatch = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case TFINO:
- if (value == null) {
- unsetTfino();
- } else {
- setTfino((org.apache.accumulo.core.trace.thrift.TInfo)value);
- }
- break;
-
- case CREDENTIALS:
- if (value == null) {
- unsetCredentials();
- } else {
- setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
- }
- break;
-
- case TABLE_NAME:
- if (value == null) {
- unsetTableName();
- } else {
- setTableName((java.lang.String)value);
- }
- break;
-
- case LOGS_TO_WATCH:
- if (value == null) {
- unsetLogsToWatch();
- } else {
- setLogsToWatch((java.util.Set<java.lang.String>)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case TFINO:
- return getTfino();
-
- case CREDENTIALS:
- return getCredentials();
-
- case TABLE_NAME:
- return getTableName();
-
- case LOGS_TO_WATCH:
- return getLogsToWatch();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case TFINO:
- return isSetTfino();
- case CREDENTIALS:
- return isSetCredentials();
- case TABLE_NAME:
- return isSetTableName();
- case LOGS_TO_WATCH:
- return isSetLogsToWatch();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof drainReplicationTable_args)
- return this.equals((drainReplicationTable_args)that);
- return false;
- }
-
- public boolean equals(drainReplicationTable_args that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_tfino = true && this.isSetTfino();
- boolean that_present_tfino = true && that.isSetTfino();
- if (this_present_tfino || that_present_tfino) {
- if (!(this_present_tfino && that_present_tfino))
- return false;
- if (!this.tfino.equals(that.tfino))
- return false;
- }
-
- boolean this_present_credentials = true && this.isSetCredentials();
- boolean that_present_credentials = true && that.isSetCredentials();
- if (this_present_credentials || that_present_credentials) {
- if (!(this_present_credentials && that_present_credentials))
- return false;
- if (!this.credentials.equals(that.credentials))
- return false;
- }
-
- boolean this_present_tableName = true && this.isSetTableName();
- boolean that_present_tableName = true && that.isSetTableName();
- if (this_present_tableName || that_present_tableName) {
- if (!(this_present_tableName && that_present_tableName))
- return false;
- if (!this.tableName.equals(that.tableName))
- return false;
- }
-
- boolean this_present_logsToWatch = true && this.isSetLogsToWatch();
- boolean that_present_logsToWatch = true && that.isSetLogsToWatch();
- if (this_present_logsToWatch || that_present_logsToWatch) {
- if (!(this_present_logsToWatch && that_present_logsToWatch))
- return false;
- if (!this.logsToWatch.equals(that.logsToWatch))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetTfino()) ? 131071 : 524287);
- if (isSetTfino())
- hashCode = hashCode * 8191 + tfino.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
- if (isSetCredentials())
- hashCode = hashCode * 8191 + credentials.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetTableName()) ? 131071 : 524287);
- if (isSetTableName())
- hashCode = hashCode * 8191 + tableName.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetLogsToWatch()) ? 131071 : 524287);
- if (isSetLogsToWatch())
- hashCode = hashCode * 8191 + logsToWatch.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(drainReplicationTable_args other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetTfino(), other.isSetTfino());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetTfino()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tfino, other.tfino);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetCredentials()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetTableName(), other.isSetTableName());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetTableName()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetLogsToWatch(), other.isSetLogsToWatch());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetLogsToWatch()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.logsToWatch, other.logsToWatch);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("drainReplicationTable_args(");
- boolean first = true;
-
- sb.append("tfino:");
- if (this.tfino == null) {
- sb.append("null");
- } else {
- sb.append(this.tfino);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("credentials:");
- if (this.credentials == null) {
- sb.append("null");
- } else {
- sb.append(this.credentials);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("tableName:");
- if (this.tableName == null) {
- sb.append("null");
- } else {
- sb.append(this.tableName);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("logsToWatch:");
- if (this.logsToWatch == null) {
- sb.append("null");
- } else {
- sb.append(this.logsToWatch);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- if (tfino != null) {
- tfino.validate();
- }
- if (credentials != null) {
- credentials.validate();
- }
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class drainReplicationTable_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public drainReplicationTable_argsStandardScheme getScheme() {
- return new drainReplicationTable_argsStandardScheme();
- }
- }
-
- private static class drainReplicationTable_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<drainReplicationTable_args> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, drainReplicationTable_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 1: // TFINO
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tfino = new org.apache.accumulo.core.trace.thrift.TInfo();
- struct.tfino.read(iprot);
- struct.setTfinoIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 2: // CREDENTIALS
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
- struct.credentials.read(iprot);
- struct.setCredentialsIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 3: // TABLE_NAME
- if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
- struct.tableName = iprot.readString();
- struct.setTableNameIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 4: // LOGS_TO_WATCH
- if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
- {
- org.apache.thrift.protocol.TSet _set86 = iprot.readSetBegin();
- struct.logsToWatch = new java.util.HashSet<java.lang.String>(2*_set86.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem87;
- for (int _i88 = 0; _i88 < _set86.size; ++_i88)
- {
- _elem87 = iprot.readString();
- struct.logsToWatch.add(_elem87);
- }
- iprot.readSetEnd();
- }
- struct.setLogsToWatchIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, drainReplicationTable_args struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.tfino != null) {
- oprot.writeFieldBegin(TFINO_FIELD_DESC);
- struct.tfino.write(oprot);
- oprot.writeFieldEnd();
- }
- if (struct.credentials != null) {
- oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
- struct.credentials.write(oprot);
- oprot.writeFieldEnd();
- }
- if (struct.tableName != null) {
- oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
- oprot.writeString(struct.tableName);
- oprot.writeFieldEnd();
- }
- if (struct.logsToWatch != null) {
- oprot.writeFieldBegin(LOGS_TO_WATCH_FIELD_DESC);
- {
- oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.logsToWatch.size()));
- for (java.lang.String _iter89 : struct.logsToWatch)
- {
- oprot.writeString(_iter89);
- }
- oprot.writeSetEnd();
- }
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class drainReplicationTable_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public drainReplicationTable_argsTupleScheme getScheme() {
- return new drainReplicationTable_argsTupleScheme();
- }
- }
-
- private static class drainReplicationTable_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<drainReplicationTable_args> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, drainReplicationTable_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetTfino()) {
- optionals.set(0);
- }
- if (struct.isSetCredentials()) {
- optionals.set(1);
- }
- if (struct.isSetTableName()) {
- optionals.set(2);
- }
- if (struct.isSetLogsToWatch()) {
- optionals.set(3);
- }
- oprot.writeBitSet(optionals, 4);
- if (struct.isSetTfino()) {
- struct.tfino.write(oprot);
- }
- if (struct.isSetCredentials()) {
- struct.credentials.write(oprot);
- }
- if (struct.isSetTableName()) {
- oprot.writeString(struct.tableName);
- }
- if (struct.isSetLogsToWatch()) {
- {
- oprot.writeI32(struct.logsToWatch.size());
- for (java.lang.String _iter90 : struct.logsToWatch)
- {
- oprot.writeString(_iter90);
- }
- }
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, drainReplicationTable_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(4);
- if (incoming.get(0)) {
- struct.tfino = new org.apache.accumulo.core.trace.thrift.TInfo();
- struct.tfino.read(iprot);
- struct.setTfinoIsSet(true);
- }
- if (incoming.get(1)) {
- struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
- struct.credentials.read(iprot);
- struct.setCredentialsIsSet(true);
- }
- if (incoming.get(2)) {
- struct.tableName = iprot.readString();
- struct.setTableNameIsSet(true);
- }
- if (incoming.get(3)) {
- {
- org.apache.thrift.protocol.TSet _set91 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
- struct.logsToWatch = new java.util.HashSet<java.lang.String>(2*_set91.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem92;
- for (int _i93 = 0; _i93 < _set91.size; ++_i93)
- {
- _elem92 = iprot.readString();
- struct.logsToWatch.add(_elem92);
- }
- }
- struct.setLogsToWatchIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- }
-
- @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
- public static class drainReplicationTable_result implements org.apache.thrift.TBase<drainReplicationTable_result, drainReplicationTable_result._Fields>, java.io.Serializable, Cloneable, Comparable<drainReplicationTable_result> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("drainReplicationTable_result");
-
- private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
- private static final org.apache.thrift.protocol.TField TNASE_FIELD_DESC = new org.apache.thrift.protocol.TField("tnase", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new drainReplicationTable_resultStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new drainReplicationTable_resultTupleSchemeFactory();
-
- public boolean success; // required
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- SUCCESS((short)0, "success"),
- TNASE((short)1, "tnase");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 0: // SUCCESS
- return SUCCESS;
- case 1: // TNASE
- return TNASE;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- private static final int __SUCCESS_ISSET_ID = 0;
- private byte __isset_bitfield = 0;
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
- tmpMap.put(_Fields.TNASE, new org.apache.thrift.meta_data.FieldMetaData("tnase", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException.class)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(drainReplicationTable_result.class, metaDataMap);
- }
-
- public drainReplicationTable_result() {
- }
-
- public drainReplicationTable_result(
- boolean success,
- org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase)
- {
- this();
- this.success = success;
- setSuccessIsSet(true);
- this.tnase = tnase;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public drainReplicationTable_result(drainReplicationTable_result other) {
- __isset_bitfield = other.__isset_bitfield;
- this.success = other.success;
- if (other.isSetTnase()) {
- this.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException(other.tnase);
- }
- }
-
- @Override
- public drainReplicationTable_result deepCopy() {
- return new drainReplicationTable_result(this);
- }
-
- @Override
- public void clear() {
- setSuccessIsSet(false);
- this.success = false;
- this.tnase = null;
- }
-
- public boolean isSuccess() {
- return this.success;
- }
-
- public drainReplicationTable_result setSuccess(boolean success) {
- this.success = success;
- setSuccessIsSet(true);
- return this;
- }
-
- public void unsetSuccess() {
- __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
- }
-
- /** Returns true if field success is set (has been assigned a value) and false otherwise */
- public boolean isSetSuccess() {
- return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
- }
-
- public void setSuccessIsSet(boolean value) {
- __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
- }
-
- @org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException getTnase() {
- return this.tnase;
- }
-
- public drainReplicationTable_result setTnase(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) {
- this.tnase = tnase;
- return this;
- }
-
- public void unsetTnase() {
- this.tnase = null;
- }
-
- /** Returns true if field tnase is set (has been assigned a value) and false otherwise */
- public boolean isSetTnase() {
- return this.tnase != null;
- }
-
- public void setTnaseIsSet(boolean value) {
- if (!value) {
- this.tnase = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case SUCCESS:
- if (value == null) {
- unsetSuccess();
- } else {
- setSuccess((java.lang.Boolean)value);
- }
- break;
-
- case TNASE:
- if (value == null) {
- unsetTnase();
- } else {
- setTnase((org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case SUCCESS:
- return isSuccess();
-
- case TNASE:
- return getTnase();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case SUCCESS:
- return isSetSuccess();
- case TNASE:
- return isSetTnase();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof drainReplicationTable_result)
- return this.equals((drainReplicationTable_result)that);
- return false;
- }
-
- public boolean equals(drainReplicationTable_result that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_success = true;
- boolean that_present_success = true;
- if (this_present_success || that_present_success) {
- if (!(this_present_success && that_present_success))
- return false;
- if (this.success != that.success)
- return false;
- }
-
- boolean this_present_tnase = true && this.isSetTnase();
- boolean that_present_tnase = true && that.isSetTnase();
- if (this_present_tnase || that_present_tnase) {
- if (!(this_present_tnase && that_present_tnase))
- return false;
- if (!this.tnase.equals(that.tnase))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((success) ? 131071 : 524287);
-
- hashCode = hashCode * 8191 + ((isSetTnase()) ? 131071 : 524287);
- if (isSetTnase())
- hashCode = hashCode * 8191 + tnase.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(drainReplicationTable_result other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetSuccess()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetTnase(), other.isSetTnase());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetTnase()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tnase, other.tnase);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("drainReplicationTable_result(");
- boolean first = true;
-
- sb.append("success:");
- sb.append(this.success);
- first = false;
- if (!first) sb.append(", ");
- sb.append("tnase:");
- if (this.tnase == null) {
- sb.append("null");
- } else {
- sb.append(this.tnase);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
- __isset_bitfield = 0;
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class drainReplicationTable_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public drainReplicationTable_resultStandardScheme getScheme() {
- return new drainReplicationTable_resultStandardScheme();
- }
- }
-
- private static class drainReplicationTable_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<drainReplicationTable_result> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, drainReplicationTable_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 0: // SUCCESS
- if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
- struct.success = iprot.readBool();
- struct.setSuccessIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 1: // TNASE
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException();
- struct.tnase.read(iprot);
- struct.setTnaseIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, drainReplicationTable_result struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.isSetSuccess()) {
- oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
- oprot.writeBool(struct.success);
- oprot.writeFieldEnd();
- }
- if (struct.tnase != null) {
- oprot.writeFieldBegin(TNASE_FIELD_DESC);
- struct.tnase.write(oprot);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class drainReplicationTable_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public drainReplicationTable_resultTupleScheme getScheme() {
- return new drainReplicationTable_resultTupleScheme();
- }
- }
-
- private static class drainReplicationTable_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<drainReplicationTable_result> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, drainReplicationTable_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetSuccess()) {
- optionals.set(0);
- }
- if (struct.isSetTnase()) {
- optionals.set(1);
- }
- oprot.writeBitSet(optionals, 2);
- if (struct.isSetSuccess()) {
- oprot.writeBool(struct.success);
- }
- if (struct.isSetTnase()) {
- struct.tnase.write(oprot);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, drainReplicationTable_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(2);
- if (incoming.get(0)) {
- struct.success = iprot.readBool();
- struct.setSuccessIsSet(true);
- }
- if (incoming.get(1)) {
- struct.tnase = new org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException();
- struct.tnase.read(iprot);
- struct.setTnaseIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- }
-
private static void unusedMethod() {}
}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerMonitorInfo.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerMonitorInfo.java
index 8d2c3ce..bd7eda3 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerMonitorInfo.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerMonitorInfo.java
@@ -41,8 +41,8 @@
private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new ManagerMonitorInfoStandardSchemeFactory();
private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new ManagerMonitorInfoTupleSchemeFactory();
- public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo> tableMap; // required
- public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus> tServerInfo; // required
+ public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,TableInfo> tableMap; // required
+ public @org.apache.thrift.annotation.Nullable java.util.List<TabletServerStatus> tServerInfo; // required
public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.Byte> badTServers; // required
/**
*
@@ -57,7 +57,7 @@
public int unassignedTablets; // required
public @org.apache.thrift.annotation.Nullable java.util.Set<java.lang.String> serversShuttingDown; // required
public @org.apache.thrift.annotation.Nullable java.util.List<DeadServer> deadTabletServers; // required
- public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus> bulkImports; // required
+ public @org.apache.thrift.annotation.Nullable java.util.List<BulkImportStatus> bulkImports; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -162,10 +162,10 @@
tmpMap.put(_Fields.TABLE_MAP, new org.apache.thrift.meta_data.FieldMetaData("tableMap", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.master.thrift.TableInfo.class))));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TableInfo.class))));
tmpMap.put(_Fields.T_SERVER_INFO, new org.apache.thrift.meta_data.FieldMetaData("tServerInfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.master.thrift.TabletServerStatus.class))));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TabletServerStatus.class))));
tmpMap.put(_Fields.BAD_TSERVERS, new org.apache.thrift.meta_data.FieldMetaData("badTServers", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
@@ -184,7 +184,7 @@
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DeadServer.class))));
tmpMap.put(_Fields.BULK_IMPORTS, new org.apache.thrift.meta_data.FieldMetaData("bulkImports", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.master.thrift.BulkImportStatus.class))));
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, BulkImportStatus.class))));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ManagerMonitorInfo.class, metaDataMap);
}
@@ -193,15 +193,15 @@
}
public ManagerMonitorInfo(
- java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo> tableMap,
- java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus> tServerInfo,
+ java.util.Map<java.lang.String,TableInfo> tableMap,
+ java.util.List<TabletServerStatus> tServerInfo,
java.util.Map<java.lang.String,java.lang.Byte> badTServers,
ManagerState state,
ManagerGoalState goalState,
int unassignedTablets,
java.util.Set<java.lang.String> serversShuttingDown,
java.util.List<DeadServer> deadTabletServers,
- java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus> bulkImports)
+ java.util.List<BulkImportStatus> bulkImports)
{
this();
this.tableMap = tableMap;
@@ -222,24 +222,24 @@
public ManagerMonitorInfo(ManagerMonitorInfo other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTableMap()) {
- java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo> __this__tableMap = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo>(other.tableMap.size());
- for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.master.thrift.TableInfo> other_element : other.tableMap.entrySet()) {
+ java.util.Map<java.lang.String,TableInfo> __this__tableMap = new java.util.HashMap<java.lang.String,TableInfo>(other.tableMap.size());
+ for (java.util.Map.Entry<java.lang.String, TableInfo> other_element : other.tableMap.entrySet()) {
java.lang.String other_element_key = other_element.getKey();
- org.apache.accumulo.core.master.thrift.TableInfo other_element_value = other_element.getValue();
+ TableInfo other_element_value = other_element.getValue();
java.lang.String __this__tableMap_copy_key = other_element_key;
- org.apache.accumulo.core.master.thrift.TableInfo __this__tableMap_copy_value = new org.apache.accumulo.core.master.thrift.TableInfo(other_element_value);
+ TableInfo __this__tableMap_copy_value = new TableInfo(other_element_value);
__this__tableMap.put(__this__tableMap_copy_key, __this__tableMap_copy_value);
}
this.tableMap = __this__tableMap;
}
if (other.isSetTServerInfo()) {
- java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus> __this__tServerInfo = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.TabletServerStatus>(other.tServerInfo.size());
- for (org.apache.accumulo.core.master.thrift.TabletServerStatus other_element : other.tServerInfo) {
- __this__tServerInfo.add(new org.apache.accumulo.core.master.thrift.TabletServerStatus(other_element));
+ java.util.List<TabletServerStatus> __this__tServerInfo = new java.util.ArrayList<TabletServerStatus>(other.tServerInfo.size());
+ for (TabletServerStatus other_element : other.tServerInfo) {
+ __this__tServerInfo.add(new TabletServerStatus(other_element));
}
this.tServerInfo = __this__tServerInfo;
}
@@ -266,9 +266,9 @@
this.deadTabletServers = __this__deadTabletServers;
}
if (other.isSetBulkImports()) {
- java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus> __this__bulkImports = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.BulkImportStatus>(other.bulkImports.size());
- for (org.apache.accumulo.core.master.thrift.BulkImportStatus other_element : other.bulkImports) {
- __this__bulkImports.add(new org.apache.accumulo.core.master.thrift.BulkImportStatus(other_element));
+ java.util.List<BulkImportStatus> __this__bulkImports = new java.util.ArrayList<BulkImportStatus>(other.bulkImports.size());
+ for (BulkImportStatus other_element : other.bulkImports) {
+ __this__bulkImports.add(new BulkImportStatus(other_element));
}
this.bulkImports = __this__bulkImports;
}
@@ -297,19 +297,19 @@
return (this.tableMap == null) ? 0 : this.tableMap.size();
}
- public void putToTableMap(java.lang.String key, org.apache.accumulo.core.master.thrift.TableInfo val) {
+ public void putToTableMap(java.lang.String key, TableInfo val) {
if (this.tableMap == null) {
- this.tableMap = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo>();
+ this.tableMap = new java.util.HashMap<java.lang.String,TableInfo>();
}
this.tableMap.put(key, val);
}
@org.apache.thrift.annotation.Nullable
- public java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo> getTableMap() {
+ public java.util.Map<java.lang.String,TableInfo> getTableMap() {
return this.tableMap;
}
- public ManagerMonitorInfo setTableMap(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo> tableMap) {
+ public ManagerMonitorInfo setTableMap(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,TableInfo> tableMap) {
this.tableMap = tableMap;
return this;
}
@@ -334,23 +334,23 @@
}
@org.apache.thrift.annotation.Nullable
- public java.util.Iterator<org.apache.accumulo.core.master.thrift.TabletServerStatus> getTServerInfoIterator() {
+ public java.util.Iterator<TabletServerStatus> getTServerInfoIterator() {
return (this.tServerInfo == null) ? null : this.tServerInfo.iterator();
}
- public void addToTServerInfo(org.apache.accumulo.core.master.thrift.TabletServerStatus elem) {
+ public void addToTServerInfo(TabletServerStatus elem) {
if (this.tServerInfo == null) {
- this.tServerInfo = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.TabletServerStatus>();
+ this.tServerInfo = new java.util.ArrayList<TabletServerStatus>();
}
this.tServerInfo.add(elem);
}
@org.apache.thrift.annotation.Nullable
- public java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus> getTServerInfo() {
+ public java.util.List<TabletServerStatus> getTServerInfo() {
return this.tServerInfo;
}
- public ManagerMonitorInfo setTServerInfo(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus> tServerInfo) {
+ public ManagerMonitorInfo setTServerInfo(@org.apache.thrift.annotation.Nullable java.util.List<TabletServerStatus> tServerInfo) {
this.tServerInfo = tServerInfo;
return this;
}
@@ -582,23 +582,23 @@
}
@org.apache.thrift.annotation.Nullable
- public java.util.Iterator<org.apache.accumulo.core.master.thrift.BulkImportStatus> getBulkImportsIterator() {
+ public java.util.Iterator<BulkImportStatus> getBulkImportsIterator() {
return (this.bulkImports == null) ? null : this.bulkImports.iterator();
}
- public void addToBulkImports(org.apache.accumulo.core.master.thrift.BulkImportStatus elem) {
+ public void addToBulkImports(BulkImportStatus elem) {
if (this.bulkImports == null) {
- this.bulkImports = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.BulkImportStatus>();
+ this.bulkImports = new java.util.ArrayList<BulkImportStatus>();
}
this.bulkImports.add(elem);
}
@org.apache.thrift.annotation.Nullable
- public java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus> getBulkImports() {
+ public java.util.List<BulkImportStatus> getBulkImports() {
return this.bulkImports;
}
- public ManagerMonitorInfo setBulkImports(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus> bulkImports) {
+ public ManagerMonitorInfo setBulkImports(@org.apache.thrift.annotation.Nullable java.util.List<BulkImportStatus> bulkImports) {
this.bulkImports = bulkImports;
return this;
}
@@ -625,7 +625,7 @@
if (value == null) {
unsetTableMap();
} else {
- setTableMap((java.util.Map<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo>)value);
+ setTableMap((java.util.Map<java.lang.String,TableInfo>)value);
}
break;
@@ -633,7 +633,7 @@
if (value == null) {
unsetTServerInfo();
} else {
- setTServerInfo((java.util.List<org.apache.accumulo.core.master.thrift.TabletServerStatus>)value);
+ setTServerInfo((java.util.List<TabletServerStatus>)value);
}
break;
@@ -689,7 +689,7 @@
if (value == null) {
unsetBulkImports();
} else {
- setBulkImports((java.util.List<org.apache.accumulo.core.master.thrift.BulkImportStatus>)value);
+ setBulkImports((java.util.List<BulkImportStatus>)value);
}
break;
@@ -1138,16 +1138,16 @@
case 1: // TABLE_MAP
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map8 = iprot.readMapBegin();
- struct.tableMap = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo>(2*_map8.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key9;
- @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.TableInfo _val10;
- for (int _i11 = 0; _i11 < _map8.size; ++_i11)
+ org.apache.thrift.protocol.TMap _map34 = iprot.readMapBegin();
+ struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map34.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key35;
+ @org.apache.thrift.annotation.Nullable TableInfo _val36;
+ for (int _i37 = 0; _i37 < _map34.size; ++_i37)
{
- _key9 = iprot.readString();
- _val10 = new org.apache.accumulo.core.master.thrift.TableInfo();
- _val10.read(iprot);
- struct.tableMap.put(_key9, _val10);
+ _key35 = iprot.readString();
+ _val36 = new TableInfo();
+ _val36.read(iprot);
+ struct.tableMap.put(_key35, _val36);
}
iprot.readMapEnd();
}
@@ -1159,14 +1159,14 @@
case 2: // T_SERVER_INFO
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
- org.apache.thrift.protocol.TList _list12 = iprot.readListBegin();
- struct.tServerInfo = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.TabletServerStatus>(_list12.size);
- @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.TabletServerStatus _elem13;
- for (int _i14 = 0; _i14 < _list12.size; ++_i14)
+ org.apache.thrift.protocol.TList _list38 = iprot.readListBegin();
+ struct.tServerInfo = new java.util.ArrayList<TabletServerStatus>(_list38.size);
+ @org.apache.thrift.annotation.Nullable TabletServerStatus _elem39;
+ for (int _i40 = 0; _i40 < _list38.size; ++_i40)
{
- _elem13 = new org.apache.accumulo.core.master.thrift.TabletServerStatus();
- _elem13.read(iprot);
- struct.tServerInfo.add(_elem13);
+ _elem39 = new TabletServerStatus();
+ _elem39.read(iprot);
+ struct.tServerInfo.add(_elem39);
}
iprot.readListEnd();
}
@@ -1178,15 +1178,15 @@
case 3: // BAD_TSERVERS
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map15 = iprot.readMapBegin();
- struct.badTServers = new java.util.HashMap<java.lang.String,java.lang.Byte>(2*_map15.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key16;
- byte _val17;
- for (int _i18 = 0; _i18 < _map15.size; ++_i18)
+ org.apache.thrift.protocol.TMap _map41 = iprot.readMapBegin();
+ struct.badTServers = new java.util.HashMap<java.lang.String,java.lang.Byte>(2*_map41.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key42;
+ byte _val43;
+ for (int _i44 = 0; _i44 < _map41.size; ++_i44)
{
- _key16 = iprot.readString();
- _val17 = iprot.readByte();
- struct.badTServers.put(_key16, _val17);
+ _key42 = iprot.readString();
+ _val43 = iprot.readByte();
+ struct.badTServers.put(_key42, _val43);
}
iprot.readMapEnd();
}
@@ -1222,13 +1222,13 @@
case 7: // SERVERS_SHUTTING_DOWN
if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
{
- org.apache.thrift.protocol.TSet _set19 = iprot.readSetBegin();
- struct.serversShuttingDown = new java.util.HashSet<java.lang.String>(2*_set19.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem20;
- for (int _i21 = 0; _i21 < _set19.size; ++_i21)
+ org.apache.thrift.protocol.TSet _set45 = iprot.readSetBegin();
+ struct.serversShuttingDown = new java.util.HashSet<java.lang.String>(2*_set45.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _elem46;
+ for (int _i47 = 0; _i47 < _set45.size; ++_i47)
{
- _elem20 = iprot.readString();
- struct.serversShuttingDown.add(_elem20);
+ _elem46 = iprot.readString();
+ struct.serversShuttingDown.add(_elem46);
}
iprot.readSetEnd();
}
@@ -1240,14 +1240,14 @@
case 8: // DEAD_TABLET_SERVERS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
- org.apache.thrift.protocol.TList _list22 = iprot.readListBegin();
- struct.deadTabletServers = new java.util.ArrayList<DeadServer>(_list22.size);
- @org.apache.thrift.annotation.Nullable DeadServer _elem23;
- for (int _i24 = 0; _i24 < _list22.size; ++_i24)
+ org.apache.thrift.protocol.TList _list48 = iprot.readListBegin();
+ struct.deadTabletServers = new java.util.ArrayList<DeadServer>(_list48.size);
+ @org.apache.thrift.annotation.Nullable DeadServer _elem49;
+ for (int _i50 = 0; _i50 < _list48.size; ++_i50)
{
- _elem23 = new DeadServer();
- _elem23.read(iprot);
- struct.deadTabletServers.add(_elem23);
+ _elem49 = new DeadServer();
+ _elem49.read(iprot);
+ struct.deadTabletServers.add(_elem49);
}
iprot.readListEnd();
}
@@ -1259,14 +1259,14 @@
case 9: // BULK_IMPORTS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
- org.apache.thrift.protocol.TList _list25 = iprot.readListBegin();
- struct.bulkImports = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.BulkImportStatus>(_list25.size);
- @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.BulkImportStatus _elem26;
- for (int _i27 = 0; _i27 < _list25.size; ++_i27)
+ org.apache.thrift.protocol.TList _list51 = iprot.readListBegin();
+ struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list51.size);
+ @org.apache.thrift.annotation.Nullable BulkImportStatus _elem52;
+ for (int _i53 = 0; _i53 < _list51.size; ++_i53)
{
- _elem26 = new org.apache.accumulo.core.master.thrift.BulkImportStatus();
- _elem26.read(iprot);
- struct.bulkImports.add(_elem26);
+ _elem52 = new BulkImportStatus();
+ _elem52.read(iprot);
+ struct.bulkImports.add(_elem52);
}
iprot.readListEnd();
}
@@ -1295,10 +1295,10 @@
oprot.writeFieldBegin(TABLE_MAP_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.tableMap.size()));
- for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.master.thrift.TableInfo> _iter28 : struct.tableMap.entrySet())
+ for (java.util.Map.Entry<java.lang.String, TableInfo> _iter54 : struct.tableMap.entrySet())
{
- oprot.writeString(_iter28.getKey());
- _iter28.getValue().write(oprot);
+ oprot.writeString(_iter54.getKey());
+ _iter54.getValue().write(oprot);
}
oprot.writeMapEnd();
}
@@ -1308,9 +1308,9 @@
oprot.writeFieldBegin(T_SERVER_INFO_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tServerInfo.size()));
- for (org.apache.accumulo.core.master.thrift.TabletServerStatus _iter29 : struct.tServerInfo)
+ for (TabletServerStatus _iter55 : struct.tServerInfo)
{
- _iter29.write(oprot);
+ _iter55.write(oprot);
}
oprot.writeListEnd();
}
@@ -1320,10 +1320,10 @@
oprot.writeFieldBegin(BAD_TSERVERS_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BYTE, struct.badTServers.size()));
- for (java.util.Map.Entry<java.lang.String, java.lang.Byte> _iter30 : struct.badTServers.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.Byte> _iter56 : struct.badTServers.entrySet())
{
- oprot.writeString(_iter30.getKey());
- oprot.writeByte(_iter30.getValue());
+ oprot.writeString(_iter56.getKey());
+ oprot.writeByte(_iter56.getValue());
}
oprot.writeMapEnd();
}
@@ -1346,9 +1346,9 @@
oprot.writeFieldBegin(SERVERS_SHUTTING_DOWN_FIELD_DESC);
{
oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.serversShuttingDown.size()));
- for (java.lang.String _iter31 : struct.serversShuttingDown)
+ for (java.lang.String _iter57 : struct.serversShuttingDown)
{
- oprot.writeString(_iter31);
+ oprot.writeString(_iter57);
}
oprot.writeSetEnd();
}
@@ -1358,9 +1358,9 @@
oprot.writeFieldBegin(DEAD_TABLET_SERVERS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.deadTabletServers.size()));
- for (DeadServer _iter32 : struct.deadTabletServers)
+ for (DeadServer _iter58 : struct.deadTabletServers)
{
- _iter32.write(oprot);
+ _iter58.write(oprot);
}
oprot.writeListEnd();
}
@@ -1370,9 +1370,9 @@
oprot.writeFieldBegin(BULK_IMPORTS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.bulkImports.size()));
- for (org.apache.accumulo.core.master.thrift.BulkImportStatus _iter33 : struct.bulkImports)
+ for (BulkImportStatus _iter59 : struct.bulkImports)
{
- _iter33.write(oprot);
+ _iter59.write(oprot);
}
oprot.writeListEnd();
}
@@ -1428,29 +1428,29 @@
if (struct.isSetTableMap()) {
{
oprot.writeI32(struct.tableMap.size());
- for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.master.thrift.TableInfo> _iter34 : struct.tableMap.entrySet())
+ for (java.util.Map.Entry<java.lang.String, TableInfo> _iter60 : struct.tableMap.entrySet())
{
- oprot.writeString(_iter34.getKey());
- _iter34.getValue().write(oprot);
+ oprot.writeString(_iter60.getKey());
+ _iter60.getValue().write(oprot);
}
}
}
if (struct.isSetTServerInfo()) {
{
oprot.writeI32(struct.tServerInfo.size());
- for (org.apache.accumulo.core.master.thrift.TabletServerStatus _iter35 : struct.tServerInfo)
+ for (TabletServerStatus _iter61 : struct.tServerInfo)
{
- _iter35.write(oprot);
+ _iter61.write(oprot);
}
}
}
if (struct.isSetBadTServers()) {
{
oprot.writeI32(struct.badTServers.size());
- for (java.util.Map.Entry<java.lang.String, java.lang.Byte> _iter36 : struct.badTServers.entrySet())
+ for (java.util.Map.Entry<java.lang.String, java.lang.Byte> _iter62 : struct.badTServers.entrySet())
{
- oprot.writeString(_iter36.getKey());
- oprot.writeByte(_iter36.getValue());
+ oprot.writeString(_iter62.getKey());
+ oprot.writeByte(_iter62.getValue());
}
}
}
@@ -1466,27 +1466,27 @@
if (struct.isSetServersShuttingDown()) {
{
oprot.writeI32(struct.serversShuttingDown.size());
- for (java.lang.String _iter37 : struct.serversShuttingDown)
+ for (java.lang.String _iter63 : struct.serversShuttingDown)
{
- oprot.writeString(_iter37);
+ oprot.writeString(_iter63);
}
}
}
if (struct.isSetDeadTabletServers()) {
{
oprot.writeI32(struct.deadTabletServers.size());
- for (DeadServer _iter38 : struct.deadTabletServers)
+ for (DeadServer _iter64 : struct.deadTabletServers)
{
- _iter38.write(oprot);
+ _iter64.write(oprot);
}
}
}
if (struct.isSetBulkImports()) {
{
oprot.writeI32(struct.bulkImports.size());
- for (org.apache.accumulo.core.master.thrift.BulkImportStatus _iter39 : struct.bulkImports)
+ for (BulkImportStatus _iter65 : struct.bulkImports)
{
- _iter39.write(oprot);
+ _iter65.write(oprot);
}
}
}
@@ -1498,45 +1498,45 @@
java.util.BitSet incoming = iprot.readBitSet(9);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TMap _map40 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT);
- struct.tableMap = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.master.thrift.TableInfo>(2*_map40.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key41;
- @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.TableInfo _val42;
- for (int _i43 = 0; _i43 < _map40.size; ++_i43)
+ org.apache.thrift.protocol.TMap _map66 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT);
+ struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map66.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key67;
+ @org.apache.thrift.annotation.Nullable TableInfo _val68;
+ for (int _i69 = 0; _i69 < _map66.size; ++_i69)
{
- _key41 = iprot.readString();
- _val42 = new org.apache.accumulo.core.master.thrift.TableInfo();
- _val42.read(iprot);
- struct.tableMap.put(_key41, _val42);
+ _key67 = iprot.readString();
+ _val68 = new TableInfo();
+ _val68.read(iprot);
+ struct.tableMap.put(_key67, _val68);
}
}
struct.setTableMapIsSet(true);
}
if (incoming.get(1)) {
{
- org.apache.thrift.protocol.TList _list44 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
- struct.tServerInfo = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.TabletServerStatus>(_list44.size);
- @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.TabletServerStatus _elem45;
- for (int _i46 = 0; _i46 < _list44.size; ++_i46)
+ org.apache.thrift.protocol.TList _list70 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+ struct.tServerInfo = new java.util.ArrayList<TabletServerStatus>(_list70.size);
+ @org.apache.thrift.annotation.Nullable TabletServerStatus _elem71;
+ for (int _i72 = 0; _i72 < _list70.size; ++_i72)
{
- _elem45 = new org.apache.accumulo.core.master.thrift.TabletServerStatus();
- _elem45.read(iprot);
- struct.tServerInfo.add(_elem45);
+ _elem71 = new TabletServerStatus();
+ _elem71.read(iprot);
+ struct.tServerInfo.add(_elem71);
}
}
struct.setTServerInfoIsSet(true);
}
if (incoming.get(2)) {
{
- org.apache.thrift.protocol.TMap _map47 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BYTE);
- struct.badTServers = new java.util.HashMap<java.lang.String,java.lang.Byte>(2*_map47.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key48;
- byte _val49;
- for (int _i50 = 0; _i50 < _map47.size; ++_i50)
+ org.apache.thrift.protocol.TMap _map73 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BYTE);
+ struct.badTServers = new java.util.HashMap<java.lang.String,java.lang.Byte>(2*_map73.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key74;
+ byte _val75;
+ for (int _i76 = 0; _i76 < _map73.size; ++_i76)
{
- _key48 = iprot.readString();
- _val49 = iprot.readByte();
- struct.badTServers.put(_key48, _val49);
+ _key74 = iprot.readString();
+ _val75 = iprot.readByte();
+ struct.badTServers.put(_key74, _val75);
}
}
struct.setBadTServersIsSet(true);
@@ -1555,41 +1555,41 @@
}
if (incoming.get(6)) {
{
- org.apache.thrift.protocol.TSet _set51 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
- struct.serversShuttingDown = new java.util.HashSet<java.lang.String>(2*_set51.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _elem52;
- for (int _i53 = 0; _i53 < _set51.size; ++_i53)
+ org.apache.thrift.protocol.TSet _set77 = iprot.readSetBegin(org.apache.thrift.protocol.TType.STRING);
+ struct.serversShuttingDown = new java.util.HashSet<java.lang.String>(2*_set77.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _elem78;
+ for (int _i79 = 0; _i79 < _set77.size; ++_i79)
{
- _elem52 = iprot.readString();
- struct.serversShuttingDown.add(_elem52);
+ _elem78 = iprot.readString();
+ struct.serversShuttingDown.add(_elem78);
}
}
struct.setServersShuttingDownIsSet(true);
}
if (incoming.get(7)) {
{
- org.apache.thrift.protocol.TList _list54 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
- struct.deadTabletServers = new java.util.ArrayList<DeadServer>(_list54.size);
- @org.apache.thrift.annotation.Nullable DeadServer _elem55;
- for (int _i56 = 0; _i56 < _list54.size; ++_i56)
+ org.apache.thrift.protocol.TList _list80 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+ struct.deadTabletServers = new java.util.ArrayList<DeadServer>(_list80.size);
+ @org.apache.thrift.annotation.Nullable DeadServer _elem81;
+ for (int _i82 = 0; _i82 < _list80.size; ++_i82)
{
- _elem55 = new DeadServer();
- _elem55.read(iprot);
- struct.deadTabletServers.add(_elem55);
+ _elem81 = new DeadServer();
+ _elem81.read(iprot);
+ struct.deadTabletServers.add(_elem81);
}
}
struct.setDeadTabletServersIsSet(true);
}
if (incoming.get(8)) {
{
- org.apache.thrift.protocol.TList _list57 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
- struct.bulkImports = new java.util.ArrayList<org.apache.accumulo.core.master.thrift.BulkImportStatus>(_list57.size);
- @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.BulkImportStatus _elem58;
- for (int _i59 = 0; _i59 < _list57.size; ++_i59)
+ org.apache.thrift.protocol.TList _list83 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+ struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list83.size);
+ @org.apache.thrift.annotation.Nullable BulkImportStatus _elem84;
+ for (int _i85 = 0; _i85 < _list83.size; ++_i85)
{
- _elem58 = new org.apache.accumulo.core.master.thrift.BulkImportStatus();
- _elem58.read(iprot);
- struct.bulkImports.add(_elem58);
+ _elem84 = new BulkImportStatus();
+ _elem84.read(iprot);
+ struct.bulkImports.add(_elem84);
}
}
struct.setBulkImportsIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryException.java
deleted file mode 100644
index 32eb228..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryException.java
+++ /dev/null
@@ -1,406 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- * @generated
- */
-package org.apache.accumulo.core.manager.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class RecoveryException extends org.apache.thrift.TException implements org.apache.thrift.TBase<RecoveryException, RecoveryException._Fields>, java.io.Serializable, Cloneable, Comparable<RecoveryException> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RecoveryException");
-
- private static final org.apache.thrift.protocol.TField WHY_FIELD_DESC = new org.apache.thrift.protocol.TField("why", org.apache.thrift.protocol.TType.STRING, (short)1);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new RecoveryExceptionStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new RecoveryExceptionTupleSchemeFactory();
-
- public @org.apache.thrift.annotation.Nullable java.lang.String why; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- WHY((short)1, "why");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 1: // WHY
- return WHY;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.WHY, new org.apache.thrift.meta_data.FieldMetaData("why", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RecoveryException.class, metaDataMap);
- }
-
- public RecoveryException() {
- }
-
- public RecoveryException(
- java.lang.String why)
- {
- this();
- this.why = why;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public RecoveryException(RecoveryException other) {
- if (other.isSetWhy()) {
- this.why = other.why;
- }
- }
-
- @Override
- public RecoveryException deepCopy() {
- return new RecoveryException(this);
- }
-
- @Override
- public void clear() {
- this.why = null;
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.lang.String getWhy() {
- return this.why;
- }
-
- public RecoveryException setWhy(@org.apache.thrift.annotation.Nullable java.lang.String why) {
- this.why = why;
- return this;
- }
-
- public void unsetWhy() {
- this.why = null;
- }
-
- /** Returns true if field why is set (has been assigned a value) and false otherwise */
- public boolean isSetWhy() {
- return this.why != null;
- }
-
- public void setWhyIsSet(boolean value) {
- if (!value) {
- this.why = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case WHY:
- if (value == null) {
- unsetWhy();
- } else {
- setWhy((java.lang.String)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case WHY:
- return getWhy();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case WHY:
- return isSetWhy();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof RecoveryException)
- return this.equals((RecoveryException)that);
- return false;
- }
-
- public boolean equals(RecoveryException that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_why = true && this.isSetWhy();
- boolean that_present_why = true && that.isSetWhy();
- if (this_present_why || that_present_why) {
- if (!(this_present_why && that_present_why))
- return false;
- if (!this.why.equals(that.why))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetWhy()) ? 131071 : 524287);
- if (isSetWhy())
- hashCode = hashCode * 8191 + why.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(RecoveryException other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetWhy(), other.isSetWhy());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetWhy()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.why, other.why);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("RecoveryException(");
- boolean first = true;
-
- sb.append("why:");
- if (this.why == null) {
- sb.append("null");
- } else {
- sb.append(this.why);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class RecoveryExceptionStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public RecoveryExceptionStandardScheme getScheme() {
- return new RecoveryExceptionStandardScheme();
- }
- }
-
- private static class RecoveryExceptionStandardScheme extends org.apache.thrift.scheme.StandardScheme<RecoveryException> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, RecoveryException struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 1: // WHY
- if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
- struct.why = iprot.readString();
- struct.setWhyIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, RecoveryException struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.why != null) {
- oprot.writeFieldBegin(WHY_FIELD_DESC);
- oprot.writeString(struct.why);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class RecoveryExceptionTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public RecoveryExceptionTupleScheme getScheme() {
- return new RecoveryExceptionTupleScheme();
- }
- }
-
- private static class RecoveryExceptionTupleScheme extends org.apache.thrift.scheme.TupleScheme<RecoveryException> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, RecoveryException struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetWhy()) {
- optionals.set(0);
- }
- oprot.writeBitSet(optionals, 1);
- if (struct.isSetWhy()) {
- oprot.writeString(struct.why);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, RecoveryException struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(1);
- if (incoming.get(0)) {
- struct.why = iprot.readString();
- struct.setWhyIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- private static void unusedMethod() {}
-}
-
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/RecoveryStatus.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryStatus.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/RecoveryStatus.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryStatus.java
index 9372d5f..021911c 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/RecoveryStatus.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/RecoveryStatus.java
@@ -22,7 +22,7 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
public class RecoveryStatus implements org.apache.thrift.TBase<RecoveryStatus, RecoveryStatus._Fields>, java.io.Serializable, Cloneable, Comparable<RecoveryStatus> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TableInfo.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TableInfo.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TableInfo.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TableInfo.java
index c9fb233..846adee 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TableInfo.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TableInfo.java
@@ -22,7 +22,7 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
public class TableInfo implements org.apache.thrift.TBase<TableInfo, TableInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TableInfo> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletLoadState.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletLoadState.java
index aa83733..8f06a49 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletLoadState.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletLoadState.java
@@ -30,8 +30,7 @@
LOAD_FAILURE(1),
UNLOADED(2),
UNLOAD_FAILURE_NOT_SERVING(3),
- UNLOAD_ERROR(4),
- CHOPPED(5);
+ UNLOAD_ERROR(4);
private final int value;
@@ -64,8 +63,6 @@
return UNLOAD_FAILURE_NOT_SERVING;
case 4:
return UNLOAD_ERROR;
- case 5:
- return CHOPPED;
default:
return null;
}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TabletServerStatus.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletServerStatus.java
similarity index 96%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TabletServerStatus.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletServerStatus.java
index c6261fc..36bf831 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/master/thrift/TabletServerStatus.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TabletServerStatus.java
@@ -22,7 +22,7 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.master.thrift;
+package org.apache.accumulo.core.manager.thrift;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
public class TabletServerStatus implements org.apache.thrift.TBase<TabletServerStatus, TabletServerStatus._Fields>, java.io.Serializable, Cloneable, Comparable<TabletServerStatus> {
@@ -1557,16 +1557,16 @@
case 1: // TABLE_MAP
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
- org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin();
- struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map0.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key1;
- @org.apache.thrift.annotation.Nullable TableInfo _val2;
- for (int _i3 = 0; _i3 < _map0.size; ++_i3)
+ org.apache.thrift.protocol.TMap _map8 = iprot.readMapBegin();
+ struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map8.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key9;
+ @org.apache.thrift.annotation.Nullable TableInfo _val10;
+ for (int _i11 = 0; _i11 < _map8.size; ++_i11)
{
- _key1 = iprot.readString();
- _val2 = new TableInfo();
- _val2.read(iprot);
- struct.tableMap.put(_key1, _val2);
+ _key9 = iprot.readString();
+ _val10 = new TableInfo();
+ _val10.read(iprot);
+ struct.tableMap.put(_key9, _val10);
}
iprot.readMapEnd();
}
@@ -1650,14 +1650,14 @@
case 14: // LOG_SORTS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
- org.apache.thrift.protocol.TList _list4 = iprot.readListBegin();
- struct.logSorts = new java.util.ArrayList<RecoveryStatus>(_list4.size);
- @org.apache.thrift.annotation.Nullable RecoveryStatus _elem5;
- for (int _i6 = 0; _i6 < _list4.size; ++_i6)
+ org.apache.thrift.protocol.TList _list12 = iprot.readListBegin();
+ struct.logSorts = new java.util.ArrayList<RecoveryStatus>(_list12.size);
+ @org.apache.thrift.annotation.Nullable RecoveryStatus _elem13;
+ for (int _i14 = 0; _i14 < _list12.size; ++_i14)
{
- _elem5 = new RecoveryStatus();
- _elem5.read(iprot);
- struct.logSorts.add(_elem5);
+ _elem13 = new RecoveryStatus();
+ _elem13.read(iprot);
+ struct.logSorts.add(_elem13);
}
iprot.readListEnd();
}
@@ -1685,14 +1685,14 @@
case 17: // BULK_IMPORTS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
- org.apache.thrift.protocol.TList _list7 = iprot.readListBegin();
- struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list7.size);
- @org.apache.thrift.annotation.Nullable BulkImportStatus _elem8;
- for (int _i9 = 0; _i9 < _list7.size; ++_i9)
+ org.apache.thrift.protocol.TList _list15 = iprot.readListBegin();
+ struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list15.size);
+ @org.apache.thrift.annotation.Nullable BulkImportStatus _elem16;
+ for (int _i17 = 0; _i17 < _list15.size; ++_i17)
{
- _elem8 = new BulkImportStatus();
- _elem8.read(iprot);
- struct.bulkImports.add(_elem8);
+ _elem16 = new BulkImportStatus();
+ _elem16.read(iprot);
+ struct.bulkImports.add(_elem16);
}
iprot.readListEnd();
}
@@ -1737,10 +1737,10 @@
oprot.writeFieldBegin(TABLE_MAP_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.tableMap.size()));
- for (java.util.Map.Entry<java.lang.String, TableInfo> _iter10 : struct.tableMap.entrySet())
+ for (java.util.Map.Entry<java.lang.String, TableInfo> _iter18 : struct.tableMap.entrySet())
{
- oprot.writeString(_iter10.getKey());
- _iter10.getValue().write(oprot);
+ oprot.writeString(_iter18.getKey());
+ _iter18.getValue().write(oprot);
}
oprot.writeMapEnd();
}
@@ -1779,9 +1779,9 @@
oprot.writeFieldBegin(LOG_SORTS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.logSorts.size()));
- for (RecoveryStatus _iter11 : struct.logSorts)
+ for (RecoveryStatus _iter19 : struct.logSorts)
{
- _iter11.write(oprot);
+ _iter19.write(oprot);
}
oprot.writeListEnd();
}
@@ -1797,9 +1797,9 @@
oprot.writeFieldBegin(BULK_IMPORTS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.bulkImports.size()));
- for (BulkImportStatus _iter12 : struct.bulkImports)
+ for (BulkImportStatus _iter20 : struct.bulkImports)
{
- _iter12.write(oprot);
+ _iter20.write(oprot);
}
oprot.writeListEnd();
}
@@ -1884,10 +1884,10 @@
if (struct.isSetTableMap()) {
{
oprot.writeI32(struct.tableMap.size());
- for (java.util.Map.Entry<java.lang.String, TableInfo> _iter13 : struct.tableMap.entrySet())
+ for (java.util.Map.Entry<java.lang.String, TableInfo> _iter21 : struct.tableMap.entrySet())
{
- oprot.writeString(_iter13.getKey());
- _iter13.getValue().write(oprot);
+ oprot.writeString(_iter21.getKey());
+ _iter21.getValue().write(oprot);
}
}
}
@@ -1921,9 +1921,9 @@
if (struct.isSetLogSorts()) {
{
oprot.writeI32(struct.logSorts.size());
- for (RecoveryStatus _iter14 : struct.logSorts)
+ for (RecoveryStatus _iter22 : struct.logSorts)
{
- _iter14.write(oprot);
+ _iter22.write(oprot);
}
}
}
@@ -1936,9 +1936,9 @@
if (struct.isSetBulkImports()) {
{
oprot.writeI32(struct.bulkImports.size());
- for (BulkImportStatus _iter15 : struct.bulkImports)
+ for (BulkImportStatus _iter23 : struct.bulkImports)
{
- _iter15.write(oprot);
+ _iter23.write(oprot);
}
}
}
@@ -1956,16 +1956,16 @@
java.util.BitSet incoming = iprot.readBitSet(16);
if (incoming.get(0)) {
{
- org.apache.thrift.protocol.TMap _map16 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT);
- struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map16.size);
- @org.apache.thrift.annotation.Nullable java.lang.String _key17;
- @org.apache.thrift.annotation.Nullable TableInfo _val18;
- for (int _i19 = 0; _i19 < _map16.size; ++_i19)
+ org.apache.thrift.protocol.TMap _map24 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT);
+ struct.tableMap = new java.util.HashMap<java.lang.String,TableInfo>(2*_map24.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _key25;
+ @org.apache.thrift.annotation.Nullable TableInfo _val26;
+ for (int _i27 = 0; _i27 < _map24.size; ++_i27)
{
- _key17 = iprot.readString();
- _val18 = new TableInfo();
- _val18.read(iprot);
- struct.tableMap.put(_key17, _val18);
+ _key25 = iprot.readString();
+ _val26 = new TableInfo();
+ _val26.read(iprot);
+ struct.tableMap.put(_key25, _val26);
}
}
struct.setTableMapIsSet(true);
@@ -2008,14 +2008,14 @@
}
if (incoming.get(10)) {
{
- org.apache.thrift.protocol.TList _list20 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
- struct.logSorts = new java.util.ArrayList<RecoveryStatus>(_list20.size);
- @org.apache.thrift.annotation.Nullable RecoveryStatus _elem21;
- for (int _i22 = 0; _i22 < _list20.size; ++_i22)
+ org.apache.thrift.protocol.TList _list28 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+ struct.logSorts = new java.util.ArrayList<RecoveryStatus>(_list28.size);
+ @org.apache.thrift.annotation.Nullable RecoveryStatus _elem29;
+ for (int _i30 = 0; _i30 < _list28.size; ++_i30)
{
- _elem21 = new RecoveryStatus();
- _elem21.read(iprot);
- struct.logSorts.add(_elem21);
+ _elem29 = new RecoveryStatus();
+ _elem29.read(iprot);
+ struct.logSorts.add(_elem29);
}
}
struct.setLogSortsIsSet(true);
@@ -2030,14 +2030,14 @@
}
if (incoming.get(13)) {
{
- org.apache.thrift.protocol.TList _list23 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
- struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list23.size);
- @org.apache.thrift.annotation.Nullable BulkImportStatus _elem24;
- for (int _i25 = 0; _i25 < _list23.size; ++_i25)
+ org.apache.thrift.protocol.TList _list31 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+ struct.bulkImports = new java.util.ArrayList<BulkImportStatus>(_list31.size);
+ @org.apache.thrift.annotation.Nullable BulkImportStatus _elem32;
+ for (int _i33 = 0; _i33 < _list31.size; ++_i33)
{
- _elem24 = new BulkImportStatus();
- _elem24.read(iprot);
- struct.bulkImports.add(_elem24);
+ _elem32 = new BulkImportStatus();
+ _elem32.read(iprot);
+ struct.bulkImports.add(_elem32);
}
}
struct.setBulkImportsIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/KeyValues.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/KeyValues.java
deleted file mode 100644
index b700d31..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/KeyValues.java
+++ /dev/null
@@ -1,461 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- * @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class KeyValues implements org.apache.thrift.TBase<KeyValues, KeyValues._Fields>, java.io.Serializable, Cloneable, Comparable<KeyValues> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("KeyValues");
-
- private static final org.apache.thrift.protocol.TField KEY_VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("keyValues", org.apache.thrift.protocol.TType.LIST, (short)1);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new KeyValuesStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new KeyValuesTupleSchemeFactory();
-
- public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> keyValues; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- KEY_VALUES((short)1, "keyValues");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 1: // KEY_VALUES
- return KEY_VALUES;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.KEY_VALUES, new org.apache.thrift.meta_data.FieldMetaData("keyValues", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyValue.class))));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(KeyValues.class, metaDataMap);
- }
-
- public KeyValues() {
- }
-
- public KeyValues(
- java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> keyValues)
- {
- this();
- this.keyValues = keyValues;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public KeyValues(KeyValues other) {
- if (other.isSetKeyValues()) {
- java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> __this__keyValues = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyValue>(other.keyValues.size());
- for (org.apache.accumulo.core.dataImpl.thrift.TKeyValue other_element : other.keyValues) {
- __this__keyValues.add(new org.apache.accumulo.core.dataImpl.thrift.TKeyValue(other_element));
- }
- this.keyValues = __this__keyValues;
- }
- }
-
- @Override
- public KeyValues deepCopy() {
- return new KeyValues(this);
- }
-
- @Override
- public void clear() {
- this.keyValues = null;
- }
-
- public int getKeyValuesSize() {
- return (this.keyValues == null) ? 0 : this.keyValues.size();
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.util.Iterator<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> getKeyValuesIterator() {
- return (this.keyValues == null) ? null : this.keyValues.iterator();
- }
-
- public void addToKeyValues(org.apache.accumulo.core.dataImpl.thrift.TKeyValue elem) {
- if (this.keyValues == null) {
- this.keyValues = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyValue>();
- }
- this.keyValues.add(elem);
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> getKeyValues() {
- return this.keyValues;
- }
-
- public KeyValues setKeyValues(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue> keyValues) {
- this.keyValues = keyValues;
- return this;
- }
-
- public void unsetKeyValues() {
- this.keyValues = null;
- }
-
- /** Returns true if field keyValues is set (has been assigned a value) and false otherwise */
- public boolean isSetKeyValues() {
- return this.keyValues != null;
- }
-
- public void setKeyValuesIsSet(boolean value) {
- if (!value) {
- this.keyValues = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case KEY_VALUES:
- if (value == null) {
- unsetKeyValues();
- } else {
- setKeyValues((java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyValue>)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case KEY_VALUES:
- return getKeyValues();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case KEY_VALUES:
- return isSetKeyValues();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof KeyValues)
- return this.equals((KeyValues)that);
- return false;
- }
-
- public boolean equals(KeyValues that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_keyValues = true && this.isSetKeyValues();
- boolean that_present_keyValues = true && that.isSetKeyValues();
- if (this_present_keyValues || that_present_keyValues) {
- if (!(this_present_keyValues && that_present_keyValues))
- return false;
- if (!this.keyValues.equals(that.keyValues))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetKeyValues()) ? 131071 : 524287);
- if (isSetKeyValues())
- hashCode = hashCode * 8191 + keyValues.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(KeyValues other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetKeyValues(), other.isSetKeyValues());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetKeyValues()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keyValues, other.keyValues);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("KeyValues(");
- boolean first = true;
-
- sb.append("keyValues:");
- if (this.keyValues == null) {
- sb.append("null");
- } else {
- sb.append(this.keyValues);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class KeyValuesStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public KeyValuesStandardScheme getScheme() {
- return new KeyValuesStandardScheme();
- }
- }
-
- private static class KeyValuesStandardScheme extends org.apache.thrift.scheme.StandardScheme<KeyValues> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, KeyValues struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 1: // KEY_VALUES
- if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
- {
- org.apache.thrift.protocol.TList _list8 = iprot.readListBegin();
- struct.keyValues = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyValue>(_list8.size);
- @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyValue _elem9;
- for (int _i10 = 0; _i10 < _list8.size; ++_i10)
- {
- _elem9 = new org.apache.accumulo.core.dataImpl.thrift.TKeyValue();
- _elem9.read(iprot);
- struct.keyValues.add(_elem9);
- }
- iprot.readListEnd();
- }
- struct.setKeyValuesIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, KeyValues struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.keyValues != null) {
- oprot.writeFieldBegin(KEY_VALUES_FIELD_DESC);
- {
- oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.keyValues.size()));
- for (org.apache.accumulo.core.dataImpl.thrift.TKeyValue _iter11 : struct.keyValues)
- {
- _iter11.write(oprot);
- }
- oprot.writeListEnd();
- }
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class KeyValuesTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public KeyValuesTupleScheme getScheme() {
- return new KeyValuesTupleScheme();
- }
- }
-
- private static class KeyValuesTupleScheme extends org.apache.thrift.scheme.TupleScheme<KeyValues> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, KeyValues struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetKeyValues()) {
- optionals.set(0);
- }
- oprot.writeBitSet(optionals, 1);
- if (struct.isSetKeyValues()) {
- {
- oprot.writeI32(struct.keyValues.size());
- for (org.apache.accumulo.core.dataImpl.thrift.TKeyValue _iter12 : struct.keyValues)
- {
- _iter12.write(oprot);
- }
- }
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, KeyValues struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(1);
- if (incoming.get(0)) {
- {
- org.apache.thrift.protocol.TList _list13 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
- struct.keyValues = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyValue>(_list13.size);
- @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyValue _elem14;
- for (int _i15 = 0; _i15 < _list13.size; ++_i15)
- {
- _elem14 = new org.apache.accumulo.core.dataImpl.thrift.TKeyValue();
- _elem14.read(iprot);
- struct.keyValues.add(_elem14);
- }
- }
- struct.setKeyValuesIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- private static void unusedMethod() {}
-}
-
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationErrorCode.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationErrorCode.java
deleted file mode 100644
index 9d9d810..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationErrorCode.java
+++ /dev/null
@@ -1,70 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- * @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-
-public enum RemoteReplicationErrorCode implements org.apache.thrift.TEnum {
- COULD_NOT_DESERIALIZE(0),
- COULD_NOT_APPLY(1),
- TABLE_DOES_NOT_EXIST(2),
- CANNOT_AUTHENTICATE(3),
- CANNOT_INSTANTIATE_REPLAYER(4);
-
- private final int value;
-
- private RemoteReplicationErrorCode(int value) {
- this.value = value;
- }
-
- /**
- * Get the integer value of this enum value, as defined in the Thrift IDL.
- */
- @Override
- public int getValue() {
- return value;
- }
-
- /**
- * Find a the enum type by its integer value, as defined in the Thrift IDL.
- * @return null if the value is not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static RemoteReplicationErrorCode findByValue(int value) {
- switch (value) {
- case 0:
- return COULD_NOT_DESERIALIZE;
- case 1:
- return COULD_NOT_APPLY;
- case 2:
- return TABLE_DOES_NOT_EXIST;
- case 3:
- return CANNOT_AUTHENTICATE;
- case 4:
- return CANNOT_INSTANTIATE_REPLAYER;
- default:
- return null;
- }
- }
-}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationException.java
deleted file mode 100644
index 8c250e0..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/RemoteReplicationException.java
+++ /dev/null
@@ -1,527 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- * @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class RemoteReplicationException extends org.apache.thrift.TException implements org.apache.thrift.TBase<RemoteReplicationException, RemoteReplicationException._Fields>, java.io.Serializable, Cloneable, Comparable<RemoteReplicationException> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RemoteReplicationException");
-
- private static final org.apache.thrift.protocol.TField CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("code", org.apache.thrift.protocol.TType.I32, (short)1);
- private static final org.apache.thrift.protocol.TField REASON_FIELD_DESC = new org.apache.thrift.protocol.TField("reason", org.apache.thrift.protocol.TType.STRING, (short)2);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new RemoteReplicationExceptionStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new RemoteReplicationExceptionTupleSchemeFactory();
-
- /**
- *
- * @see RemoteReplicationErrorCode
- */
- public @org.apache.thrift.annotation.Nullable RemoteReplicationErrorCode code; // required
- public @org.apache.thrift.annotation.Nullable java.lang.String reason; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- /**
- *
- * @see RemoteReplicationErrorCode
- */
- CODE((short)1, "code"),
- REASON((short)2, "reason");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 1: // CODE
- return CODE;
- case 2: // REASON
- return REASON;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.CODE, new org.apache.thrift.meta_data.FieldMetaData("code", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, RemoteReplicationErrorCode.class)));
- tmpMap.put(_Fields.REASON, new org.apache.thrift.meta_data.FieldMetaData("reason", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RemoteReplicationException.class, metaDataMap);
- }
-
- public RemoteReplicationException() {
- }
-
- public RemoteReplicationException(
- RemoteReplicationErrorCode code,
- java.lang.String reason)
- {
- this();
- this.code = code;
- this.reason = reason;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public RemoteReplicationException(RemoteReplicationException other) {
- if (other.isSetCode()) {
- this.code = other.code;
- }
- if (other.isSetReason()) {
- this.reason = other.reason;
- }
- }
-
- @Override
- public RemoteReplicationException deepCopy() {
- return new RemoteReplicationException(this);
- }
-
- @Override
- public void clear() {
- this.code = null;
- this.reason = null;
- }
-
- /**
- *
- * @see RemoteReplicationErrorCode
- */
- @org.apache.thrift.annotation.Nullable
- public RemoteReplicationErrorCode getCode() {
- return this.code;
- }
-
- /**
- *
- * @see RemoteReplicationErrorCode
- */
- public RemoteReplicationException setCode(@org.apache.thrift.annotation.Nullable RemoteReplicationErrorCode code) {
- this.code = code;
- return this;
- }
-
- public void unsetCode() {
- this.code = null;
- }
-
- /** Returns true if field code is set (has been assigned a value) and false otherwise */
- public boolean isSetCode() {
- return this.code != null;
- }
-
- public void setCodeIsSet(boolean value) {
- if (!value) {
- this.code = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.lang.String getReason() {
- return this.reason;
- }
-
- public RemoteReplicationException setReason(@org.apache.thrift.annotation.Nullable java.lang.String reason) {
- this.reason = reason;
- return this;
- }
-
- public void unsetReason() {
- this.reason = null;
- }
-
- /** Returns true if field reason is set (has been assigned a value) and false otherwise */
- public boolean isSetReason() {
- return this.reason != null;
- }
-
- public void setReasonIsSet(boolean value) {
- if (!value) {
- this.reason = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case CODE:
- if (value == null) {
- unsetCode();
- } else {
- setCode((RemoteReplicationErrorCode)value);
- }
- break;
-
- case REASON:
- if (value == null) {
- unsetReason();
- } else {
- setReason((java.lang.String)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case CODE:
- return getCode();
-
- case REASON:
- return getReason();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case CODE:
- return isSetCode();
- case REASON:
- return isSetReason();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof RemoteReplicationException)
- return this.equals((RemoteReplicationException)that);
- return false;
- }
-
- public boolean equals(RemoteReplicationException that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_code = true && this.isSetCode();
- boolean that_present_code = true && that.isSetCode();
- if (this_present_code || that_present_code) {
- if (!(this_present_code && that_present_code))
- return false;
- if (!this.code.equals(that.code))
- return false;
- }
-
- boolean this_present_reason = true && this.isSetReason();
- boolean that_present_reason = true && that.isSetReason();
- if (this_present_reason || that_present_reason) {
- if (!(this_present_reason && that_present_reason))
- return false;
- if (!this.reason.equals(that.reason))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetCode()) ? 131071 : 524287);
- if (isSetCode())
- hashCode = hashCode * 8191 + code.getValue();
-
- hashCode = hashCode * 8191 + ((isSetReason()) ? 131071 : 524287);
- if (isSetReason())
- hashCode = hashCode * 8191 + reason.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(RemoteReplicationException other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetCode(), other.isSetCode());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetCode()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.code, other.code);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetReason(), other.isSetReason());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetReason()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reason, other.reason);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("RemoteReplicationException(");
- boolean first = true;
-
- sb.append("code:");
- if (this.code == null) {
- sb.append("null");
- } else {
- sb.append(this.code);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("reason:");
- if (this.reason == null) {
- sb.append("null");
- } else {
- sb.append(this.reason);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class RemoteReplicationExceptionStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public RemoteReplicationExceptionStandardScheme getScheme() {
- return new RemoteReplicationExceptionStandardScheme();
- }
- }
-
- private static class RemoteReplicationExceptionStandardScheme extends org.apache.thrift.scheme.StandardScheme<RemoteReplicationException> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, RemoteReplicationException struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 1: // CODE
- if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
- struct.code = org.apache.accumulo.core.replication.thrift.RemoteReplicationErrorCode.findByValue(iprot.readI32());
- struct.setCodeIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 2: // REASON
- if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
- struct.reason = iprot.readString();
- struct.setReasonIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, RemoteReplicationException struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.code != null) {
- oprot.writeFieldBegin(CODE_FIELD_DESC);
- oprot.writeI32(struct.code.getValue());
- oprot.writeFieldEnd();
- }
- if (struct.reason != null) {
- oprot.writeFieldBegin(REASON_FIELD_DESC);
- oprot.writeString(struct.reason);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class RemoteReplicationExceptionTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public RemoteReplicationExceptionTupleScheme getScheme() {
- return new RemoteReplicationExceptionTupleScheme();
- }
- }
-
- private static class RemoteReplicationExceptionTupleScheme extends org.apache.thrift.scheme.TupleScheme<RemoteReplicationException> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, RemoteReplicationException struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetCode()) {
- optionals.set(0);
- }
- if (struct.isSetReason()) {
- optionals.set(1);
- }
- oprot.writeBitSet(optionals, 2);
- if (struct.isSetCode()) {
- oprot.writeI32(struct.code.getValue());
- }
- if (struct.isSetReason()) {
- oprot.writeString(struct.reason);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, RemoteReplicationException struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(2);
- if (incoming.get(0)) {
- struct.code = org.apache.accumulo.core.replication.thrift.RemoteReplicationErrorCode.findByValue(iprot.readI32());
- struct.setCodeIsSet(true);
- }
- if (incoming.get(1)) {
- struct.reason = iprot.readString();
- struct.setReasonIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- private static void unusedMethod() {}
-}
-
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinator.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinator.java
deleted file mode 100644
index b4e79d6..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinator.java
+++ /dev/null
@@ -1,1263 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- * @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class ReplicationCoordinator {
-
- public interface Iface {
-
- public java.lang.String getServicerAddress(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ReplicationCoordinatorException, org.apache.thrift.TException;
-
- }
-
- public interface AsyncIface {
-
- public void getServicerAddress(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
-
- }
-
- public static class Client extends org.apache.thrift.TServiceClient implements Iface {
- public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
- public Factory() {}
- @Override
- public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
- return new Client(prot);
- }
- @Override
- public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
- return new Client(iprot, oprot);
- }
- }
-
- public Client(org.apache.thrift.protocol.TProtocol prot)
- {
- super(prot, prot);
- }
-
- public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
- super(iprot, oprot);
- }
-
- @Override
- public java.lang.String getServicerAddress(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws ReplicationCoordinatorException, org.apache.thrift.TException
- {
- send_getServicerAddress(remoteTableId, credentials);
- return recv_getServicerAddress();
- }
-
- public void send_getServicerAddress(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
- {
- getServicerAddress_args args = new getServicerAddress_args();
- args.setRemoteTableId(remoteTableId);
- args.setCredentials(credentials);
- sendBase("getServicerAddress", args);
- }
-
- public java.lang.String recv_getServicerAddress() throws ReplicationCoordinatorException, org.apache.thrift.TException
- {
- getServicerAddress_result result = new getServicerAddress_result();
- receiveBase(result, "getServicerAddress");
- if (result.isSetSuccess()) {
- return result.success;
- }
- if (result.e != null) {
- throw result.e;
- }
- throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getServicerAddress failed: unknown result");
- }
-
- }
- public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
- public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
- private org.apache.thrift.async.TAsyncClientManager clientManager;
- private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
- public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
- this.clientManager = clientManager;
- this.protocolFactory = protocolFactory;
- }
- @Override
- public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
- return new AsyncClient(protocolFactory, clientManager, transport);
- }
- }
-
- public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
- super(protocolFactory, clientManager, transport);
- }
-
- @Override
- public void getServicerAddress(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
- checkReady();
- getServicerAddress_call method_call = new getServicerAddress_call(remoteTableId, credentials, resultHandler, this, ___protocolFactory, ___transport);
- this.___currentMethod = method_call;
- ___manager.call(method_call);
- }
-
- public static class getServicerAddress_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
- private java.lang.String remoteTableId;
- private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public getServicerAddress_call(java.lang.String remoteTableId, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
- super(client, protocolFactory, transport, resultHandler, false);
- this.remoteTableId = remoteTableId;
- this.credentials = credentials;
- }
-
- @Override
- public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
- prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getServicerAddress", org.apache.thrift.protocol.TMessageType.CALL, 0));
- getServicerAddress_args args = new getServicerAddress_args();
- args.setRemoteTableId(remoteTableId);
- args.setCredentials(credentials);
- args.write(prot);
- prot.writeMessageEnd();
- }
-
- @Override
- public java.lang.String getResult() throws ReplicationCoordinatorException, org.apache.thrift.TException {
- if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
- throw new java.lang.IllegalStateException("Method call not finished!");
- }
- org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
- org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
- return (new Client(prot)).recv_getServicerAddress();
- }
- }
-
- }
-
- public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
- private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
- public Processor(I iface) {
- super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
- }
-
- protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
- super(iface, getProcessMap(processMap));
- }
-
- private static <I extends Iface> java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
- processMap.put("getServicerAddress", new getServicerAddress());
- return processMap;
- }
-
- public static class getServicerAddress<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getServicerAddress_args> {
- public getServicerAddress() {
- super("getServicerAddress");
- }
-
- @Override
- public getServicerAddress_args getEmptyArgsInstance() {
- return new getServicerAddress_args();
- }
-
- @Override
- protected boolean isOneway() {
- return false;
- }
-
- @Override
- protected boolean rethrowUnhandledExceptions() {
- return false;
- }
-
- @Override
- public getServicerAddress_result getResult(I iface, getServicerAddress_args args) throws org.apache.thrift.TException {
- getServicerAddress_result result = new getServicerAddress_result();
- try {
- result.success = iface.getServicerAddress(args.remoteTableId, args.credentials);
- } catch (ReplicationCoordinatorException e) {
- result.e = e;
- }
- return result;
- }
- }
-
- }
-
- public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
- private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
- public AsyncProcessor(I iface) {
- super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
- }
-
- protected AsyncProcessor(I iface, java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>> processMap) {
- super(iface, getProcessMap(processMap));
- }
-
- private static <I extends AsyncIface> java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>> processMap) {
- processMap.put("getServicerAddress", new getServicerAddress());
- return processMap;
- }
-
- public static class getServicerAddress<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getServicerAddress_args, java.lang.String> {
- public getServicerAddress() {
- super("getServicerAddress");
- }
-
- @Override
- public getServicerAddress_args getEmptyArgsInstance() {
- return new getServicerAddress_args();
- }
-
- @Override
- public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
- final org.apache.thrift.AsyncProcessFunction fcall = this;
- return new org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() {
- @Override
- public void onComplete(java.lang.String o) {
- getServicerAddress_result result = new getServicerAddress_result();
- result.success = o;
- try {
- fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
- } catch (org.apache.thrift.transport.TTransportException e) {
- _LOGGER.error("TTransportException writing to internal frame buffer", e);
- fb.close();
- } catch (java.lang.Exception e) {
- _LOGGER.error("Exception writing to internal frame buffer", e);
- onError(e);
- }
- }
- @Override
- public void onError(java.lang.Exception e) {
- byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
- org.apache.thrift.TSerializable msg;
- getServicerAddress_result result = new getServicerAddress_result();
- if (e instanceof ReplicationCoordinatorException) {
- result.e = (ReplicationCoordinatorException) e;
- result.setEIsSet(true);
- msg = result;
- } else if (e instanceof org.apache.thrift.transport.TTransportException) {
- _LOGGER.error("TTransportException inside handler", e);
- fb.close();
- return;
- } else if (e instanceof org.apache.thrift.TApplicationException) {
- _LOGGER.error("TApplicationException inside handler", e);
- msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
- msg = (org.apache.thrift.TApplicationException)e;
- } else {
- _LOGGER.error("Exception inside handler", e);
- msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
- msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
- }
- try {
- fcall.sendResponse(fb,msg,msgType,seqid);
- } catch (java.lang.Exception ex) {
- _LOGGER.error("Exception writing to internal frame buffer", ex);
- fb.close();
- }
- }
- };
- }
-
- @Override
- protected boolean isOneway() {
- return false;
- }
-
- @Override
- public void start(I iface, getServicerAddress_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
- iface.getServicerAddress(args.remoteTableId, args.credentials,resultHandler);
- }
- }
-
- }
-
- @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
- public static class getServicerAddress_args implements org.apache.thrift.TBase<getServicerAddress_args, getServicerAddress_args._Fields>, java.io.Serializable, Cloneable, Comparable<getServicerAddress_args> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getServicerAddress_args");
-
- private static final org.apache.thrift.protocol.TField REMOTE_TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("remoteTableId", org.apache.thrift.protocol.TType.STRING, (short)1);
- private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getServicerAddress_argsStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getServicerAddress_argsTupleSchemeFactory();
-
- public @org.apache.thrift.annotation.Nullable java.lang.String remoteTableId; // required
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- REMOTE_TABLE_ID((short)1, "remoteTableId"),
- CREDENTIALS((short)2, "credentials");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 1: // REMOTE_TABLE_ID
- return REMOTE_TABLE_ID;
- case 2: // CREDENTIALS
- return CREDENTIALS;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.REMOTE_TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("remoteTableId", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
- tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getServicerAddress_args.class, metaDataMap);
- }
-
- public getServicerAddress_args() {
- }
-
- public getServicerAddress_args(
- java.lang.String remoteTableId,
- org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
- {
- this();
- this.remoteTableId = remoteTableId;
- this.credentials = credentials;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public getServicerAddress_args(getServicerAddress_args other) {
- if (other.isSetRemoteTableId()) {
- this.remoteTableId = other.remoteTableId;
- }
- if (other.isSetCredentials()) {
- this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
- }
- }
-
- @Override
- public getServicerAddress_args deepCopy() {
- return new getServicerAddress_args(this);
- }
-
- @Override
- public void clear() {
- this.remoteTableId = null;
- this.credentials = null;
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.lang.String getRemoteTableId() {
- return this.remoteTableId;
- }
-
- public getServicerAddress_args setRemoteTableId(@org.apache.thrift.annotation.Nullable java.lang.String remoteTableId) {
- this.remoteTableId = remoteTableId;
- return this;
- }
-
- public void unsetRemoteTableId() {
- this.remoteTableId = null;
- }
-
- /** Returns true if field remoteTableId is set (has been assigned a value) and false otherwise */
- public boolean isSetRemoteTableId() {
- return this.remoteTableId != null;
- }
-
- public void setRemoteTableIdIsSet(boolean value) {
- if (!value) {
- this.remoteTableId = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
- return this.credentials;
- }
-
- public getServicerAddress_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
- this.credentials = credentials;
- return this;
- }
-
- public void unsetCredentials() {
- this.credentials = null;
- }
-
- /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
- public boolean isSetCredentials() {
- return this.credentials != null;
- }
-
- public void setCredentialsIsSet(boolean value) {
- if (!value) {
- this.credentials = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case REMOTE_TABLE_ID:
- if (value == null) {
- unsetRemoteTableId();
- } else {
- setRemoteTableId((java.lang.String)value);
- }
- break;
-
- case CREDENTIALS:
- if (value == null) {
- unsetCredentials();
- } else {
- setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case REMOTE_TABLE_ID:
- return getRemoteTableId();
-
- case CREDENTIALS:
- return getCredentials();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case REMOTE_TABLE_ID:
- return isSetRemoteTableId();
- case CREDENTIALS:
- return isSetCredentials();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof getServicerAddress_args)
- return this.equals((getServicerAddress_args)that);
- return false;
- }
-
- public boolean equals(getServicerAddress_args that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_remoteTableId = true && this.isSetRemoteTableId();
- boolean that_present_remoteTableId = true && that.isSetRemoteTableId();
- if (this_present_remoteTableId || that_present_remoteTableId) {
- if (!(this_present_remoteTableId && that_present_remoteTableId))
- return false;
- if (!this.remoteTableId.equals(that.remoteTableId))
- return false;
- }
-
- boolean this_present_credentials = true && this.isSetCredentials();
- boolean that_present_credentials = true && that.isSetCredentials();
- if (this_present_credentials || that_present_credentials) {
- if (!(this_present_credentials && that_present_credentials))
- return false;
- if (!this.credentials.equals(that.credentials))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetRemoteTableId()) ? 131071 : 524287);
- if (isSetRemoteTableId())
- hashCode = hashCode * 8191 + remoteTableId.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
- if (isSetCredentials())
- hashCode = hashCode * 8191 + credentials.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(getServicerAddress_args other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetRemoteTableId(), other.isSetRemoteTableId());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetRemoteTableId()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.remoteTableId, other.remoteTableId);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetCredentials()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("getServicerAddress_args(");
- boolean first = true;
-
- sb.append("remoteTableId:");
- if (this.remoteTableId == null) {
- sb.append("null");
- } else {
- sb.append(this.remoteTableId);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("credentials:");
- if (this.credentials == null) {
- sb.append("null");
- } else {
- sb.append(this.credentials);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- if (credentials != null) {
- credentials.validate();
- }
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class getServicerAddress_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public getServicerAddress_argsStandardScheme getScheme() {
- return new getServicerAddress_argsStandardScheme();
- }
- }
-
- private static class getServicerAddress_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getServicerAddress_args> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, getServicerAddress_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 1: // REMOTE_TABLE_ID
- if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
- struct.remoteTableId = iprot.readString();
- struct.setRemoteTableIdIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 2: // CREDENTIALS
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
- struct.credentials.read(iprot);
- struct.setCredentialsIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, getServicerAddress_args struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.remoteTableId != null) {
- oprot.writeFieldBegin(REMOTE_TABLE_ID_FIELD_DESC);
- oprot.writeString(struct.remoteTableId);
- oprot.writeFieldEnd();
- }
- if (struct.credentials != null) {
- oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
- struct.credentials.write(oprot);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class getServicerAddress_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public getServicerAddress_argsTupleScheme getScheme() {
- return new getServicerAddress_argsTupleScheme();
- }
- }
-
- private static class getServicerAddress_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getServicerAddress_args> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetRemoteTableId()) {
- optionals.set(0);
- }
- if (struct.isSetCredentials()) {
- optionals.set(1);
- }
- oprot.writeBitSet(optionals, 2);
- if (struct.isSetRemoteTableId()) {
- oprot.writeString(struct.remoteTableId);
- }
- if (struct.isSetCredentials()) {
- struct.credentials.write(oprot);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(2);
- if (incoming.get(0)) {
- struct.remoteTableId = iprot.readString();
- struct.setRemoteTableIdIsSet(true);
- }
- if (incoming.get(1)) {
- struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
- struct.credentials.read(iprot);
- struct.setCredentialsIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- }
-
- @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
- public static class getServicerAddress_result implements org.apache.thrift.TBase<getServicerAddress_result, getServicerAddress_result._Fields>, java.io.Serializable, Cloneable, Comparable<getServicerAddress_result> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getServicerAddress_result");
-
- private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0);
- private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getServicerAddress_resultStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getServicerAddress_resultTupleSchemeFactory();
-
- public @org.apache.thrift.annotation.Nullable java.lang.String success; // required
- public @org.apache.thrift.annotation.Nullable ReplicationCoordinatorException e; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- SUCCESS((short)0, "success"),
- E((short)1, "e");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 0: // SUCCESS
- return SUCCESS;
- case 1: // E
- return E;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
- tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ReplicationCoordinatorException.class)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getServicerAddress_result.class, metaDataMap);
- }
-
- public getServicerAddress_result() {
- }
-
- public getServicerAddress_result(
- java.lang.String success,
- ReplicationCoordinatorException e)
- {
- this();
- this.success = success;
- this.e = e;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public getServicerAddress_result(getServicerAddress_result other) {
- if (other.isSetSuccess()) {
- this.success = other.success;
- }
- if (other.isSetE()) {
- this.e = new ReplicationCoordinatorException(other.e);
- }
- }
-
- @Override
- public getServicerAddress_result deepCopy() {
- return new getServicerAddress_result(this);
- }
-
- @Override
- public void clear() {
- this.success = null;
- this.e = null;
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.lang.String getSuccess() {
- return this.success;
- }
-
- public getServicerAddress_result setSuccess(@org.apache.thrift.annotation.Nullable java.lang.String success) {
- this.success = success;
- return this;
- }
-
- public void unsetSuccess() {
- this.success = null;
- }
-
- /** Returns true if field success is set (has been assigned a value) and false otherwise */
- public boolean isSetSuccess() {
- return this.success != null;
- }
-
- public void setSuccessIsSet(boolean value) {
- if (!value) {
- this.success = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public ReplicationCoordinatorException getE() {
- return this.e;
- }
-
- public getServicerAddress_result setE(@org.apache.thrift.annotation.Nullable ReplicationCoordinatorException e) {
- this.e = e;
- return this;
- }
-
- public void unsetE() {
- this.e = null;
- }
-
- /** Returns true if field e is set (has been assigned a value) and false otherwise */
- public boolean isSetE() {
- return this.e != null;
- }
-
- public void setEIsSet(boolean value) {
- if (!value) {
- this.e = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case SUCCESS:
- if (value == null) {
- unsetSuccess();
- } else {
- setSuccess((java.lang.String)value);
- }
- break;
-
- case E:
- if (value == null) {
- unsetE();
- } else {
- setE((ReplicationCoordinatorException)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case SUCCESS:
- return getSuccess();
-
- case E:
- return getE();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case SUCCESS:
- return isSetSuccess();
- case E:
- return isSetE();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof getServicerAddress_result)
- return this.equals((getServicerAddress_result)that);
- return false;
- }
-
- public boolean equals(getServicerAddress_result that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_success = true && this.isSetSuccess();
- boolean that_present_success = true && that.isSetSuccess();
- if (this_present_success || that_present_success) {
- if (!(this_present_success && that_present_success))
- return false;
- if (!this.success.equals(that.success))
- return false;
- }
-
- boolean this_present_e = true && this.isSetE();
- boolean that_present_e = true && that.isSetE();
- if (this_present_e || that_present_e) {
- if (!(this_present_e && that_present_e))
- return false;
- if (!this.e.equals(that.e))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
- if (isSetSuccess())
- hashCode = hashCode * 8191 + success.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetE()) ? 131071 : 524287);
- if (isSetE())
- hashCode = hashCode * 8191 + e.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(getServicerAddress_result other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetSuccess()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetE(), other.isSetE());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetE()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("getServicerAddress_result(");
- boolean first = true;
-
- sb.append("success:");
- if (this.success == null) {
- sb.append("null");
- } else {
- sb.append(this.success);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("e:");
- if (this.e == null) {
- sb.append("null");
- } else {
- sb.append(this.e);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class getServicerAddress_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public getServicerAddress_resultStandardScheme getScheme() {
- return new getServicerAddress_resultStandardScheme();
- }
- }
-
- private static class getServicerAddress_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getServicerAddress_result> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, getServicerAddress_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 0: // SUCCESS
- if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
- struct.success = iprot.readString();
- struct.setSuccessIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 1: // E
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.e = new ReplicationCoordinatorException();
- struct.e.read(iprot);
- struct.setEIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, getServicerAddress_result struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.success != null) {
- oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
- oprot.writeString(struct.success);
- oprot.writeFieldEnd();
- }
- if (struct.e != null) {
- oprot.writeFieldBegin(E_FIELD_DESC);
- struct.e.write(oprot);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class getServicerAddress_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public getServicerAddress_resultTupleScheme getScheme() {
- return new getServicerAddress_resultTupleScheme();
- }
- }
-
- private static class getServicerAddress_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getServicerAddress_result> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetSuccess()) {
- optionals.set(0);
- }
- if (struct.isSetE()) {
- optionals.set(1);
- }
- oprot.writeBitSet(optionals, 2);
- if (struct.isSetSuccess()) {
- oprot.writeString(struct.success);
- }
- if (struct.isSetE()) {
- struct.e.write(oprot);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(2);
- if (incoming.get(0)) {
- struct.success = iprot.readString();
- struct.setSuccessIsSet(true);
- }
- if (incoming.get(1)) {
- struct.e = new ReplicationCoordinatorException();
- struct.e.read(iprot);
- struct.setEIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- }
-
- private static void unusedMethod() {}
-}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorErrorCode.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorErrorCode.java
deleted file mode 100644
index 2f1c337..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorErrorCode.java
+++ /dev/null
@@ -1,64 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- * @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-
-public enum ReplicationCoordinatorErrorCode implements org.apache.thrift.TEnum {
- NO_AVAILABLE_SERVERS(0),
- SERVICE_CONFIGURATION_UNAVAILABLE(1),
- CANNOT_AUTHENTICATE(2);
-
- private final int value;
-
- private ReplicationCoordinatorErrorCode(int value) {
- this.value = value;
- }
-
- /**
- * Get the integer value of this enum value, as defined in the Thrift IDL.
- */
- @Override
- public int getValue() {
- return value;
- }
-
- /**
- * Find a the enum type by its integer value, as defined in the Thrift IDL.
- * @return null if the value is not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static ReplicationCoordinatorErrorCode findByValue(int value) {
- switch (value) {
- case 0:
- return NO_AVAILABLE_SERVERS;
- case 1:
- return SERVICE_CONFIGURATION_UNAVAILABLE;
- case 2:
- return CANNOT_AUTHENTICATE;
- default:
- return null;
- }
- }
-}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorException.java
deleted file mode 100644
index 48ad99a..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationCoordinatorException.java
+++ /dev/null
@@ -1,527 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- * @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class ReplicationCoordinatorException extends org.apache.thrift.TException implements org.apache.thrift.TBase<ReplicationCoordinatorException, ReplicationCoordinatorException._Fields>, java.io.Serializable, Cloneable, Comparable<ReplicationCoordinatorException> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ReplicationCoordinatorException");
-
- private static final org.apache.thrift.protocol.TField CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("code", org.apache.thrift.protocol.TType.I32, (short)1);
- private static final org.apache.thrift.protocol.TField REASON_FIELD_DESC = new org.apache.thrift.protocol.TField("reason", org.apache.thrift.protocol.TType.STRING, (short)2);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new ReplicationCoordinatorExceptionStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new ReplicationCoordinatorExceptionTupleSchemeFactory();
-
- /**
- *
- * @see ReplicationCoordinatorErrorCode
- */
- public @org.apache.thrift.annotation.Nullable ReplicationCoordinatorErrorCode code; // required
- public @org.apache.thrift.annotation.Nullable java.lang.String reason; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- /**
- *
- * @see ReplicationCoordinatorErrorCode
- */
- CODE((short)1, "code"),
- REASON((short)2, "reason");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 1: // CODE
- return CODE;
- case 2: // REASON
- return REASON;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.CODE, new org.apache.thrift.meta_data.FieldMetaData("code", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ReplicationCoordinatorErrorCode.class)));
- tmpMap.put(_Fields.REASON, new org.apache.thrift.meta_data.FieldMetaData("reason", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ReplicationCoordinatorException.class, metaDataMap);
- }
-
- public ReplicationCoordinatorException() {
- }
-
- public ReplicationCoordinatorException(
- ReplicationCoordinatorErrorCode code,
- java.lang.String reason)
- {
- this();
- this.code = code;
- this.reason = reason;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public ReplicationCoordinatorException(ReplicationCoordinatorException other) {
- if (other.isSetCode()) {
- this.code = other.code;
- }
- if (other.isSetReason()) {
- this.reason = other.reason;
- }
- }
-
- @Override
- public ReplicationCoordinatorException deepCopy() {
- return new ReplicationCoordinatorException(this);
- }
-
- @Override
- public void clear() {
- this.code = null;
- this.reason = null;
- }
-
- /**
- *
- * @see ReplicationCoordinatorErrorCode
- */
- @org.apache.thrift.annotation.Nullable
- public ReplicationCoordinatorErrorCode getCode() {
- return this.code;
- }
-
- /**
- *
- * @see ReplicationCoordinatorErrorCode
- */
- public ReplicationCoordinatorException setCode(@org.apache.thrift.annotation.Nullable ReplicationCoordinatorErrorCode code) {
- this.code = code;
- return this;
- }
-
- public void unsetCode() {
- this.code = null;
- }
-
- /** Returns true if field code is set (has been assigned a value) and false otherwise */
- public boolean isSetCode() {
- return this.code != null;
- }
-
- public void setCodeIsSet(boolean value) {
- if (!value) {
- this.code = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.lang.String getReason() {
- return this.reason;
- }
-
- public ReplicationCoordinatorException setReason(@org.apache.thrift.annotation.Nullable java.lang.String reason) {
- this.reason = reason;
- return this;
- }
-
- public void unsetReason() {
- this.reason = null;
- }
-
- /** Returns true if field reason is set (has been assigned a value) and false otherwise */
- public boolean isSetReason() {
- return this.reason != null;
- }
-
- public void setReasonIsSet(boolean value) {
- if (!value) {
- this.reason = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case CODE:
- if (value == null) {
- unsetCode();
- } else {
- setCode((ReplicationCoordinatorErrorCode)value);
- }
- break;
-
- case REASON:
- if (value == null) {
- unsetReason();
- } else {
- setReason((java.lang.String)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case CODE:
- return getCode();
-
- case REASON:
- return getReason();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case CODE:
- return isSetCode();
- case REASON:
- return isSetReason();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof ReplicationCoordinatorException)
- return this.equals((ReplicationCoordinatorException)that);
- return false;
- }
-
- public boolean equals(ReplicationCoordinatorException that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_code = true && this.isSetCode();
- boolean that_present_code = true && that.isSetCode();
- if (this_present_code || that_present_code) {
- if (!(this_present_code && that_present_code))
- return false;
- if (!this.code.equals(that.code))
- return false;
- }
-
- boolean this_present_reason = true && this.isSetReason();
- boolean that_present_reason = true && that.isSetReason();
- if (this_present_reason || that_present_reason) {
- if (!(this_present_reason && that_present_reason))
- return false;
- if (!this.reason.equals(that.reason))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetCode()) ? 131071 : 524287);
- if (isSetCode())
- hashCode = hashCode * 8191 + code.getValue();
-
- hashCode = hashCode * 8191 + ((isSetReason()) ? 131071 : 524287);
- if (isSetReason())
- hashCode = hashCode * 8191 + reason.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(ReplicationCoordinatorException other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetCode(), other.isSetCode());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetCode()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.code, other.code);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetReason(), other.isSetReason());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetReason()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reason, other.reason);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("ReplicationCoordinatorException(");
- boolean first = true;
-
- sb.append("code:");
- if (this.code == null) {
- sb.append("null");
- } else {
- sb.append(this.code);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("reason:");
- if (this.reason == null) {
- sb.append("null");
- } else {
- sb.append(this.reason);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class ReplicationCoordinatorExceptionStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public ReplicationCoordinatorExceptionStandardScheme getScheme() {
- return new ReplicationCoordinatorExceptionStandardScheme();
- }
- }
-
- private static class ReplicationCoordinatorExceptionStandardScheme extends org.apache.thrift.scheme.StandardScheme<ReplicationCoordinatorException> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, ReplicationCoordinatorException struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 1: // CODE
- if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
- struct.code = org.apache.accumulo.core.replication.thrift.ReplicationCoordinatorErrorCode.findByValue(iprot.readI32());
- struct.setCodeIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 2: // REASON
- if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
- struct.reason = iprot.readString();
- struct.setReasonIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, ReplicationCoordinatorException struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.code != null) {
- oprot.writeFieldBegin(CODE_FIELD_DESC);
- oprot.writeI32(struct.code.getValue());
- oprot.writeFieldEnd();
- }
- if (struct.reason != null) {
- oprot.writeFieldBegin(REASON_FIELD_DESC);
- oprot.writeString(struct.reason);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class ReplicationCoordinatorExceptionTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public ReplicationCoordinatorExceptionTupleScheme getScheme() {
- return new ReplicationCoordinatorExceptionTupleScheme();
- }
- }
-
- private static class ReplicationCoordinatorExceptionTupleScheme extends org.apache.thrift.scheme.TupleScheme<ReplicationCoordinatorException> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, ReplicationCoordinatorException struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetCode()) {
- optionals.set(0);
- }
- if (struct.isSetReason()) {
- optionals.set(1);
- }
- oprot.writeBitSet(optionals, 2);
- if (struct.isSetCode()) {
- oprot.writeI32(struct.code.getValue());
- }
- if (struct.isSetReason()) {
- oprot.writeString(struct.reason);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, ReplicationCoordinatorException struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(2);
- if (incoming.get(0)) {
- struct.code = org.apache.accumulo.core.replication.thrift.ReplicationCoordinatorErrorCode.findByValue(iprot.readI32());
- struct.setCodeIsSet(true);
- }
- if (incoming.get(1)) {
- struct.reason = iprot.readString();
- struct.setReasonIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- private static void unusedMethod() {}
-}
-
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationServicer.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationServicer.java
deleted file mode 100644
index 7d7e41c..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/ReplicationServicer.java
+++ /dev/null
@@ -1,2638 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- * @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class ReplicationServicer {
-
- public interface Iface {
-
- public long replicateLog(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws RemoteReplicationException, org.apache.thrift.TException;
-
- public long replicateKeyValues(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws RemoteReplicationException, org.apache.thrift.TException;
-
- }
-
- public interface AsyncIface {
-
- public void replicateLog(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
-
- public void replicateKeyValues(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
-
- }
-
- public static class Client extends org.apache.thrift.TServiceClient implements Iface {
- public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
- public Factory() {}
- @Override
- public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
- return new Client(prot);
- }
- @Override
- public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
- return new Client(iprot, oprot);
- }
- }
-
- public Client(org.apache.thrift.protocol.TProtocol prot)
- {
- super(prot, prot);
- }
-
- public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
- super(iprot, oprot);
- }
-
- @Override
- public long replicateLog(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws RemoteReplicationException, org.apache.thrift.TException
- {
- send_replicateLog(remoteTableId, data, credentials);
- return recv_replicateLog();
- }
-
- public void send_replicateLog(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
- {
- replicateLog_args args = new replicateLog_args();
- args.setRemoteTableId(remoteTableId);
- args.setData(data);
- args.setCredentials(credentials);
- sendBase("replicateLog", args);
- }
-
- public long recv_replicateLog() throws RemoteReplicationException, org.apache.thrift.TException
- {
- replicateLog_result result = new replicateLog_result();
- receiveBase(result, "replicateLog");
- if (result.isSetSuccess()) {
- return result.success;
- }
- if (result.e != null) {
- throw result.e;
- }
- throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "replicateLog failed: unknown result");
- }
-
- @Override
- public long replicateKeyValues(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws RemoteReplicationException, org.apache.thrift.TException
- {
- send_replicateKeyValues(remoteTableId, data, credentials);
- return recv_replicateKeyValues();
- }
-
- public void send_replicateKeyValues(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
- {
- replicateKeyValues_args args = new replicateKeyValues_args();
- args.setRemoteTableId(remoteTableId);
- args.setData(data);
- args.setCredentials(credentials);
- sendBase("replicateKeyValues", args);
- }
-
- public long recv_replicateKeyValues() throws RemoteReplicationException, org.apache.thrift.TException
- {
- replicateKeyValues_result result = new replicateKeyValues_result();
- receiveBase(result, "replicateKeyValues");
- if (result.isSetSuccess()) {
- return result.success;
- }
- if (result.e != null) {
- throw result.e;
- }
- throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "replicateKeyValues failed: unknown result");
- }
-
- }
- public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
- public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
- private org.apache.thrift.async.TAsyncClientManager clientManager;
- private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
- public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
- this.clientManager = clientManager;
- this.protocolFactory = protocolFactory;
- }
- @Override
- public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
- return new AsyncClient(protocolFactory, clientManager, transport);
- }
- }
-
- public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
- super(protocolFactory, clientManager, transport);
- }
-
- @Override
- public void replicateLog(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
- checkReady();
- replicateLog_call method_call = new replicateLog_call(remoteTableId, data, credentials, resultHandler, this, ___protocolFactory, ___transport);
- this.___currentMethod = method_call;
- ___manager.call(method_call);
- }
-
- public static class replicateLog_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
- private java.lang.String remoteTableId;
- private WalEdits data;
- private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public replicateLog_call(java.lang.String remoteTableId, WalEdits data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
- super(client, protocolFactory, transport, resultHandler, false);
- this.remoteTableId = remoteTableId;
- this.data = data;
- this.credentials = credentials;
- }
-
- @Override
- public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
- prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("replicateLog", org.apache.thrift.protocol.TMessageType.CALL, 0));
- replicateLog_args args = new replicateLog_args();
- args.setRemoteTableId(remoteTableId);
- args.setData(data);
- args.setCredentials(credentials);
- args.write(prot);
- prot.writeMessageEnd();
- }
-
- @Override
- public java.lang.Long getResult() throws RemoteReplicationException, org.apache.thrift.TException {
- if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
- throw new java.lang.IllegalStateException("Method call not finished!");
- }
- org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
- org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
- return (new Client(prot)).recv_replicateLog();
- }
- }
-
- @Override
- public void replicateKeyValues(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
- checkReady();
- replicateKeyValues_call method_call = new replicateKeyValues_call(remoteTableId, data, credentials, resultHandler, this, ___protocolFactory, ___transport);
- this.___currentMethod = method_call;
- ___manager.call(method_call);
- }
-
- public static class replicateKeyValues_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
- private java.lang.String remoteTableId;
- private KeyValues data;
- private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
- public replicateKeyValues_call(java.lang.String remoteTableId, KeyValues data, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
- super(client, protocolFactory, transport, resultHandler, false);
- this.remoteTableId = remoteTableId;
- this.data = data;
- this.credentials = credentials;
- }
-
- @Override
- public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
- prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("replicateKeyValues", org.apache.thrift.protocol.TMessageType.CALL, 0));
- replicateKeyValues_args args = new replicateKeyValues_args();
- args.setRemoteTableId(remoteTableId);
- args.setData(data);
- args.setCredentials(credentials);
- args.write(prot);
- prot.writeMessageEnd();
- }
-
- @Override
- public java.lang.Long getResult() throws RemoteReplicationException, org.apache.thrift.TException {
- if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
- throw new java.lang.IllegalStateException("Method call not finished!");
- }
- org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
- org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
- return (new Client(prot)).recv_replicateKeyValues();
- }
- }
-
- }
-
- public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
- private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
- public Processor(I iface) {
- super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
- }
-
- protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
- super(iface, getProcessMap(processMap));
- }
-
- private static <I extends Iface> java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
- processMap.put("replicateLog", new replicateLog());
- processMap.put("replicateKeyValues", new replicateKeyValues());
- return processMap;
- }
-
- public static class replicateLog<I extends Iface> extends org.apache.thrift.ProcessFunction<I, replicateLog_args> {
- public replicateLog() {
- super("replicateLog");
- }
-
- @Override
- public replicateLog_args getEmptyArgsInstance() {
- return new replicateLog_args();
- }
-
- @Override
- protected boolean isOneway() {
- return false;
- }
-
- @Override
- protected boolean rethrowUnhandledExceptions() {
- return false;
- }
-
- @Override
- public replicateLog_result getResult(I iface, replicateLog_args args) throws org.apache.thrift.TException {
- replicateLog_result result = new replicateLog_result();
- try {
- result.success = iface.replicateLog(args.remoteTableId, args.data, args.credentials);
- result.setSuccessIsSet(true);
- } catch (RemoteReplicationException e) {
- result.e = e;
- }
- return result;
- }
- }
-
- public static class replicateKeyValues<I extends Iface> extends org.apache.thrift.ProcessFunction<I, replicateKeyValues_args> {
- public replicateKeyValues() {
- super("replicateKeyValues");
- }
-
- @Override
- public replicateKeyValues_args getEmptyArgsInstance() {
- return new replicateKeyValues_args();
- }
-
- @Override
- protected boolean isOneway() {
- return false;
- }
-
- @Override
- protected boolean rethrowUnhandledExceptions() {
- return false;
- }
-
- @Override
- public replicateKeyValues_result getResult(I iface, replicateKeyValues_args args) throws org.apache.thrift.TException {
- replicateKeyValues_result result = new replicateKeyValues_result();
- try {
- result.success = iface.replicateKeyValues(args.remoteTableId, args.data, args.credentials);
- result.setSuccessIsSet(true);
- } catch (RemoteReplicationException e) {
- result.e = e;
- }
- return result;
- }
- }
-
- }
-
- public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
- private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
- public AsyncProcessor(I iface) {
- super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
- }
-
- protected AsyncProcessor(I iface, java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>> processMap) {
- super(iface, getProcessMap(processMap));
- }
-
- private static <I extends AsyncIface> java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>> processMap) {
- processMap.put("replicateLog", new replicateLog());
- processMap.put("replicateKeyValues", new replicateKeyValues());
- return processMap;
- }
-
- public static class replicateLog<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, replicateLog_args, java.lang.Long> {
- public replicateLog() {
- super("replicateLog");
- }
-
- @Override
- public replicateLog_args getEmptyArgsInstance() {
- return new replicateLog_args();
- }
-
- @Override
- public org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
- final org.apache.thrift.AsyncProcessFunction fcall = this;
- return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Long>() {
- @Override
- public void onComplete(java.lang.Long o) {
- replicateLog_result result = new replicateLog_result();
- result.success = o;
- result.setSuccessIsSet(true);
- try {
- fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
- } catch (org.apache.thrift.transport.TTransportException e) {
- _LOGGER.error("TTransportException writing to internal frame buffer", e);
- fb.close();
- } catch (java.lang.Exception e) {
- _LOGGER.error("Exception writing to internal frame buffer", e);
- onError(e);
- }
- }
- @Override
- public void onError(java.lang.Exception e) {
- byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
- org.apache.thrift.TSerializable msg;
- replicateLog_result result = new replicateLog_result();
- if (e instanceof RemoteReplicationException) {
- result.e = (RemoteReplicationException) e;
- result.setEIsSet(true);
- msg = result;
- } else if (e instanceof org.apache.thrift.transport.TTransportException) {
- _LOGGER.error("TTransportException inside handler", e);
- fb.close();
- return;
- } else if (e instanceof org.apache.thrift.TApplicationException) {
- _LOGGER.error("TApplicationException inside handler", e);
- msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
- msg = (org.apache.thrift.TApplicationException)e;
- } else {
- _LOGGER.error("Exception inside handler", e);
- msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
- msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
- }
- try {
- fcall.sendResponse(fb,msg,msgType,seqid);
- } catch (java.lang.Exception ex) {
- _LOGGER.error("Exception writing to internal frame buffer", ex);
- fb.close();
- }
- }
- };
- }
-
- @Override
- protected boolean isOneway() {
- return false;
- }
-
- @Override
- public void start(I iface, replicateLog_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
- iface.replicateLog(args.remoteTableId, args.data, args.credentials,resultHandler);
- }
- }
-
- public static class replicateKeyValues<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, replicateKeyValues_args, java.lang.Long> {
- public replicateKeyValues() {
- super("replicateKeyValues");
- }
-
- @Override
- public replicateKeyValues_args getEmptyArgsInstance() {
- return new replicateKeyValues_args();
- }
-
- @Override
- public org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
- final org.apache.thrift.AsyncProcessFunction fcall = this;
- return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Long>() {
- @Override
- public void onComplete(java.lang.Long o) {
- replicateKeyValues_result result = new replicateKeyValues_result();
- result.success = o;
- result.setSuccessIsSet(true);
- try {
- fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
- } catch (org.apache.thrift.transport.TTransportException e) {
- _LOGGER.error("TTransportException writing to internal frame buffer", e);
- fb.close();
- } catch (java.lang.Exception e) {
- _LOGGER.error("Exception writing to internal frame buffer", e);
- onError(e);
- }
- }
- @Override
- public void onError(java.lang.Exception e) {
- byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
- org.apache.thrift.TSerializable msg;
- replicateKeyValues_result result = new replicateKeyValues_result();
- if (e instanceof RemoteReplicationException) {
- result.e = (RemoteReplicationException) e;
- result.setEIsSet(true);
- msg = result;
- } else if (e instanceof org.apache.thrift.transport.TTransportException) {
- _LOGGER.error("TTransportException inside handler", e);
- fb.close();
- return;
- } else if (e instanceof org.apache.thrift.TApplicationException) {
- _LOGGER.error("TApplicationException inside handler", e);
- msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
- msg = (org.apache.thrift.TApplicationException)e;
- } else {
- _LOGGER.error("Exception inside handler", e);
- msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
- msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
- }
- try {
- fcall.sendResponse(fb,msg,msgType,seqid);
- } catch (java.lang.Exception ex) {
- _LOGGER.error("Exception writing to internal frame buffer", ex);
- fb.close();
- }
- }
- };
- }
-
- @Override
- protected boolean isOneway() {
- return false;
- }
-
- @Override
- public void start(I iface, replicateKeyValues_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
- iface.replicateKeyValues(args.remoteTableId, args.data, args.credentials,resultHandler);
- }
- }
-
- }
-
- @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
- public static class replicateLog_args implements org.apache.thrift.TBase<replicateLog_args, replicateLog_args._Fields>, java.io.Serializable, Cloneable, Comparable<replicateLog_args> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("replicateLog_args");
-
- private static final org.apache.thrift.protocol.TField REMOTE_TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("remoteTableId", org.apache.thrift.protocol.TType.STRING, (short)1);
- private static final org.apache.thrift.protocol.TField DATA_FIELD_DESC = new org.apache.thrift.protocol.TField("data", org.apache.thrift.protocol.TType.STRUCT, (short)2);
- private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new replicateLog_argsStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new replicateLog_argsTupleSchemeFactory();
-
- public @org.apache.thrift.annotation.Nullable java.lang.String remoteTableId; // required
- public @org.apache.thrift.annotation.Nullable WalEdits data; // required
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- REMOTE_TABLE_ID((short)1, "remoteTableId"),
- DATA((short)2, "data"),
- CREDENTIALS((short)3, "credentials");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 1: // REMOTE_TABLE_ID
- return REMOTE_TABLE_ID;
- case 2: // DATA
- return DATA;
- case 3: // CREDENTIALS
- return CREDENTIALS;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.REMOTE_TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("remoteTableId", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
- tmpMap.put(_Fields.DATA, new org.apache.thrift.meta_data.FieldMetaData("data", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WalEdits.class)));
- tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(replicateLog_args.class, metaDataMap);
- }
-
- public replicateLog_args() {
- }
-
- public replicateLog_args(
- java.lang.String remoteTableId,
- WalEdits data,
- org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
- {
- this();
- this.remoteTableId = remoteTableId;
- this.data = data;
- this.credentials = credentials;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public replicateLog_args(replicateLog_args other) {
- if (other.isSetRemoteTableId()) {
- this.remoteTableId = other.remoteTableId;
- }
- if (other.isSetData()) {
- this.data = new WalEdits(other.data);
- }
- if (other.isSetCredentials()) {
- this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
- }
- }
-
- @Override
- public replicateLog_args deepCopy() {
- return new replicateLog_args(this);
- }
-
- @Override
- public void clear() {
- this.remoteTableId = null;
- this.data = null;
- this.credentials = null;
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.lang.String getRemoteTableId() {
- return this.remoteTableId;
- }
-
- public replicateLog_args setRemoteTableId(@org.apache.thrift.annotation.Nullable java.lang.String remoteTableId) {
- this.remoteTableId = remoteTableId;
- return this;
- }
-
- public void unsetRemoteTableId() {
- this.remoteTableId = null;
- }
-
- /** Returns true if field remoteTableId is set (has been assigned a value) and false otherwise */
- public boolean isSetRemoteTableId() {
- return this.remoteTableId != null;
- }
-
- public void setRemoteTableIdIsSet(boolean value) {
- if (!value) {
- this.remoteTableId = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public WalEdits getData() {
- return this.data;
- }
-
- public replicateLog_args setData(@org.apache.thrift.annotation.Nullable WalEdits data) {
- this.data = data;
- return this;
- }
-
- public void unsetData() {
- this.data = null;
- }
-
- /** Returns true if field data is set (has been assigned a value) and false otherwise */
- public boolean isSetData() {
- return this.data != null;
- }
-
- public void setDataIsSet(boolean value) {
- if (!value) {
- this.data = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
- return this.credentials;
- }
-
- public replicateLog_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
- this.credentials = credentials;
- return this;
- }
-
- public void unsetCredentials() {
- this.credentials = null;
- }
-
- /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
- public boolean isSetCredentials() {
- return this.credentials != null;
- }
-
- public void setCredentialsIsSet(boolean value) {
- if (!value) {
- this.credentials = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case REMOTE_TABLE_ID:
- if (value == null) {
- unsetRemoteTableId();
- } else {
- setRemoteTableId((java.lang.String)value);
- }
- break;
-
- case DATA:
- if (value == null) {
- unsetData();
- } else {
- setData((WalEdits)value);
- }
- break;
-
- case CREDENTIALS:
- if (value == null) {
- unsetCredentials();
- } else {
- setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case REMOTE_TABLE_ID:
- return getRemoteTableId();
-
- case DATA:
- return getData();
-
- case CREDENTIALS:
- return getCredentials();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case REMOTE_TABLE_ID:
- return isSetRemoteTableId();
- case DATA:
- return isSetData();
- case CREDENTIALS:
- return isSetCredentials();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof replicateLog_args)
- return this.equals((replicateLog_args)that);
- return false;
- }
-
- public boolean equals(replicateLog_args that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_remoteTableId = true && this.isSetRemoteTableId();
- boolean that_present_remoteTableId = true && that.isSetRemoteTableId();
- if (this_present_remoteTableId || that_present_remoteTableId) {
- if (!(this_present_remoteTableId && that_present_remoteTableId))
- return false;
- if (!this.remoteTableId.equals(that.remoteTableId))
- return false;
- }
-
- boolean this_present_data = true && this.isSetData();
- boolean that_present_data = true && that.isSetData();
- if (this_present_data || that_present_data) {
- if (!(this_present_data && that_present_data))
- return false;
- if (!this.data.equals(that.data))
- return false;
- }
-
- boolean this_present_credentials = true && this.isSetCredentials();
- boolean that_present_credentials = true && that.isSetCredentials();
- if (this_present_credentials || that_present_credentials) {
- if (!(this_present_credentials && that_present_credentials))
- return false;
- if (!this.credentials.equals(that.credentials))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetRemoteTableId()) ? 131071 : 524287);
- if (isSetRemoteTableId())
- hashCode = hashCode * 8191 + remoteTableId.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetData()) ? 131071 : 524287);
- if (isSetData())
- hashCode = hashCode * 8191 + data.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
- if (isSetCredentials())
- hashCode = hashCode * 8191 + credentials.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(replicateLog_args other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetRemoteTableId(), other.isSetRemoteTableId());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetRemoteTableId()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.remoteTableId, other.remoteTableId);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetData(), other.isSetData());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetData()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.data, other.data);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetCredentials()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("replicateLog_args(");
- boolean first = true;
-
- sb.append("remoteTableId:");
- if (this.remoteTableId == null) {
- sb.append("null");
- } else {
- sb.append(this.remoteTableId);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("data:");
- if (this.data == null) {
- sb.append("null");
- } else {
- sb.append(this.data);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("credentials:");
- if (this.credentials == null) {
- sb.append("null");
- } else {
- sb.append(this.credentials);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- if (data != null) {
- data.validate();
- }
- if (credentials != null) {
- credentials.validate();
- }
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class replicateLog_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public replicateLog_argsStandardScheme getScheme() {
- return new replicateLog_argsStandardScheme();
- }
- }
-
- private static class replicateLog_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<replicateLog_args> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, replicateLog_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 1: // REMOTE_TABLE_ID
- if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
- struct.remoteTableId = iprot.readString();
- struct.setRemoteTableIdIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 2: // DATA
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.data = new WalEdits();
- struct.data.read(iprot);
- struct.setDataIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 3: // CREDENTIALS
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
- struct.credentials.read(iprot);
- struct.setCredentialsIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, replicateLog_args struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.remoteTableId != null) {
- oprot.writeFieldBegin(REMOTE_TABLE_ID_FIELD_DESC);
- oprot.writeString(struct.remoteTableId);
- oprot.writeFieldEnd();
- }
- if (struct.data != null) {
- oprot.writeFieldBegin(DATA_FIELD_DESC);
- struct.data.write(oprot);
- oprot.writeFieldEnd();
- }
- if (struct.credentials != null) {
- oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
- struct.credentials.write(oprot);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class replicateLog_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public replicateLog_argsTupleScheme getScheme() {
- return new replicateLog_argsTupleScheme();
- }
- }
-
- private static class replicateLog_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<replicateLog_args> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, replicateLog_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetRemoteTableId()) {
- optionals.set(0);
- }
- if (struct.isSetData()) {
- optionals.set(1);
- }
- if (struct.isSetCredentials()) {
- optionals.set(2);
- }
- oprot.writeBitSet(optionals, 3);
- if (struct.isSetRemoteTableId()) {
- oprot.writeString(struct.remoteTableId);
- }
- if (struct.isSetData()) {
- struct.data.write(oprot);
- }
- if (struct.isSetCredentials()) {
- struct.credentials.write(oprot);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, replicateLog_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(3);
- if (incoming.get(0)) {
- struct.remoteTableId = iprot.readString();
- struct.setRemoteTableIdIsSet(true);
- }
- if (incoming.get(1)) {
- struct.data = new WalEdits();
- struct.data.read(iprot);
- struct.setDataIsSet(true);
- }
- if (incoming.get(2)) {
- struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
- struct.credentials.read(iprot);
- struct.setCredentialsIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- }
-
- @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
- public static class replicateLog_result implements org.apache.thrift.TBase<replicateLog_result, replicateLog_result._Fields>, java.io.Serializable, Cloneable, Comparable<replicateLog_result> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("replicateLog_result");
-
- private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0);
- private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new replicateLog_resultStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new replicateLog_resultTupleSchemeFactory();
-
- public long success; // required
- public @org.apache.thrift.annotation.Nullable RemoteReplicationException e; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- SUCCESS((short)0, "success"),
- E((short)1, "e");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 0: // SUCCESS
- return SUCCESS;
- case 1: // E
- return E;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- private static final int __SUCCESS_ISSET_ID = 0;
- private byte __isset_bitfield = 0;
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
- tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RemoteReplicationException.class)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(replicateLog_result.class, metaDataMap);
- }
-
- public replicateLog_result() {
- }
-
- public replicateLog_result(
- long success,
- RemoteReplicationException e)
- {
- this();
- this.success = success;
- setSuccessIsSet(true);
- this.e = e;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public replicateLog_result(replicateLog_result other) {
- __isset_bitfield = other.__isset_bitfield;
- this.success = other.success;
- if (other.isSetE()) {
- this.e = new RemoteReplicationException(other.e);
- }
- }
-
- @Override
- public replicateLog_result deepCopy() {
- return new replicateLog_result(this);
- }
-
- @Override
- public void clear() {
- setSuccessIsSet(false);
- this.success = 0;
- this.e = null;
- }
-
- public long getSuccess() {
- return this.success;
- }
-
- public replicateLog_result setSuccess(long success) {
- this.success = success;
- setSuccessIsSet(true);
- return this;
- }
-
- public void unsetSuccess() {
- __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
- }
-
- /** Returns true if field success is set (has been assigned a value) and false otherwise */
- public boolean isSetSuccess() {
- return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
- }
-
- public void setSuccessIsSet(boolean value) {
- __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
- }
-
- @org.apache.thrift.annotation.Nullable
- public RemoteReplicationException getE() {
- return this.e;
- }
-
- public replicateLog_result setE(@org.apache.thrift.annotation.Nullable RemoteReplicationException e) {
- this.e = e;
- return this;
- }
-
- public void unsetE() {
- this.e = null;
- }
-
- /** Returns true if field e is set (has been assigned a value) and false otherwise */
- public boolean isSetE() {
- return this.e != null;
- }
-
- public void setEIsSet(boolean value) {
- if (!value) {
- this.e = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case SUCCESS:
- if (value == null) {
- unsetSuccess();
- } else {
- setSuccess((java.lang.Long)value);
- }
- break;
-
- case E:
- if (value == null) {
- unsetE();
- } else {
- setE((RemoteReplicationException)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case SUCCESS:
- return getSuccess();
-
- case E:
- return getE();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case SUCCESS:
- return isSetSuccess();
- case E:
- return isSetE();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof replicateLog_result)
- return this.equals((replicateLog_result)that);
- return false;
- }
-
- public boolean equals(replicateLog_result that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_success = true;
- boolean that_present_success = true;
- if (this_present_success || that_present_success) {
- if (!(this_present_success && that_present_success))
- return false;
- if (this.success != that.success)
- return false;
- }
-
- boolean this_present_e = true && this.isSetE();
- boolean that_present_e = true && that.isSetE();
- if (this_present_e || that_present_e) {
- if (!(this_present_e && that_present_e))
- return false;
- if (!this.e.equals(that.e))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(success);
-
- hashCode = hashCode * 8191 + ((isSetE()) ? 131071 : 524287);
- if (isSetE())
- hashCode = hashCode * 8191 + e.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(replicateLog_result other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetSuccess()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetE(), other.isSetE());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetE()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("replicateLog_result(");
- boolean first = true;
-
- sb.append("success:");
- sb.append(this.success);
- first = false;
- if (!first) sb.append(", ");
- sb.append("e:");
- if (this.e == null) {
- sb.append("null");
- } else {
- sb.append(this.e);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
- __isset_bitfield = 0;
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class replicateLog_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public replicateLog_resultStandardScheme getScheme() {
- return new replicateLog_resultStandardScheme();
- }
- }
-
- private static class replicateLog_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<replicateLog_result> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, replicateLog_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 0: // SUCCESS
- if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
- struct.success = iprot.readI64();
- struct.setSuccessIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 1: // E
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.e = new RemoteReplicationException();
- struct.e.read(iprot);
- struct.setEIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, replicateLog_result struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.isSetSuccess()) {
- oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
- oprot.writeI64(struct.success);
- oprot.writeFieldEnd();
- }
- if (struct.e != null) {
- oprot.writeFieldBegin(E_FIELD_DESC);
- struct.e.write(oprot);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class replicateLog_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public replicateLog_resultTupleScheme getScheme() {
- return new replicateLog_resultTupleScheme();
- }
- }
-
- private static class replicateLog_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<replicateLog_result> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, replicateLog_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetSuccess()) {
- optionals.set(0);
- }
- if (struct.isSetE()) {
- optionals.set(1);
- }
- oprot.writeBitSet(optionals, 2);
- if (struct.isSetSuccess()) {
- oprot.writeI64(struct.success);
- }
- if (struct.isSetE()) {
- struct.e.write(oprot);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, replicateLog_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(2);
- if (incoming.get(0)) {
- struct.success = iprot.readI64();
- struct.setSuccessIsSet(true);
- }
- if (incoming.get(1)) {
- struct.e = new RemoteReplicationException();
- struct.e.read(iprot);
- struct.setEIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- }
-
- @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
- public static class replicateKeyValues_args implements org.apache.thrift.TBase<replicateKeyValues_args, replicateKeyValues_args._Fields>, java.io.Serializable, Cloneable, Comparable<replicateKeyValues_args> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("replicateKeyValues_args");
-
- private static final org.apache.thrift.protocol.TField REMOTE_TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("remoteTableId", org.apache.thrift.protocol.TType.STRING, (short)1);
- private static final org.apache.thrift.protocol.TField DATA_FIELD_DESC = new org.apache.thrift.protocol.TField("data", org.apache.thrift.protocol.TType.STRUCT, (short)2);
- private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new replicateKeyValues_argsStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new replicateKeyValues_argsTupleSchemeFactory();
-
- public @org.apache.thrift.annotation.Nullable java.lang.String remoteTableId; // required
- public @org.apache.thrift.annotation.Nullable KeyValues data; // required
- public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- REMOTE_TABLE_ID((short)1, "remoteTableId"),
- DATA((short)2, "data"),
- CREDENTIALS((short)3, "credentials");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 1: // REMOTE_TABLE_ID
- return REMOTE_TABLE_ID;
- case 2: // DATA
- return DATA;
- case 3: // CREDENTIALS
- return CREDENTIALS;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.REMOTE_TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("remoteTableId", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
- tmpMap.put(_Fields.DATA, new org.apache.thrift.meta_data.FieldMetaData("data", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeyValues.class)));
- tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(replicateKeyValues_args.class, metaDataMap);
- }
-
- public replicateKeyValues_args() {
- }
-
- public replicateKeyValues_args(
- java.lang.String remoteTableId,
- KeyValues data,
- org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
- {
- this();
- this.remoteTableId = remoteTableId;
- this.data = data;
- this.credentials = credentials;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public replicateKeyValues_args(replicateKeyValues_args other) {
- if (other.isSetRemoteTableId()) {
- this.remoteTableId = other.remoteTableId;
- }
- if (other.isSetData()) {
- this.data = new KeyValues(other.data);
- }
- if (other.isSetCredentials()) {
- this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
- }
- }
-
- @Override
- public replicateKeyValues_args deepCopy() {
- return new replicateKeyValues_args(this);
- }
-
- @Override
- public void clear() {
- this.remoteTableId = null;
- this.data = null;
- this.credentials = null;
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.lang.String getRemoteTableId() {
- return this.remoteTableId;
- }
-
- public replicateKeyValues_args setRemoteTableId(@org.apache.thrift.annotation.Nullable java.lang.String remoteTableId) {
- this.remoteTableId = remoteTableId;
- return this;
- }
-
- public void unsetRemoteTableId() {
- this.remoteTableId = null;
- }
-
- /** Returns true if field remoteTableId is set (has been assigned a value) and false otherwise */
- public boolean isSetRemoteTableId() {
- return this.remoteTableId != null;
- }
-
- public void setRemoteTableIdIsSet(boolean value) {
- if (!value) {
- this.remoteTableId = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public KeyValues getData() {
- return this.data;
- }
-
- public replicateKeyValues_args setData(@org.apache.thrift.annotation.Nullable KeyValues data) {
- this.data = data;
- return this;
- }
-
- public void unsetData() {
- this.data = null;
- }
-
- /** Returns true if field data is set (has been assigned a value) and false otherwise */
- public boolean isSetData() {
- return this.data != null;
- }
-
- public void setDataIsSet(boolean value) {
- if (!value) {
- this.data = null;
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
- return this.credentials;
- }
-
- public replicateKeyValues_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
- this.credentials = credentials;
- return this;
- }
-
- public void unsetCredentials() {
- this.credentials = null;
- }
-
- /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
- public boolean isSetCredentials() {
- return this.credentials != null;
- }
-
- public void setCredentialsIsSet(boolean value) {
- if (!value) {
- this.credentials = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case REMOTE_TABLE_ID:
- if (value == null) {
- unsetRemoteTableId();
- } else {
- setRemoteTableId((java.lang.String)value);
- }
- break;
-
- case DATA:
- if (value == null) {
- unsetData();
- } else {
- setData((KeyValues)value);
- }
- break;
-
- case CREDENTIALS:
- if (value == null) {
- unsetCredentials();
- } else {
- setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case REMOTE_TABLE_ID:
- return getRemoteTableId();
-
- case DATA:
- return getData();
-
- case CREDENTIALS:
- return getCredentials();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case REMOTE_TABLE_ID:
- return isSetRemoteTableId();
- case DATA:
- return isSetData();
- case CREDENTIALS:
- return isSetCredentials();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof replicateKeyValues_args)
- return this.equals((replicateKeyValues_args)that);
- return false;
- }
-
- public boolean equals(replicateKeyValues_args that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_remoteTableId = true && this.isSetRemoteTableId();
- boolean that_present_remoteTableId = true && that.isSetRemoteTableId();
- if (this_present_remoteTableId || that_present_remoteTableId) {
- if (!(this_present_remoteTableId && that_present_remoteTableId))
- return false;
- if (!this.remoteTableId.equals(that.remoteTableId))
- return false;
- }
-
- boolean this_present_data = true && this.isSetData();
- boolean that_present_data = true && that.isSetData();
- if (this_present_data || that_present_data) {
- if (!(this_present_data && that_present_data))
- return false;
- if (!this.data.equals(that.data))
- return false;
- }
-
- boolean this_present_credentials = true && this.isSetCredentials();
- boolean that_present_credentials = true && that.isSetCredentials();
- if (this_present_credentials || that_present_credentials) {
- if (!(this_present_credentials && that_present_credentials))
- return false;
- if (!this.credentials.equals(that.credentials))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetRemoteTableId()) ? 131071 : 524287);
- if (isSetRemoteTableId())
- hashCode = hashCode * 8191 + remoteTableId.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetData()) ? 131071 : 524287);
- if (isSetData())
- hashCode = hashCode * 8191 + data.hashCode();
-
- hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
- if (isSetCredentials())
- hashCode = hashCode * 8191 + credentials.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(replicateKeyValues_args other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetRemoteTableId(), other.isSetRemoteTableId());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetRemoteTableId()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.remoteTableId, other.remoteTableId);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetData(), other.isSetData());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetData()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.data, other.data);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetCredentials()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("replicateKeyValues_args(");
- boolean first = true;
-
- sb.append("remoteTableId:");
- if (this.remoteTableId == null) {
- sb.append("null");
- } else {
- sb.append(this.remoteTableId);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("data:");
- if (this.data == null) {
- sb.append("null");
- } else {
- sb.append(this.data);
- }
- first = false;
- if (!first) sb.append(", ");
- sb.append("credentials:");
- if (this.credentials == null) {
- sb.append("null");
- } else {
- sb.append(this.credentials);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- if (data != null) {
- data.validate();
- }
- if (credentials != null) {
- credentials.validate();
- }
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class replicateKeyValues_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public replicateKeyValues_argsStandardScheme getScheme() {
- return new replicateKeyValues_argsStandardScheme();
- }
- }
-
- private static class replicateKeyValues_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<replicateKeyValues_args> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, replicateKeyValues_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 1: // REMOTE_TABLE_ID
- if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
- struct.remoteTableId = iprot.readString();
- struct.setRemoteTableIdIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 2: // DATA
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.data = new KeyValues();
- struct.data.read(iprot);
- struct.setDataIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 3: // CREDENTIALS
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
- struct.credentials.read(iprot);
- struct.setCredentialsIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, replicateKeyValues_args struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.remoteTableId != null) {
- oprot.writeFieldBegin(REMOTE_TABLE_ID_FIELD_DESC);
- oprot.writeString(struct.remoteTableId);
- oprot.writeFieldEnd();
- }
- if (struct.data != null) {
- oprot.writeFieldBegin(DATA_FIELD_DESC);
- struct.data.write(oprot);
- oprot.writeFieldEnd();
- }
- if (struct.credentials != null) {
- oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
- struct.credentials.write(oprot);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class replicateKeyValues_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public replicateKeyValues_argsTupleScheme getScheme() {
- return new replicateKeyValues_argsTupleScheme();
- }
- }
-
- private static class replicateKeyValues_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<replicateKeyValues_args> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, replicateKeyValues_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetRemoteTableId()) {
- optionals.set(0);
- }
- if (struct.isSetData()) {
- optionals.set(1);
- }
- if (struct.isSetCredentials()) {
- optionals.set(2);
- }
- oprot.writeBitSet(optionals, 3);
- if (struct.isSetRemoteTableId()) {
- oprot.writeString(struct.remoteTableId);
- }
- if (struct.isSetData()) {
- struct.data.write(oprot);
- }
- if (struct.isSetCredentials()) {
- struct.credentials.write(oprot);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, replicateKeyValues_args struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(3);
- if (incoming.get(0)) {
- struct.remoteTableId = iprot.readString();
- struct.setRemoteTableIdIsSet(true);
- }
- if (incoming.get(1)) {
- struct.data = new KeyValues();
- struct.data.read(iprot);
- struct.setDataIsSet(true);
- }
- if (incoming.get(2)) {
- struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
- struct.credentials.read(iprot);
- struct.setCredentialsIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- }
-
- @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
- public static class replicateKeyValues_result implements org.apache.thrift.TBase<replicateKeyValues_result, replicateKeyValues_result._Fields>, java.io.Serializable, Cloneable, Comparable<replicateKeyValues_result> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("replicateKeyValues_result");
-
- private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0);
- private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new replicateKeyValues_resultStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new replicateKeyValues_resultTupleSchemeFactory();
-
- public long success; // required
- public @org.apache.thrift.annotation.Nullable RemoteReplicationException e; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- SUCCESS((short)0, "success"),
- E((short)1, "e");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 0: // SUCCESS
- return SUCCESS;
- case 1: // E
- return E;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- private static final int __SUCCESS_ISSET_ID = 0;
- private byte __isset_bitfield = 0;
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
- tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RemoteReplicationException.class)));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(replicateKeyValues_result.class, metaDataMap);
- }
-
- public replicateKeyValues_result() {
- }
-
- public replicateKeyValues_result(
- long success,
- RemoteReplicationException e)
- {
- this();
- this.success = success;
- setSuccessIsSet(true);
- this.e = e;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public replicateKeyValues_result(replicateKeyValues_result other) {
- __isset_bitfield = other.__isset_bitfield;
- this.success = other.success;
- if (other.isSetE()) {
- this.e = new RemoteReplicationException(other.e);
- }
- }
-
- @Override
- public replicateKeyValues_result deepCopy() {
- return new replicateKeyValues_result(this);
- }
-
- @Override
- public void clear() {
- setSuccessIsSet(false);
- this.success = 0;
- this.e = null;
- }
-
- public long getSuccess() {
- return this.success;
- }
-
- public replicateKeyValues_result setSuccess(long success) {
- this.success = success;
- setSuccessIsSet(true);
- return this;
- }
-
- public void unsetSuccess() {
- __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
- }
-
- /** Returns true if field success is set (has been assigned a value) and false otherwise */
- public boolean isSetSuccess() {
- return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
- }
-
- public void setSuccessIsSet(boolean value) {
- __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
- }
-
- @org.apache.thrift.annotation.Nullable
- public RemoteReplicationException getE() {
- return this.e;
- }
-
- public replicateKeyValues_result setE(@org.apache.thrift.annotation.Nullable RemoteReplicationException e) {
- this.e = e;
- return this;
- }
-
- public void unsetE() {
- this.e = null;
- }
-
- /** Returns true if field e is set (has been assigned a value) and false otherwise */
- public boolean isSetE() {
- return this.e != null;
- }
-
- public void setEIsSet(boolean value) {
- if (!value) {
- this.e = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case SUCCESS:
- if (value == null) {
- unsetSuccess();
- } else {
- setSuccess((java.lang.Long)value);
- }
- break;
-
- case E:
- if (value == null) {
- unsetE();
- } else {
- setE((RemoteReplicationException)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case SUCCESS:
- return getSuccess();
-
- case E:
- return getE();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case SUCCESS:
- return isSetSuccess();
- case E:
- return isSetE();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof replicateKeyValues_result)
- return this.equals((replicateKeyValues_result)that);
- return false;
- }
-
- public boolean equals(replicateKeyValues_result that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_success = true;
- boolean that_present_success = true;
- if (this_present_success || that_present_success) {
- if (!(this_present_success && that_present_success))
- return false;
- if (this.success != that.success)
- return false;
- }
-
- boolean this_present_e = true && this.isSetE();
- boolean that_present_e = true && that.isSetE();
- if (this_present_e || that_present_e) {
- if (!(this_present_e && that_present_e))
- return false;
- if (!this.e.equals(that.e))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(success);
-
- hashCode = hashCode * 8191 + ((isSetE()) ? 131071 : 524287);
- if (isSetE())
- hashCode = hashCode * 8191 + e.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(replicateKeyValues_result other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetSuccess()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- lastComparison = java.lang.Boolean.compare(isSetE(), other.isSetE());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetE()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("replicateKeyValues_result(");
- boolean first = true;
-
- sb.append("success:");
- sb.append(this.success);
- first = false;
- if (!first) sb.append(", ");
- sb.append("e:");
- if (this.e == null) {
- sb.append("null");
- } else {
- sb.append(this.e);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
- __isset_bitfield = 0;
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class replicateKeyValues_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public replicateKeyValues_resultStandardScheme getScheme() {
- return new replicateKeyValues_resultStandardScheme();
- }
- }
-
- private static class replicateKeyValues_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<replicateKeyValues_result> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, replicateKeyValues_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 0: // SUCCESS
- if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
- struct.success = iprot.readI64();
- struct.setSuccessIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- case 1: // E
- if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
- struct.e = new RemoteReplicationException();
- struct.e.read(iprot);
- struct.setEIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, replicateKeyValues_result struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.isSetSuccess()) {
- oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
- oprot.writeI64(struct.success);
- oprot.writeFieldEnd();
- }
- if (struct.e != null) {
- oprot.writeFieldBegin(E_FIELD_DESC);
- struct.e.write(oprot);
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class replicateKeyValues_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public replicateKeyValues_resultTupleScheme getScheme() {
- return new replicateKeyValues_resultTupleScheme();
- }
- }
-
- private static class replicateKeyValues_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<replicateKeyValues_result> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, replicateKeyValues_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetSuccess()) {
- optionals.set(0);
- }
- if (struct.isSetE()) {
- optionals.set(1);
- }
- oprot.writeBitSet(optionals, 2);
- if (struct.isSetSuccess()) {
- oprot.writeI64(struct.success);
- }
- if (struct.isSetE()) {
- struct.e.write(oprot);
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, replicateKeyValues_result struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(2);
- if (incoming.get(0)) {
- struct.success = iprot.readI64();
- struct.setSuccessIsSet(true);
- }
- if (incoming.get(1)) {
- struct.e = new RemoteReplicationException();
- struct.e.read(iprot);
- struct.setEIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- }
-
- private static void unusedMethod() {}
-}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/WalEdits.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/WalEdits.java
deleted file mode 100644
index 48df668..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/replication/thrift/WalEdits.java
+++ /dev/null
@@ -1,456 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.17.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- * @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class WalEdits implements org.apache.thrift.TBase<WalEdits, WalEdits._Fields>, java.io.Serializable, Cloneable, Comparable<WalEdits> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WalEdits");
-
- private static final org.apache.thrift.protocol.TField EDITS_FIELD_DESC = new org.apache.thrift.protocol.TField("edits", org.apache.thrift.protocol.TType.LIST, (short)1);
-
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new WalEditsStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new WalEditsTupleSchemeFactory();
-
- public @org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> edits; // required
-
- /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- public enum _Fields implements org.apache.thrift.TFieldIdEnum {
- EDITS((short)1, "edits");
-
- private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
- static {
- for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
- byName.put(field.getFieldName(), field);
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByThriftId(int fieldId) {
- switch(fieldId) {
- case 1: // EDITS
- return EDITS;
- default:
- return null;
- }
- }
-
- /**
- * Find the _Fields constant that matches fieldId, throwing an exception
- * if it is not found.
- */
- public static _Fields findByThriftIdOrThrow(int fieldId) {
- _Fields fields = findByThriftId(fieldId);
- if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
- return fields;
- }
-
- /**
- * Find the _Fields constant that matches name, or null if its not found.
- */
- @org.apache.thrift.annotation.Nullable
- public static _Fields findByName(java.lang.String name) {
- return byName.get(name);
- }
-
- private final short _thriftId;
- private final java.lang.String _fieldName;
-
- _Fields(short thriftId, java.lang.String fieldName) {
- _thriftId = thriftId;
- _fieldName = fieldName;
- }
-
- @Override
- public short getThriftFieldId() {
- return _thriftId;
- }
-
- @Override
- public java.lang.String getFieldName() {
- return _fieldName;
- }
- }
-
- // isset id assignments
- public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
- static {
- java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
- tmpMap.put(_Fields.EDITS, new org.apache.thrift.meta_data.FieldMetaData("edits", org.apache.thrift.TFieldRequirementType.DEFAULT,
- new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
- new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))));
- metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WalEdits.class, metaDataMap);
- }
-
- public WalEdits() {
- }
-
- public WalEdits(
- java.util.List<java.nio.ByteBuffer> edits)
- {
- this();
- this.edits = edits;
- }
-
- /**
- * Performs a deep copy on <i>other</i>.
- */
- public WalEdits(WalEdits other) {
- if (other.isSetEdits()) {
- java.util.List<java.nio.ByteBuffer> __this__edits = new java.util.ArrayList<java.nio.ByteBuffer>(other.edits);
- this.edits = __this__edits;
- }
- }
-
- @Override
- public WalEdits deepCopy() {
- return new WalEdits(this);
- }
-
- @Override
- public void clear() {
- this.edits = null;
- }
-
- public int getEditsSize() {
- return (this.edits == null) ? 0 : this.edits.size();
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.util.Iterator<java.nio.ByteBuffer> getEditsIterator() {
- return (this.edits == null) ? null : this.edits.iterator();
- }
-
- public void addToEdits(java.nio.ByteBuffer elem) {
- if (this.edits == null) {
- this.edits = new java.util.ArrayList<java.nio.ByteBuffer>();
- }
- this.edits.add(elem);
- }
-
- @org.apache.thrift.annotation.Nullable
- public java.util.List<java.nio.ByteBuffer> getEdits() {
- return this.edits;
- }
-
- public WalEdits setEdits(@org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> edits) {
- this.edits = edits;
- return this;
- }
-
- public void unsetEdits() {
- this.edits = null;
- }
-
- /** Returns true if field edits is set (has been assigned a value) and false otherwise */
- public boolean isSetEdits() {
- return this.edits != null;
- }
-
- public void setEditsIsSet(boolean value) {
- if (!value) {
- this.edits = null;
- }
- }
-
- @Override
- public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
- switch (field) {
- case EDITS:
- if (value == null) {
- unsetEdits();
- } else {
- setEdits((java.util.List<java.nio.ByteBuffer>)value);
- }
- break;
-
- }
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public java.lang.Object getFieldValue(_Fields field) {
- switch (field) {
- case EDITS:
- return getEdits();
-
- }
- throw new java.lang.IllegalStateException();
- }
-
- /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
- @Override
- public boolean isSet(_Fields field) {
- if (field == null) {
- throw new java.lang.IllegalArgumentException();
- }
-
- switch (field) {
- case EDITS:
- return isSetEdits();
- }
- throw new java.lang.IllegalStateException();
- }
-
- @Override
- public boolean equals(java.lang.Object that) {
- if (that instanceof WalEdits)
- return this.equals((WalEdits)that);
- return false;
- }
-
- public boolean equals(WalEdits that) {
- if (that == null)
- return false;
- if (this == that)
- return true;
-
- boolean this_present_edits = true && this.isSetEdits();
- boolean that_present_edits = true && that.isSetEdits();
- if (this_present_edits || that_present_edits) {
- if (!(this_present_edits && that_present_edits))
- return false;
- if (!this.edits.equals(that.edits))
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
- int hashCode = 1;
-
- hashCode = hashCode * 8191 + ((isSetEdits()) ? 131071 : 524287);
- if (isSetEdits())
- hashCode = hashCode * 8191 + edits.hashCode();
-
- return hashCode;
- }
-
- @Override
- public int compareTo(WalEdits other) {
- if (!getClass().equals(other.getClass())) {
- return getClass().getName().compareTo(other.getClass().getName());
- }
-
- int lastComparison = 0;
-
- lastComparison = java.lang.Boolean.compare(isSetEdits(), other.isSetEdits());
- if (lastComparison != 0) {
- return lastComparison;
- }
- if (isSetEdits()) {
- lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.edits, other.edits);
- if (lastComparison != 0) {
- return lastComparison;
- }
- }
- return 0;
- }
-
- @org.apache.thrift.annotation.Nullable
- @Override
- public _Fields fieldForId(int fieldId) {
- return _Fields.findByThriftId(fieldId);
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
- scheme(iprot).read(iprot, this);
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
- scheme(oprot).write(oprot, this);
- }
-
- @Override
- public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("WalEdits(");
- boolean first = true;
-
- sb.append("edits:");
- if (this.edits == null) {
- sb.append("null");
- } else {
- org.apache.thrift.TBaseHelper.toString(this.edits, sb);
- }
- first = false;
- sb.append(")");
- return sb.toString();
- }
-
- public void validate() throws org.apache.thrift.TException {
- // check for required fields
- // check for sub-struct validity
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
- try {
- write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
- try {
- read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
- } catch (org.apache.thrift.TException te) {
- throw new java.io.IOException(te);
- }
- }
-
- private static class WalEditsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public WalEditsStandardScheme getScheme() {
- return new WalEditsStandardScheme();
- }
- }
-
- private static class WalEditsStandardScheme extends org.apache.thrift.scheme.StandardScheme<WalEdits> {
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, WalEdits struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TField schemeField;
- iprot.readStructBegin();
- while (true)
- {
- schemeField = iprot.readFieldBegin();
- if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
- break;
- }
- switch (schemeField.id) {
- case 1: // EDITS
- if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
- {
- org.apache.thrift.protocol.TList _list0 = iprot.readListBegin();
- struct.edits = new java.util.ArrayList<java.nio.ByteBuffer>(_list0.size);
- @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem1;
- for (int _i2 = 0; _i2 < _list0.size; ++_i2)
- {
- _elem1 = iprot.readBinary();
- struct.edits.add(_elem1);
- }
- iprot.readListEnd();
- }
- struct.setEditsIsSet(true);
- } else {
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- break;
- default:
- org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
- }
- iprot.readFieldEnd();
- }
- iprot.readStructEnd();
-
- // check for required fields of primitive type, which can't be checked in the validate method
- struct.validate();
- }
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, WalEdits struct) throws org.apache.thrift.TException {
- struct.validate();
-
- oprot.writeStructBegin(STRUCT_DESC);
- if (struct.edits != null) {
- oprot.writeFieldBegin(EDITS_FIELD_DESC);
- {
- oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.edits.size()));
- for (java.nio.ByteBuffer _iter3 : struct.edits)
- {
- oprot.writeBinary(_iter3);
- }
- oprot.writeListEnd();
- }
- oprot.writeFieldEnd();
- }
- oprot.writeFieldStop();
- oprot.writeStructEnd();
- }
-
- }
-
- private static class WalEditsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
- @Override
- public WalEditsTupleScheme getScheme() {
- return new WalEditsTupleScheme();
- }
- }
-
- private static class WalEditsTupleScheme extends org.apache.thrift.scheme.TupleScheme<WalEdits> {
-
- @Override
- public void write(org.apache.thrift.protocol.TProtocol prot, WalEdits struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet optionals = new java.util.BitSet();
- if (struct.isSetEdits()) {
- optionals.set(0);
- }
- oprot.writeBitSet(optionals, 1);
- if (struct.isSetEdits()) {
- {
- oprot.writeI32(struct.edits.size());
- for (java.nio.ByteBuffer _iter4 : struct.edits)
- {
- oprot.writeBinary(_iter4);
- }
- }
- }
- }
-
- @Override
- public void read(org.apache.thrift.protocol.TProtocol prot, WalEdits struct) throws org.apache.thrift.TException {
- org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
- java.util.BitSet incoming = iprot.readBitSet(1);
- if (incoming.get(0)) {
- {
- org.apache.thrift.protocol.TList _list5 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
- struct.edits = new java.util.ArrayList<java.nio.ByteBuffer>(_list5.size);
- @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem6;
- for (int _i7 = 0; _i7 < _list5.size; ++_i7)
- {
- _elem6 = iprot.readBinary();
- struct.edits.add(_elem6);
- }
- }
- struct.setEditsIsSet(true);
- }
- }
- }
-
- private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
- return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
- }
- private static void unusedMethod() {}
-}
-
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TUnloadTabletGoal.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TUnloadTabletGoal.java
similarity index 96%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TUnloadTabletGoal.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TUnloadTabletGoal.java
index 4c65ad9..747f4ee 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TUnloadTabletGoal.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TUnloadTabletGoal.java
@@ -22,7 +22,7 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.tabletserver.thrift;
+package org.apache.accumulo.core.tablet.thrift;
public enum TUnloadTabletGoal implements org.apache.thrift.TEnum {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TabletManagementClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TabletManagementClientService.java
new file mode 100644
index 0000000..6b83212
--- /dev/null
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tablet/thrift/TabletManagementClientService.java
@@ -0,0 +1,4267 @@
+/*
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.17.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ * @generated
+ */
+package org.apache.accumulo.core.tablet.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class TabletManagementClientService {
+
+ public interface Iface {
+
+ public void loadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException;
+
+ public void unloadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime) throws org.apache.thrift.TException;
+
+ public void splitTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, org.apache.thrift.TException;
+
+ public void flushTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException;
+
+ }
+
+ public interface AsyncIface {
+
+ public void loadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+ public void unloadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+ public void splitTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+ public void flushTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+ }
+
+ public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+ public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
+ public Factory() {}
+ @Override
+ public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
+ return new Client(prot);
+ }
+ @Override
+ public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+ return new Client(iprot, oprot);
+ }
+ }
+
+ public Client(org.apache.thrift.protocol.TProtocol prot)
+ {
+ super(prot, prot);
+ }
+
+ public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+ super(iprot, oprot);
+ }
+
+ @Override
+ public void loadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+ {
+ send_loadTablet(tinfo, credentials, lock, extent);
+ }
+
+ public void send_loadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+ {
+ loadTablet_args args = new loadTablet_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setLock(lock);
+ args.setExtent(extent);
+ sendBaseOneway("loadTablet", args);
+ }
+
+ @Override
+ public void unloadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime) throws org.apache.thrift.TException
+ {
+ send_unloadTablet(tinfo, credentials, lock, extent, goal, requestTime);
+ }
+
+ public void send_unloadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime) throws org.apache.thrift.TException
+ {
+ unloadTablet_args args = new unloadTablet_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setLock(lock);
+ args.setExtent(extent);
+ args.setGoal(goal);
+ args.setRequestTime(requestTime);
+ sendBaseOneway("unloadTablet", args);
+ }
+
+ @Override
+ public void splitTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, org.apache.thrift.TException
+ {
+ send_splitTablet(tinfo, credentials, extent, splitPoint);
+ recv_splitTablet();
+ }
+
+ public void send_splitTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint) throws org.apache.thrift.TException
+ {
+ splitTablet_args args = new splitTablet_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setExtent(extent);
+ args.setSplitPoint(splitPoint);
+ sendBase("splitTablet", args);
+ }
+
+ public void recv_splitTablet() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, org.apache.thrift.TException
+ {
+ splitTablet_result result = new splitTablet_result();
+ receiveBase(result, "splitTablet");
+ if (result.sec != null) {
+ throw result.sec;
+ }
+ if (result.nste != null) {
+ throw result.nste;
+ }
+ return;
+ }
+
+ @Override
+ public void flushTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+ {
+ send_flushTablet(tinfo, credentials, lock, extent);
+ }
+
+ public void send_flushTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) throws org.apache.thrift.TException
+ {
+ flushTablet_args args = new flushTablet_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setLock(lock);
+ args.setExtent(extent);
+ sendBaseOneway("flushTablet", args);
+ }
+
+ }
+ public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+ public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
+ private org.apache.thrift.async.TAsyncClientManager clientManager;
+ private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+ public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+ this.clientManager = clientManager;
+ this.protocolFactory = protocolFactory;
+ }
+ @Override
+ public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
+ return new AsyncClient(protocolFactory, clientManager, transport);
+ }
+ }
+
+ public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
+ super(protocolFactory, clientManager, transport);
+ }
+
+ @Override
+ public void loadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ loadTablet_call method_call = new loadTablet_call(tinfo, credentials, lock, extent, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class loadTablet_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+ private java.lang.String lock;
+ private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+ public loadTablet_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, true);
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.lock = lock;
+ this.extent = extent;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("loadTablet", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+ loadTablet_args args = new loadTablet_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setLock(lock);
+ args.setExtent(extent);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public Void getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return null;
+ }
+ }
+
+ @Override
+ public void unloadTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ unloadTablet_call method_call = new unloadTablet_call(tinfo, credentials, lock, extent, goal, requestTime, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class unloadTablet_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+ private java.lang.String lock;
+ private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+ private TUnloadTabletGoal goal;
+ private long requestTime;
+ public unloadTablet_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, TUnloadTabletGoal goal, long requestTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, true);
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.lock = lock;
+ this.extent = extent;
+ this.goal = goal;
+ this.requestTime = requestTime;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("unloadTablet", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+ unloadTablet_args args = new unloadTablet_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setLock(lock);
+ args.setExtent(extent);
+ args.setGoal(goal);
+ args.setRequestTime(requestTime);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public Void getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return null;
+ }
+ }
+
+ @Override
+ public void splitTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ splitTablet_call method_call = new splitTablet_call(tinfo, credentials, extent, splitPoint, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class splitTablet_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+ private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+ private java.nio.ByteBuffer splitPoint;
+ public splitTablet_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, java.nio.ByteBuffer splitPoint, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.extent = extent;
+ this.splitPoint = splitPoint;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("splitTablet", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ splitTablet_args args = new splitTablet_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setExtent(extent);
+ args.setSplitPoint(splitPoint);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ (new Client(prot)).recv_splitTablet();
+ return null;
+ }
+ }
+
+ @Override
+ public void flushTablet(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ flushTablet_call method_call = new flushTablet_call(tinfo, credentials, lock, extent, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class flushTablet_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+ private java.lang.String lock;
+ private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+ public flushTablet_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, true);
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.lock = lock;
+ this.extent = extent;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("flushTablet", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+ flushTablet_args args = new flushTablet_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setLock(lock);
+ args.setExtent(extent);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public Void getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return null;
+ }
+ }
+
+ }
+
+ public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+ private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
+ public Processor(I iface) {
+ super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+ }
+
+ protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
+ super(iface, getProcessMap(processMap));
+ }
+
+ private static <I extends Iface> java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
+ processMap.put("loadTablet", new loadTablet());
+ processMap.put("unloadTablet", new unloadTablet());
+ processMap.put("splitTablet", new splitTablet());
+ processMap.put("flushTablet", new flushTablet());
+ return processMap;
+ }
+
+ public static class loadTablet<I extends Iface> extends org.apache.thrift.ProcessFunction<I, loadTablet_args> {
+ public loadTablet() {
+ super("loadTablet");
+ }
+
+ @Override
+ public loadTablet_args getEmptyArgsInstance() {
+ return new loadTablet_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public org.apache.thrift.TBase getResult(I iface, loadTablet_args args) throws org.apache.thrift.TException {
+ iface.loadTablet(args.tinfo, args.credentials, args.lock, args.extent);
+ return null;
+ }
+ }
+
+ public static class unloadTablet<I extends Iface> extends org.apache.thrift.ProcessFunction<I, unloadTablet_args> {
+ public unloadTablet() {
+ super("unloadTablet");
+ }
+
+ @Override
+ public unloadTablet_args getEmptyArgsInstance() {
+ return new unloadTablet_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public org.apache.thrift.TBase getResult(I iface, unloadTablet_args args) throws org.apache.thrift.TException {
+ iface.unloadTablet(args.tinfo, args.credentials, args.lock, args.extent, args.goal, args.requestTime);
+ return null;
+ }
+ }
+
+ public static class splitTablet<I extends Iface> extends org.apache.thrift.ProcessFunction<I, splitTablet_args> {
+ public splitTablet() {
+ super("splitTablet");
+ }
+
+ @Override
+ public splitTablet_args getEmptyArgsInstance() {
+ return new splitTablet_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public splitTablet_result getResult(I iface, splitTablet_args args) throws org.apache.thrift.TException {
+ splitTablet_result result = new splitTablet_result();
+ try {
+ iface.splitTablet(args.tinfo, args.credentials, args.extent, args.splitPoint);
+ } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+ result.sec = sec;
+ } catch (org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste) {
+ result.nste = nste;
+ }
+ return result;
+ }
+ }
+
+ public static class flushTablet<I extends Iface> extends org.apache.thrift.ProcessFunction<I, flushTablet_args> {
+ public flushTablet() {
+ super("flushTablet");
+ }
+
+ @Override
+ public flushTablet_args getEmptyArgsInstance() {
+ return new flushTablet_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public org.apache.thrift.TBase getResult(I iface, flushTablet_args args) throws org.apache.thrift.TException {
+ iface.flushTablet(args.tinfo, args.credentials, args.lock, args.extent);
+ return null;
+ }
+ }
+
+ }
+
+ public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
+ private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
+ public AsyncProcessor(I iface) {
+ super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
+ }
+
+ protected AsyncProcessor(I iface, java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>> processMap) {
+ super(iface, getProcessMap(processMap));
+ }
+
+ private static <I extends AsyncIface> java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>> processMap) {
+ processMap.put("loadTablet", new loadTablet());
+ processMap.put("unloadTablet", new unloadTablet());
+ processMap.put("splitTablet", new splitTablet());
+ processMap.put("flushTablet", new flushTablet());
+ return processMap;
+ }
+
+ public static class loadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadTablet_args, Void> {
+ public loadTablet() {
+ super("loadTablet");
+ }
+
+ @Override
+ public loadTablet_args getEmptyArgsInstance() {
+ return new loadTablet_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<Void>() {
+ @Override
+ public void onComplete(Void o) {
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ } else {
+ _LOGGER.error("Exception inside oneway handler", e);
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ public void start(I iface, loadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ iface.loadTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
+ }
+ }
+
+ public static class unloadTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, unloadTablet_args, Void> {
+ public unloadTablet() {
+ super("unloadTablet");
+ }
+
+ @Override
+ public unloadTablet_args getEmptyArgsInstance() {
+ return new unloadTablet_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<Void>() {
+ @Override
+ public void onComplete(Void o) {
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ } else {
+ _LOGGER.error("Exception inside oneway handler", e);
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ public void start(I iface, unloadTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ iface.unloadTablet(args.tinfo, args.credentials, args.lock, args.extent, args.goal, args.requestTime,resultHandler);
+ }
+ }
+
+ public static class splitTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, splitTablet_args, Void> {
+ public splitTablet() {
+ super("splitTablet");
+ }
+
+ @Override
+ public splitTablet_args getEmptyArgsInstance() {
+ return new splitTablet_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<Void>() {
+ @Override
+ public void onComplete(Void o) {
+ splitTablet_result result = new splitTablet_result();
+ try {
+ fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+ } catch (org.apache.thrift.transport.TTransportException e) {
+ _LOGGER.error("TTransportException writing to internal frame buffer", e);
+ fb.close();
+ } catch (java.lang.Exception e) {
+ _LOGGER.error("Exception writing to internal frame buffer", e);
+ onError(e);
+ }
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+ org.apache.thrift.TSerializable msg;
+ splitTablet_result result = new splitTablet_result();
+ if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+ result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+ result.setSecIsSet(true);
+ msg = result;
+ } else if (e instanceof org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException) {
+ result.nste = (org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException) e;
+ result.setNsteIsSet(true);
+ msg = result;
+ } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ return;
+ } else if (e instanceof org.apache.thrift.TApplicationException) {
+ _LOGGER.error("TApplicationException inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = (org.apache.thrift.TApplicationException)e;
+ } else {
+ _LOGGER.error("Exception inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+ }
+ try {
+ fcall.sendResponse(fb,msg,msgType,seqid);
+ } catch (java.lang.Exception ex) {
+ _LOGGER.error("Exception writing to internal frame buffer", ex);
+ fb.close();
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ public void start(I iface, splitTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ iface.splitTablet(args.tinfo, args.credentials, args.extent, args.splitPoint,resultHandler);
+ }
+ }
+
+ public static class flushTablet<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flushTablet_args, Void> {
+ public flushTablet() {
+ super("flushTablet");
+ }
+
+ @Override
+ public flushTablet_args getEmptyArgsInstance() {
+ return new flushTablet_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<Void>() {
+ @Override
+ public void onComplete(Void o) {
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ } else {
+ _LOGGER.error("Exception inside oneway handler", e);
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ public void start(I iface, flushTablet_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ iface.flushTablet(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
+ }
+ }
+
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class loadTablet_args implements org.apache.thrift.TBase<loadTablet_args, loadTablet_args._Fields>, java.io.Serializable, Cloneable, Comparable<loadTablet_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("loadTablet_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+ private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField LOCK_FIELD_DESC = new org.apache.thrift.protocol.TField("lock", org.apache.thrift.protocol.TType.STRING, (short)4);
+ private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new loadTablet_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new loadTablet_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+ public @org.apache.thrift.annotation.Nullable java.lang.String lock; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)5, "tinfo"),
+ CREDENTIALS((short)1, "credentials"),
+ LOCK((short)4, "lock"),
+ EXTENT((short)2, "extent");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 5: // TINFO
+ return TINFO;
+ case 1: // CREDENTIALS
+ return CREDENTIALS;
+ case 4: // LOCK
+ return LOCK;
+ case 2: // EXTENT
+ return EXTENT;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+ tmpMap.put(_Fields.LOCK, new org.apache.thrift.meta_data.FieldMetaData("lock", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+ tmpMap.put(_Fields.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(loadTablet_args.class, metaDataMap);
+ }
+
+ public loadTablet_args() {
+ }
+
+ public loadTablet_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+ java.lang.String lock,
+ org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.lock = lock;
+ this.extent = extent;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public loadTablet_args(loadTablet_args other) {
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ if (other.isSetCredentials()) {
+ this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+ }
+ if (other.isSetLock()) {
+ this.lock = other.lock;
+ }
+ if (other.isSetExtent()) {
+ this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+ }
+ }
+
+ @Override
+ public loadTablet_args deepCopy() {
+ return new loadTablet_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ this.credentials = null;
+ this.lock = null;
+ this.extent = null;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public loadTablet_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+ return this.credentials;
+ }
+
+ public loadTablet_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+ this.credentials = credentials;
+ return this;
+ }
+
+ public void unsetCredentials() {
+ this.credentials = null;
+ }
+
+ /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+ public boolean isSetCredentials() {
+ return this.credentials != null;
+ }
+
+ public void setCredentialsIsSet(boolean value) {
+ if (!value) {
+ this.credentials = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.lang.String getLock() {
+ return this.lock;
+ }
+
+ public loadTablet_args setLock(@org.apache.thrift.annotation.Nullable java.lang.String lock) {
+ this.lock = lock;
+ return this;
+ }
+
+ public void unsetLock() {
+ this.lock = null;
+ }
+
+ /** Returns true if field lock is set (has been assigned a value) and false otherwise */
+ public boolean isSetLock() {
+ return this.lock != null;
+ }
+
+ public void setLockIsSet(boolean value) {
+ if (!value) {
+ this.lock = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+ return this.extent;
+ }
+
+ public loadTablet_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+ this.extent = extent;
+ return this;
+ }
+
+ public void unsetExtent() {
+ this.extent = null;
+ }
+
+ /** Returns true if field extent is set (has been assigned a value) and false otherwise */
+ public boolean isSetExtent() {
+ return this.extent != null;
+ }
+
+ public void setExtentIsSet(boolean value) {
+ if (!value) {
+ this.extent = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case CREDENTIALS:
+ if (value == null) {
+ unsetCredentials();
+ } else {
+ setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+ }
+ break;
+
+ case LOCK:
+ if (value == null) {
+ unsetLock();
+ } else {
+ setLock((java.lang.String)value);
+ }
+ break;
+
+ case EXTENT:
+ if (value == null) {
+ unsetExtent();
+ } else {
+ setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case CREDENTIALS:
+ return getCredentials();
+
+ case LOCK:
+ return getLock();
+
+ case EXTENT:
+ return getExtent();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case CREDENTIALS:
+ return isSetCredentials();
+ case LOCK:
+ return isSetLock();
+ case EXTENT:
+ return isSetExtent();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof loadTablet_args)
+ return this.equals((loadTablet_args)that);
+ return false;
+ }
+
+ public boolean equals(loadTablet_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_credentials = true && this.isSetCredentials();
+ boolean that_present_credentials = true && that.isSetCredentials();
+ if (this_present_credentials || that_present_credentials) {
+ if (!(this_present_credentials && that_present_credentials))
+ return false;
+ if (!this.credentials.equals(that.credentials))
+ return false;
+ }
+
+ boolean this_present_lock = true && this.isSetLock();
+ boolean that_present_lock = true && that.isSetLock();
+ if (this_present_lock || that_present_lock) {
+ if (!(this_present_lock && that_present_lock))
+ return false;
+ if (!this.lock.equals(that.lock))
+ return false;
+ }
+
+ boolean this_present_extent = true && this.isSetExtent();
+ boolean that_present_extent = true && that.isSetExtent();
+ if (this_present_extent || that_present_extent) {
+ if (!(this_present_extent && that_present_extent))
+ return false;
+ if (!this.extent.equals(that.extent))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+ if (isSetCredentials())
+ hashCode = hashCode * 8191 + credentials.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetLock()) ? 131071 : 524287);
+ if (isSetLock())
+ hashCode = hashCode * 8191 + lock.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+ if (isSetExtent())
+ hashCode = hashCode * 8191 + extent.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(loadTablet_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCredentials()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetLock(), other.isSetLock());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetLock()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lock, other.lock);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetExtent(), other.isSetExtent());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetExtent()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("loadTablet_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("credentials:");
+ if (this.credentials == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.credentials);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("lock:");
+ if (this.lock == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.lock);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("extent:");
+ if (this.extent == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.extent);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ if (credentials != null) {
+ credentials.validate();
+ }
+ if (extent != null) {
+ extent.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class loadTablet_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public loadTablet_argsStandardScheme getScheme() {
+ return new loadTablet_argsStandardScheme();
+ }
+ }
+
+ private static class loadTablet_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<loadTablet_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, loadTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 5: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 1: // CREDENTIALS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 4: // LOCK
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.lock = iprot.readString();
+ struct.setLockIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // EXTENT
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ struct.extent.read(iprot);
+ struct.setExtentIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, loadTablet_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.credentials != null) {
+ oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+ struct.credentials.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.extent != null) {
+ oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+ struct.extent.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.lock != null) {
+ oprot.writeFieldBegin(LOCK_FIELD_DESC);
+ oprot.writeString(struct.lock);
+ oprot.writeFieldEnd();
+ }
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class loadTablet_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public loadTablet_argsTupleScheme getScheme() {
+ return new loadTablet_argsTupleScheme();
+ }
+ }
+
+ private static class loadTablet_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<loadTablet_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, loadTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetCredentials()) {
+ optionals.set(1);
+ }
+ if (struct.isSetLock()) {
+ optionals.set(2);
+ }
+ if (struct.isSetExtent()) {
+ optionals.set(3);
+ }
+ oprot.writeBitSet(optionals, 4);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetCredentials()) {
+ struct.credentials.write(oprot);
+ }
+ if (struct.isSetLock()) {
+ oprot.writeString(struct.lock);
+ }
+ if (struct.isSetExtent()) {
+ struct.extent.write(oprot);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, loadTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(4);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ }
+ if (incoming.get(2)) {
+ struct.lock = iprot.readString();
+ struct.setLockIsSet(true);
+ }
+ if (incoming.get(3)) {
+ struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ struct.extent.read(iprot);
+ struct.setExtentIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class unloadTablet_args implements org.apache.thrift.TBase<unloadTablet_args, unloadTablet_args._Fields>, java.io.Serializable, Cloneable, Comparable<unloadTablet_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("unloadTablet_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+ private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField LOCK_FIELD_DESC = new org.apache.thrift.protocol.TField("lock", org.apache.thrift.protocol.TType.STRING, (short)4);
+ private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+ private static final org.apache.thrift.protocol.TField GOAL_FIELD_DESC = new org.apache.thrift.protocol.TField("goal", org.apache.thrift.protocol.TType.I32, (short)6);
+ private static final org.apache.thrift.protocol.TField REQUEST_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("requestTime", org.apache.thrift.protocol.TType.I64, (short)7);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new unloadTablet_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new unloadTablet_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+ public @org.apache.thrift.annotation.Nullable java.lang.String lock; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
+ /**
+ *
+ * @see TUnloadTabletGoal
+ */
+ public @org.apache.thrift.annotation.Nullable TUnloadTabletGoal goal; // required
+ public long requestTime; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)5, "tinfo"),
+ CREDENTIALS((short)1, "credentials"),
+ LOCK((short)4, "lock"),
+ EXTENT((short)2, "extent"),
+ /**
+ *
+ * @see TUnloadTabletGoal
+ */
+ GOAL((short)6, "goal"),
+ REQUEST_TIME((short)7, "requestTime");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 5: // TINFO
+ return TINFO;
+ case 1: // CREDENTIALS
+ return CREDENTIALS;
+ case 4: // LOCK
+ return LOCK;
+ case 2: // EXTENT
+ return EXTENT;
+ case 6: // GOAL
+ return GOAL;
+ case 7: // REQUEST_TIME
+ return REQUEST_TIME;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __REQUESTTIME_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+ tmpMap.put(_Fields.LOCK, new org.apache.thrift.meta_data.FieldMetaData("lock", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+ tmpMap.put(_Fields.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+ tmpMap.put(_Fields.GOAL, new org.apache.thrift.meta_data.FieldMetaData("goal", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TUnloadTabletGoal.class)));
+ tmpMap.put(_Fields.REQUEST_TIME, new org.apache.thrift.meta_data.FieldMetaData("requestTime", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(unloadTablet_args.class, metaDataMap);
+ }
+
+ public unloadTablet_args() {
+ }
+
+ public unloadTablet_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+ java.lang.String lock,
+ org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
+ TUnloadTabletGoal goal,
+ long requestTime)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.lock = lock;
+ this.extent = extent;
+ this.goal = goal;
+ this.requestTime = requestTime;
+ setRequestTimeIsSet(true);
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public unloadTablet_args(unloadTablet_args other) {
+ __isset_bitfield = other.__isset_bitfield;
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ if (other.isSetCredentials()) {
+ this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+ }
+ if (other.isSetLock()) {
+ this.lock = other.lock;
+ }
+ if (other.isSetExtent()) {
+ this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+ }
+ if (other.isSetGoal()) {
+ this.goal = other.goal;
+ }
+ this.requestTime = other.requestTime;
+ }
+
+ @Override
+ public unloadTablet_args deepCopy() {
+ return new unloadTablet_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ this.credentials = null;
+ this.lock = null;
+ this.extent = null;
+ this.goal = null;
+ setRequestTimeIsSet(false);
+ this.requestTime = 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public unloadTablet_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+ return this.credentials;
+ }
+
+ public unloadTablet_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+ this.credentials = credentials;
+ return this;
+ }
+
+ public void unsetCredentials() {
+ this.credentials = null;
+ }
+
+ /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+ public boolean isSetCredentials() {
+ return this.credentials != null;
+ }
+
+ public void setCredentialsIsSet(boolean value) {
+ if (!value) {
+ this.credentials = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.lang.String getLock() {
+ return this.lock;
+ }
+
+ public unloadTablet_args setLock(@org.apache.thrift.annotation.Nullable java.lang.String lock) {
+ this.lock = lock;
+ return this;
+ }
+
+ public void unsetLock() {
+ this.lock = null;
+ }
+
+ /** Returns true if field lock is set (has been assigned a value) and false otherwise */
+ public boolean isSetLock() {
+ return this.lock != null;
+ }
+
+ public void setLockIsSet(boolean value) {
+ if (!value) {
+ this.lock = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+ return this.extent;
+ }
+
+ public unloadTablet_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+ this.extent = extent;
+ return this;
+ }
+
+ public void unsetExtent() {
+ this.extent = null;
+ }
+
+ /** Returns true if field extent is set (has been assigned a value) and false otherwise */
+ public boolean isSetExtent() {
+ return this.extent != null;
+ }
+
+ public void setExtentIsSet(boolean value) {
+ if (!value) {
+ this.extent = null;
+ }
+ }
+
+ /**
+ *
+ * @see TUnloadTabletGoal
+ */
+ @org.apache.thrift.annotation.Nullable
+ public TUnloadTabletGoal getGoal() {
+ return this.goal;
+ }
+
+ /**
+ *
+ * @see TUnloadTabletGoal
+ */
+ public unloadTablet_args setGoal(@org.apache.thrift.annotation.Nullable TUnloadTabletGoal goal) {
+ this.goal = goal;
+ return this;
+ }
+
+ public void unsetGoal() {
+ this.goal = null;
+ }
+
+ /** Returns true if field goal is set (has been assigned a value) and false otherwise */
+ public boolean isSetGoal() {
+ return this.goal != null;
+ }
+
+ public void setGoalIsSet(boolean value) {
+ if (!value) {
+ this.goal = null;
+ }
+ }
+
+ public long getRequestTime() {
+ return this.requestTime;
+ }
+
+ public unloadTablet_args setRequestTime(long requestTime) {
+ this.requestTime = requestTime;
+ setRequestTimeIsSet(true);
+ return this;
+ }
+
+ public void unsetRequestTime() {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __REQUESTTIME_ISSET_ID);
+ }
+
+ /** Returns true if field requestTime is set (has been assigned a value) and false otherwise */
+ public boolean isSetRequestTime() {
+ return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __REQUESTTIME_ISSET_ID);
+ }
+
+ public void setRequestTimeIsSet(boolean value) {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __REQUESTTIME_ISSET_ID, value);
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case CREDENTIALS:
+ if (value == null) {
+ unsetCredentials();
+ } else {
+ setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+ }
+ break;
+
+ case LOCK:
+ if (value == null) {
+ unsetLock();
+ } else {
+ setLock((java.lang.String)value);
+ }
+ break;
+
+ case EXTENT:
+ if (value == null) {
+ unsetExtent();
+ } else {
+ setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+ }
+ break;
+
+ case GOAL:
+ if (value == null) {
+ unsetGoal();
+ } else {
+ setGoal((TUnloadTabletGoal)value);
+ }
+ break;
+
+ case REQUEST_TIME:
+ if (value == null) {
+ unsetRequestTime();
+ } else {
+ setRequestTime((java.lang.Long)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case CREDENTIALS:
+ return getCredentials();
+
+ case LOCK:
+ return getLock();
+
+ case EXTENT:
+ return getExtent();
+
+ case GOAL:
+ return getGoal();
+
+ case REQUEST_TIME:
+ return getRequestTime();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case CREDENTIALS:
+ return isSetCredentials();
+ case LOCK:
+ return isSetLock();
+ case EXTENT:
+ return isSetExtent();
+ case GOAL:
+ return isSetGoal();
+ case REQUEST_TIME:
+ return isSetRequestTime();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof unloadTablet_args)
+ return this.equals((unloadTablet_args)that);
+ return false;
+ }
+
+ public boolean equals(unloadTablet_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_credentials = true && this.isSetCredentials();
+ boolean that_present_credentials = true && that.isSetCredentials();
+ if (this_present_credentials || that_present_credentials) {
+ if (!(this_present_credentials && that_present_credentials))
+ return false;
+ if (!this.credentials.equals(that.credentials))
+ return false;
+ }
+
+ boolean this_present_lock = true && this.isSetLock();
+ boolean that_present_lock = true && that.isSetLock();
+ if (this_present_lock || that_present_lock) {
+ if (!(this_present_lock && that_present_lock))
+ return false;
+ if (!this.lock.equals(that.lock))
+ return false;
+ }
+
+ boolean this_present_extent = true && this.isSetExtent();
+ boolean that_present_extent = true && that.isSetExtent();
+ if (this_present_extent || that_present_extent) {
+ if (!(this_present_extent && that_present_extent))
+ return false;
+ if (!this.extent.equals(that.extent))
+ return false;
+ }
+
+ boolean this_present_goal = true && this.isSetGoal();
+ boolean that_present_goal = true && that.isSetGoal();
+ if (this_present_goal || that_present_goal) {
+ if (!(this_present_goal && that_present_goal))
+ return false;
+ if (!this.goal.equals(that.goal))
+ return false;
+ }
+
+ boolean this_present_requestTime = true;
+ boolean that_present_requestTime = true;
+ if (this_present_requestTime || that_present_requestTime) {
+ if (!(this_present_requestTime && that_present_requestTime))
+ return false;
+ if (this.requestTime != that.requestTime)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+ if (isSetCredentials())
+ hashCode = hashCode * 8191 + credentials.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetLock()) ? 131071 : 524287);
+ if (isSetLock())
+ hashCode = hashCode * 8191 + lock.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+ if (isSetExtent())
+ hashCode = hashCode * 8191 + extent.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetGoal()) ? 131071 : 524287);
+ if (isSetGoal())
+ hashCode = hashCode * 8191 + goal.getValue();
+
+ hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(requestTime);
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(unloadTablet_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCredentials()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetLock(), other.isSetLock());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetLock()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lock, other.lock);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetExtent(), other.isSetExtent());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetExtent()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetGoal(), other.isSetGoal());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetGoal()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.goal, other.goal);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetRequestTime(), other.isSetRequestTime());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetRequestTime()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requestTime, other.requestTime);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("unloadTablet_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("credentials:");
+ if (this.credentials == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.credentials);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("lock:");
+ if (this.lock == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.lock);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("extent:");
+ if (this.extent == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.extent);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("goal:");
+ if (this.goal == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.goal);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("requestTime:");
+ sb.append(this.requestTime);
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ if (credentials != null) {
+ credentials.validate();
+ }
+ if (extent != null) {
+ extent.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class unloadTablet_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public unloadTablet_argsStandardScheme getScheme() {
+ return new unloadTablet_argsStandardScheme();
+ }
+ }
+
+ private static class unloadTablet_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<unloadTablet_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, unloadTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 5: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 1: // CREDENTIALS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 4: // LOCK
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.lock = iprot.readString();
+ struct.setLockIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // EXTENT
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ struct.extent.read(iprot);
+ struct.setExtentIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 6: // GOAL
+ if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+ struct.goal = org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal.findByValue(iprot.readI32());
+ struct.setGoalIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 7: // REQUEST_TIME
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.requestTime = iprot.readI64();
+ struct.setRequestTimeIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, unloadTablet_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.credentials != null) {
+ oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+ struct.credentials.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.extent != null) {
+ oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+ struct.extent.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.lock != null) {
+ oprot.writeFieldBegin(LOCK_FIELD_DESC);
+ oprot.writeString(struct.lock);
+ oprot.writeFieldEnd();
+ }
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.goal != null) {
+ oprot.writeFieldBegin(GOAL_FIELD_DESC);
+ oprot.writeI32(struct.goal.getValue());
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldBegin(REQUEST_TIME_FIELD_DESC);
+ oprot.writeI64(struct.requestTime);
+ oprot.writeFieldEnd();
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class unloadTablet_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public unloadTablet_argsTupleScheme getScheme() {
+ return new unloadTablet_argsTupleScheme();
+ }
+ }
+
+ private static class unloadTablet_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<unloadTablet_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, unloadTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetCredentials()) {
+ optionals.set(1);
+ }
+ if (struct.isSetLock()) {
+ optionals.set(2);
+ }
+ if (struct.isSetExtent()) {
+ optionals.set(3);
+ }
+ if (struct.isSetGoal()) {
+ optionals.set(4);
+ }
+ if (struct.isSetRequestTime()) {
+ optionals.set(5);
+ }
+ oprot.writeBitSet(optionals, 6);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetCredentials()) {
+ struct.credentials.write(oprot);
+ }
+ if (struct.isSetLock()) {
+ oprot.writeString(struct.lock);
+ }
+ if (struct.isSetExtent()) {
+ struct.extent.write(oprot);
+ }
+ if (struct.isSetGoal()) {
+ oprot.writeI32(struct.goal.getValue());
+ }
+ if (struct.isSetRequestTime()) {
+ oprot.writeI64(struct.requestTime);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, unloadTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(6);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ }
+ if (incoming.get(2)) {
+ struct.lock = iprot.readString();
+ struct.setLockIsSet(true);
+ }
+ if (incoming.get(3)) {
+ struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ struct.extent.read(iprot);
+ struct.setExtentIsSet(true);
+ }
+ if (incoming.get(4)) {
+ struct.goal = org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal.findByValue(iprot.readI32());
+ struct.setGoalIsSet(true);
+ }
+ if (incoming.get(5)) {
+ struct.requestTime = iprot.readI64();
+ struct.setRequestTimeIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class splitTablet_args implements org.apache.thrift.TBase<splitTablet_args, splitTablet_args._Fields>, java.io.Serializable, Cloneable, Comparable<splitTablet_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("splitTablet_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+ private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+ private static final org.apache.thrift.protocol.TField SPLIT_POINT_FIELD_DESC = new org.apache.thrift.protocol.TField("splitPoint", org.apache.thrift.protocol.TType.STRING, (short)3);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new splitTablet_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new splitTablet_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
+ public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer splitPoint; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)4, "tinfo"),
+ CREDENTIALS((short)1, "credentials"),
+ EXTENT((short)2, "extent"),
+ SPLIT_POINT((short)3, "splitPoint");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 4: // TINFO
+ return TINFO;
+ case 1: // CREDENTIALS
+ return CREDENTIALS;
+ case 2: // EXTENT
+ return EXTENT;
+ case 3: // SPLIT_POINT
+ return SPLIT_POINT;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+ tmpMap.put(_Fields.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+ tmpMap.put(_Fields.SPLIT_POINT, new org.apache.thrift.meta_data.FieldMetaData("splitPoint", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(splitTablet_args.class, metaDataMap);
+ }
+
+ public splitTablet_args() {
+ }
+
+ public splitTablet_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+ org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent,
+ java.nio.ByteBuffer splitPoint)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.extent = extent;
+ this.splitPoint = org.apache.thrift.TBaseHelper.copyBinary(splitPoint);
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public splitTablet_args(splitTablet_args other) {
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ if (other.isSetCredentials()) {
+ this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+ }
+ if (other.isSetExtent()) {
+ this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+ }
+ if (other.isSetSplitPoint()) {
+ this.splitPoint = org.apache.thrift.TBaseHelper.copyBinary(other.splitPoint);
+ }
+ }
+
+ @Override
+ public splitTablet_args deepCopy() {
+ return new splitTablet_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ this.credentials = null;
+ this.extent = null;
+ this.splitPoint = null;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public splitTablet_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+ return this.credentials;
+ }
+
+ public splitTablet_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+ this.credentials = credentials;
+ return this;
+ }
+
+ public void unsetCredentials() {
+ this.credentials = null;
+ }
+
+ /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+ public boolean isSetCredentials() {
+ return this.credentials != null;
+ }
+
+ public void setCredentialsIsSet(boolean value) {
+ if (!value) {
+ this.credentials = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+ return this.extent;
+ }
+
+ public splitTablet_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+ this.extent = extent;
+ return this;
+ }
+
+ public void unsetExtent() {
+ this.extent = null;
+ }
+
+ /** Returns true if field extent is set (has been assigned a value) and false otherwise */
+ public boolean isSetExtent() {
+ return this.extent != null;
+ }
+
+ public void setExtentIsSet(boolean value) {
+ if (!value) {
+ this.extent = null;
+ }
+ }
+
+ public byte[] getSplitPoint() {
+ setSplitPoint(org.apache.thrift.TBaseHelper.rightSize(splitPoint));
+ return splitPoint == null ? null : splitPoint.array();
+ }
+
+ public java.nio.ByteBuffer bufferForSplitPoint() {
+ return org.apache.thrift.TBaseHelper.copyBinary(splitPoint);
+ }
+
+ public splitTablet_args setSplitPoint(byte[] splitPoint) {
+ this.splitPoint = splitPoint == null ? (java.nio.ByteBuffer)null : java.nio.ByteBuffer.wrap(splitPoint.clone());
+ return this;
+ }
+
+ public splitTablet_args setSplitPoint(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer splitPoint) {
+ this.splitPoint = org.apache.thrift.TBaseHelper.copyBinary(splitPoint);
+ return this;
+ }
+
+ public void unsetSplitPoint() {
+ this.splitPoint = null;
+ }
+
+ /** Returns true if field splitPoint is set (has been assigned a value) and false otherwise */
+ public boolean isSetSplitPoint() {
+ return this.splitPoint != null;
+ }
+
+ public void setSplitPointIsSet(boolean value) {
+ if (!value) {
+ this.splitPoint = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case CREDENTIALS:
+ if (value == null) {
+ unsetCredentials();
+ } else {
+ setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+ }
+ break;
+
+ case EXTENT:
+ if (value == null) {
+ unsetExtent();
+ } else {
+ setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+ }
+ break;
+
+ case SPLIT_POINT:
+ if (value == null) {
+ unsetSplitPoint();
+ } else {
+ if (value instanceof byte[]) {
+ setSplitPoint((byte[])value);
+ } else {
+ setSplitPoint((java.nio.ByteBuffer)value);
+ }
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case CREDENTIALS:
+ return getCredentials();
+
+ case EXTENT:
+ return getExtent();
+
+ case SPLIT_POINT:
+ return getSplitPoint();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case CREDENTIALS:
+ return isSetCredentials();
+ case EXTENT:
+ return isSetExtent();
+ case SPLIT_POINT:
+ return isSetSplitPoint();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof splitTablet_args)
+ return this.equals((splitTablet_args)that);
+ return false;
+ }
+
+ public boolean equals(splitTablet_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_credentials = true && this.isSetCredentials();
+ boolean that_present_credentials = true && that.isSetCredentials();
+ if (this_present_credentials || that_present_credentials) {
+ if (!(this_present_credentials && that_present_credentials))
+ return false;
+ if (!this.credentials.equals(that.credentials))
+ return false;
+ }
+
+ boolean this_present_extent = true && this.isSetExtent();
+ boolean that_present_extent = true && that.isSetExtent();
+ if (this_present_extent || that_present_extent) {
+ if (!(this_present_extent && that_present_extent))
+ return false;
+ if (!this.extent.equals(that.extent))
+ return false;
+ }
+
+ boolean this_present_splitPoint = true && this.isSetSplitPoint();
+ boolean that_present_splitPoint = true && that.isSetSplitPoint();
+ if (this_present_splitPoint || that_present_splitPoint) {
+ if (!(this_present_splitPoint && that_present_splitPoint))
+ return false;
+ if (!this.splitPoint.equals(that.splitPoint))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+ if (isSetCredentials())
+ hashCode = hashCode * 8191 + credentials.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+ if (isSetExtent())
+ hashCode = hashCode * 8191 + extent.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetSplitPoint()) ? 131071 : 524287);
+ if (isSetSplitPoint())
+ hashCode = hashCode * 8191 + splitPoint.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(splitTablet_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCredentials()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetExtent(), other.isSetExtent());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetExtent()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetSplitPoint(), other.isSetSplitPoint());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSplitPoint()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.splitPoint, other.splitPoint);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("splitTablet_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("credentials:");
+ if (this.credentials == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.credentials);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("extent:");
+ if (this.extent == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.extent);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("splitPoint:");
+ if (this.splitPoint == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.splitPoint, sb);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ if (credentials != null) {
+ credentials.validate();
+ }
+ if (extent != null) {
+ extent.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class splitTablet_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public splitTablet_argsStandardScheme getScheme() {
+ return new splitTablet_argsStandardScheme();
+ }
+ }
+
+ private static class splitTablet_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<splitTablet_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, splitTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 4: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 1: // CREDENTIALS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // EXTENT
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ struct.extent.read(iprot);
+ struct.setExtentIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 3: // SPLIT_POINT
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.splitPoint = iprot.readBinary();
+ struct.setSplitPointIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, splitTablet_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.credentials != null) {
+ oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+ struct.credentials.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.extent != null) {
+ oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+ struct.extent.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.splitPoint != null) {
+ oprot.writeFieldBegin(SPLIT_POINT_FIELD_DESC);
+ oprot.writeBinary(struct.splitPoint);
+ oprot.writeFieldEnd();
+ }
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class splitTablet_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public splitTablet_argsTupleScheme getScheme() {
+ return new splitTablet_argsTupleScheme();
+ }
+ }
+
+ private static class splitTablet_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<splitTablet_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, splitTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetCredentials()) {
+ optionals.set(1);
+ }
+ if (struct.isSetExtent()) {
+ optionals.set(2);
+ }
+ if (struct.isSetSplitPoint()) {
+ optionals.set(3);
+ }
+ oprot.writeBitSet(optionals, 4);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetCredentials()) {
+ struct.credentials.write(oprot);
+ }
+ if (struct.isSetExtent()) {
+ struct.extent.write(oprot);
+ }
+ if (struct.isSetSplitPoint()) {
+ oprot.writeBinary(struct.splitPoint);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, splitTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(4);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ }
+ if (incoming.get(2)) {
+ struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ struct.extent.read(iprot);
+ struct.setExtentIsSet(true);
+ }
+ if (incoming.get(3)) {
+ struct.splitPoint = iprot.readBinary();
+ struct.setSplitPointIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class splitTablet_result implements org.apache.thrift.TBase<splitTablet_result, splitTablet_result._Fields>, java.io.Serializable, Cloneable, Comparable<splitTablet_result> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("splitTablet_result");
+
+ private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField NSTE_FIELD_DESC = new org.apache.thrift.protocol.TField("nste", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new splitTablet_resultStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new splitTablet_resultTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ SEC((short)1, "sec"),
+ NSTE((short)2, "nste");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // SEC
+ return SEC;
+ case 2: // NSTE
+ return NSTE;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException.class)));
+ tmpMap.put(_Fields.NSTE, new org.apache.thrift.meta_data.FieldMetaData("nste", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException.class)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(splitTablet_result.class, metaDataMap);
+ }
+
+ public splitTablet_result() {
+ }
+
+ public splitTablet_result(
+ org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec,
+ org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste)
+ {
+ this();
+ this.sec = sec;
+ this.nste = nste;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public splitTablet_result(splitTablet_result other) {
+ if (other.isSetSec()) {
+ this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+ }
+ if (other.isSetNste()) {
+ this.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException(other.nste);
+ }
+ }
+
+ @Override
+ public splitTablet_result deepCopy() {
+ return new splitTablet_result(this);
+ }
+
+ @Override
+ public void clear() {
+ this.sec = null;
+ this.nste = null;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
+ return this.sec;
+ }
+
+ public splitTablet_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+ this.sec = sec;
+ return this;
+ }
+
+ public void unsetSec() {
+ this.sec = null;
+ }
+
+ /** Returns true if field sec is set (has been assigned a value) and false otherwise */
+ public boolean isSetSec() {
+ return this.sec != null;
+ }
+
+ public void setSecIsSet(boolean value) {
+ if (!value) {
+ this.sec = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException getNste() {
+ return this.nste;
+ }
+
+ public splitTablet_result setNste(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste) {
+ this.nste = nste;
+ return this;
+ }
+
+ public void unsetNste() {
+ this.nste = null;
+ }
+
+ /** Returns true if field nste is set (has been assigned a value) and false otherwise */
+ public boolean isSetNste() {
+ return this.nste != null;
+ }
+
+ public void setNsteIsSet(boolean value) {
+ if (!value) {
+ this.nste = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case SEC:
+ if (value == null) {
+ unsetSec();
+ } else {
+ setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value);
+ }
+ break;
+
+ case NSTE:
+ if (value == null) {
+ unsetNste();
+ } else {
+ setNste((org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case SEC:
+ return getSec();
+
+ case NSTE:
+ return getNste();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case SEC:
+ return isSetSec();
+ case NSTE:
+ return isSetNste();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof splitTablet_result)
+ return this.equals((splitTablet_result)that);
+ return false;
+ }
+
+ public boolean equals(splitTablet_result that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_sec = true && this.isSetSec();
+ boolean that_present_sec = true && that.isSetSec();
+ if (this_present_sec || that_present_sec) {
+ if (!(this_present_sec && that_present_sec))
+ return false;
+ if (!this.sec.equals(that.sec))
+ return false;
+ }
+
+ boolean this_present_nste = true && this.isSetNste();
+ boolean that_present_nste = true && that.isSetNste();
+ if (this_present_nste || that_present_nste) {
+ if (!(this_present_nste && that_present_nste))
+ return false;
+ if (!this.nste.equals(that.nste))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
+ if (isSetSec())
+ hashCode = hashCode * 8191 + sec.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetNste()) ? 131071 : 524287);
+ if (isSetNste())
+ hashCode = hashCode * 8191 + nste.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(splitTablet_result other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetSec(), other.isSetSec());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSec()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetNste(), other.isSetNste());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetNste()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nste, other.nste);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("splitTablet_result(");
+ boolean first = true;
+
+ sb.append("sec:");
+ if (this.sec == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.sec);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("nste:");
+ if (this.nste == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.nste);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class splitTablet_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public splitTablet_resultStandardScheme getScheme() {
+ return new splitTablet_resultStandardScheme();
+ }
+ }
+
+ private static class splitTablet_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<splitTablet_result> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, splitTablet_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // SEC
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+ struct.sec.read(iprot);
+ struct.setSecIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // NSTE
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException();
+ struct.nste.read(iprot);
+ struct.setNsteIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, splitTablet_result struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.sec != null) {
+ oprot.writeFieldBegin(SEC_FIELD_DESC);
+ struct.sec.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.nste != null) {
+ oprot.writeFieldBegin(NSTE_FIELD_DESC);
+ struct.nste.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class splitTablet_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public splitTablet_resultTupleScheme getScheme() {
+ return new splitTablet_resultTupleScheme();
+ }
+ }
+
+ private static class splitTablet_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<splitTablet_result> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, splitTablet_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetSec()) {
+ optionals.set(0);
+ }
+ if (struct.isSetNste()) {
+ optionals.set(1);
+ }
+ oprot.writeBitSet(optionals, 2);
+ if (struct.isSetSec()) {
+ struct.sec.write(oprot);
+ }
+ if (struct.isSetNste()) {
+ struct.nste.write(oprot);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, splitTablet_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(2);
+ if (incoming.get(0)) {
+ struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+ struct.sec.read(iprot);
+ struct.setSecIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException();
+ struct.nste.read(iprot);
+ struct.setNsteIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class flushTablet_args implements org.apache.thrift.TBase<flushTablet_args, flushTablet_args._Fields>, java.io.Serializable, Cloneable, Comparable<flushTablet_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("flushTablet_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+ private static final org.apache.thrift.protocol.TField LOCK_FIELD_DESC = new org.apache.thrift.protocol.TField("lock", org.apache.thrift.protocol.TType.STRING, (short)3);
+ private static final org.apache.thrift.protocol.TField EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("extent", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new flushTablet_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new flushTablet_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+ public @org.apache.thrift.annotation.Nullable java.lang.String lock; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)1, "tinfo"),
+ CREDENTIALS((short)2, "credentials"),
+ LOCK((short)3, "lock"),
+ EXTENT((short)4, "extent");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // TINFO
+ return TINFO;
+ case 2: // CREDENTIALS
+ return CREDENTIALS;
+ case 3: // LOCK
+ return LOCK;
+ case 4: // EXTENT
+ return EXTENT;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+ tmpMap.put(_Fields.LOCK, new org.apache.thrift.meta_data.FieldMetaData("lock", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+ tmpMap.put(_Fields.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(flushTablet_args.class, metaDataMap);
+ }
+
+ public flushTablet_args() {
+ }
+
+ public flushTablet_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+ java.lang.String lock,
+ org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.lock = lock;
+ this.extent = extent;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public flushTablet_args(flushTablet_args other) {
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ if (other.isSetCredentials()) {
+ this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+ }
+ if (other.isSetLock()) {
+ this.lock = other.lock;
+ }
+ if (other.isSetExtent()) {
+ this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+ }
+ }
+
+ @Override
+ public flushTablet_args deepCopy() {
+ return new flushTablet_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ this.credentials = null;
+ this.lock = null;
+ this.extent = null;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public flushTablet_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+ return this.credentials;
+ }
+
+ public flushTablet_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+ this.credentials = credentials;
+ return this;
+ }
+
+ public void unsetCredentials() {
+ this.credentials = null;
+ }
+
+ /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+ public boolean isSetCredentials() {
+ return this.credentials != null;
+ }
+
+ public void setCredentialsIsSet(boolean value) {
+ if (!value) {
+ this.credentials = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.lang.String getLock() {
+ return this.lock;
+ }
+
+ public flushTablet_args setLock(@org.apache.thrift.annotation.Nullable java.lang.String lock) {
+ this.lock = lock;
+ return this;
+ }
+
+ public void unsetLock() {
+ this.lock = null;
+ }
+
+ /** Returns true if field lock is set (has been assigned a value) and false otherwise */
+ public boolean isSetLock() {
+ return this.lock != null;
+ }
+
+ public void setLockIsSet(boolean value) {
+ if (!value) {
+ this.lock = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+ return this.extent;
+ }
+
+ public flushTablet_args setExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent) {
+ this.extent = extent;
+ return this;
+ }
+
+ public void unsetExtent() {
+ this.extent = null;
+ }
+
+ /** Returns true if field extent is set (has been assigned a value) and false otherwise */
+ public boolean isSetExtent() {
+ return this.extent != null;
+ }
+
+ public void setExtentIsSet(boolean value) {
+ if (!value) {
+ this.extent = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case CREDENTIALS:
+ if (value == null) {
+ unsetCredentials();
+ } else {
+ setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+ }
+ break;
+
+ case LOCK:
+ if (value == null) {
+ unsetLock();
+ } else {
+ setLock((java.lang.String)value);
+ }
+ break;
+
+ case EXTENT:
+ if (value == null) {
+ unsetExtent();
+ } else {
+ setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case CREDENTIALS:
+ return getCredentials();
+
+ case LOCK:
+ return getLock();
+
+ case EXTENT:
+ return getExtent();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case CREDENTIALS:
+ return isSetCredentials();
+ case LOCK:
+ return isSetLock();
+ case EXTENT:
+ return isSetExtent();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof flushTablet_args)
+ return this.equals((flushTablet_args)that);
+ return false;
+ }
+
+ public boolean equals(flushTablet_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_credentials = true && this.isSetCredentials();
+ boolean that_present_credentials = true && that.isSetCredentials();
+ if (this_present_credentials || that_present_credentials) {
+ if (!(this_present_credentials && that_present_credentials))
+ return false;
+ if (!this.credentials.equals(that.credentials))
+ return false;
+ }
+
+ boolean this_present_lock = true && this.isSetLock();
+ boolean that_present_lock = true && that.isSetLock();
+ if (this_present_lock || that_present_lock) {
+ if (!(this_present_lock && that_present_lock))
+ return false;
+ if (!this.lock.equals(that.lock))
+ return false;
+ }
+
+ boolean this_present_extent = true && this.isSetExtent();
+ boolean that_present_extent = true && that.isSetExtent();
+ if (this_present_extent || that_present_extent) {
+ if (!(this_present_extent && that_present_extent))
+ return false;
+ if (!this.extent.equals(that.extent))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+ if (isSetCredentials())
+ hashCode = hashCode * 8191 + credentials.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetLock()) ? 131071 : 524287);
+ if (isSetLock())
+ hashCode = hashCode * 8191 + lock.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+ if (isSetExtent())
+ hashCode = hashCode * 8191 + extent.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(flushTablet_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCredentials()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetLock(), other.isSetLock());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetLock()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lock, other.lock);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetExtent(), other.isSetExtent());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetExtent()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.extent, other.extent);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("flushTablet_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("credentials:");
+ if (this.credentials == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.credentials);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("lock:");
+ if (this.lock == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.lock);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("extent:");
+ if (this.extent == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.extent);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ if (credentials != null) {
+ credentials.validate();
+ }
+ if (extent != null) {
+ extent.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class flushTablet_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public flushTablet_argsStandardScheme getScheme() {
+ return new flushTablet_argsStandardScheme();
+ }
+ }
+
+ private static class flushTablet_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<flushTablet_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, flushTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // CREDENTIALS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 3: // LOCK
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.lock = iprot.readString();
+ struct.setLockIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 4: // EXTENT
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ struct.extent.read(iprot);
+ struct.setExtentIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, flushTablet_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.credentials != null) {
+ oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+ struct.credentials.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.lock != null) {
+ oprot.writeFieldBegin(LOCK_FIELD_DESC);
+ oprot.writeString(struct.lock);
+ oprot.writeFieldEnd();
+ }
+ if (struct.extent != null) {
+ oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+ struct.extent.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class flushTablet_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public flushTablet_argsTupleScheme getScheme() {
+ return new flushTablet_argsTupleScheme();
+ }
+ }
+
+ private static class flushTablet_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<flushTablet_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, flushTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetCredentials()) {
+ optionals.set(1);
+ }
+ if (struct.isSetLock()) {
+ optionals.set(2);
+ }
+ if (struct.isSetExtent()) {
+ optionals.set(3);
+ }
+ oprot.writeBitSet(optionals, 4);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetCredentials()) {
+ struct.credentials.write(oprot);
+ }
+ if (struct.isSetLock()) {
+ oprot.writeString(struct.lock);
+ }
+ if (struct.isSetExtent()) {
+ struct.extent.write(oprot);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, flushTablet_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(4);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ }
+ if (incoming.get(2)) {
+ struct.lock = iprot.readString();
+ struct.setLockIsSet(true);
+ }
+ if (incoming.get(3)) {
+ struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ struct.extent.read(iprot);
+ struct.setExtentIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ private static void unusedMethod() {}
+}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ConstraintViolationException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/ConstraintViolationException.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ConstraintViolationException.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/ConstraintViolationException.java
index ca15e65..41e5748 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ConstraintViolationException.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/ConstraintViolationException.java
@@ -22,7 +22,7 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.tabletserver.thrift;
+package org.apache.accumulo.core.tabletingest.thrift;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
public class ConstraintViolationException extends org.apache.thrift.TException implements org.apache.thrift.TBase<ConstraintViolationException, ConstraintViolationException._Fields>, java.io.Serializable, Cloneable, Comparable<ConstraintViolationException> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/dataImpl/thrift/MapFileInfo.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/DataFileInfo.java
similarity index 86%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/dataImpl/thrift/MapFileInfo.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/DataFileInfo.java
index 0d7de77..12276fd 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/dataImpl/thrift/MapFileInfo.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/DataFileInfo.java
@@ -22,16 +22,16 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.dataImpl.thrift;
+package org.apache.accumulo.core.tabletingest.thrift;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class MapFileInfo implements org.apache.thrift.TBase<MapFileInfo, MapFileInfo._Fields>, java.io.Serializable, Cloneable, Comparable<MapFileInfo> {
- private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MapFileInfo");
+public class DataFileInfo implements org.apache.thrift.TBase<DataFileInfo, DataFileInfo._Fields>, java.io.Serializable, Cloneable, Comparable<DataFileInfo> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DataFileInfo");
private static final org.apache.thrift.protocol.TField ESTIMATED_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("estimatedSize", org.apache.thrift.protocol.TType.I64, (short)1);
- private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new MapFileInfoStandardSchemeFactory();
- private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new MapFileInfoTupleSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new DataFileInfoStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new DataFileInfoTupleSchemeFactory();
public long estimatedSize; // required
@@ -106,13 +106,13 @@
tmpMap.put(_Fields.ESTIMATED_SIZE, new org.apache.thrift.meta_data.FieldMetaData("estimatedSize", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
- org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(MapFileInfo.class, metaDataMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DataFileInfo.class, metaDataMap);
}
- public MapFileInfo() {
+ public DataFileInfo() {
}
- public MapFileInfo(
+ public DataFileInfo(
long estimatedSize)
{
this();
@@ -123,14 +123,14 @@
/**
* Performs a deep copy on <i>other</i>.
*/
- public MapFileInfo(MapFileInfo other) {
+ public DataFileInfo(DataFileInfo other) {
__isset_bitfield = other.__isset_bitfield;
this.estimatedSize = other.estimatedSize;
}
@Override
- public MapFileInfo deepCopy() {
- return new MapFileInfo(this);
+ public DataFileInfo deepCopy() {
+ return new DataFileInfo(this);
}
@Override
@@ -143,7 +143,7 @@
return this.estimatedSize;
}
- public MapFileInfo setEstimatedSize(long estimatedSize) {
+ public DataFileInfo setEstimatedSize(long estimatedSize) {
this.estimatedSize = estimatedSize;
setEstimatedSizeIsSet(true);
return this;
@@ -203,12 +203,12 @@
@Override
public boolean equals(java.lang.Object that) {
- if (that instanceof MapFileInfo)
- return this.equals((MapFileInfo)that);
+ if (that instanceof DataFileInfo)
+ return this.equals((DataFileInfo)that);
return false;
}
- public boolean equals(MapFileInfo that) {
+ public boolean equals(DataFileInfo that) {
if (that == null)
return false;
if (this == that)
@@ -236,7 +236,7 @@
}
@Override
- public int compareTo(MapFileInfo other) {
+ public int compareTo(DataFileInfo other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
@@ -274,7 +274,7 @@
@Override
public java.lang.String toString() {
- java.lang.StringBuilder sb = new java.lang.StringBuilder("MapFileInfo(");
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("DataFileInfo(");
boolean first = true;
sb.append("estimatedSize:");
@@ -307,17 +307,17 @@
}
}
- private static class MapFileInfoStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ private static class DataFileInfoStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
@Override
- public MapFileInfoStandardScheme getScheme() {
- return new MapFileInfoStandardScheme();
+ public DataFileInfoStandardScheme getScheme() {
+ return new DataFileInfoStandardScheme();
}
}
- private static class MapFileInfoStandardScheme extends org.apache.thrift.scheme.StandardScheme<MapFileInfo> {
+ private static class DataFileInfoStandardScheme extends org.apache.thrift.scheme.StandardScheme<DataFileInfo> {
@Override
- public void read(org.apache.thrift.protocol.TProtocol iprot, MapFileInfo struct) throws org.apache.thrift.TException {
+ public void read(org.apache.thrift.protocol.TProtocol iprot, DataFileInfo struct) throws org.apache.thrift.TException {
org.apache.thrift.protocol.TField schemeField;
iprot.readStructBegin();
while (true)
@@ -347,7 +347,7 @@
}
@Override
- public void write(org.apache.thrift.protocol.TProtocol oprot, MapFileInfo struct) throws org.apache.thrift.TException {
+ public void write(org.apache.thrift.protocol.TProtocol oprot, DataFileInfo struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
@@ -360,17 +360,17 @@
}
- private static class MapFileInfoTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ private static class DataFileInfoTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
@Override
- public MapFileInfoTupleScheme getScheme() {
- return new MapFileInfoTupleScheme();
+ public DataFileInfoTupleScheme getScheme() {
+ return new DataFileInfoTupleScheme();
}
}
- private static class MapFileInfoTupleScheme extends org.apache.thrift.scheme.TupleScheme<MapFileInfo> {
+ private static class DataFileInfoTupleScheme extends org.apache.thrift.scheme.TupleScheme<DataFileInfo> {
@Override
- public void write(org.apache.thrift.protocol.TProtocol prot, MapFileInfo struct) throws org.apache.thrift.TException {
+ public void write(org.apache.thrift.protocol.TProtocol prot, DataFileInfo struct) throws org.apache.thrift.TException {
org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet optionals = new java.util.BitSet();
if (struct.isSetEstimatedSize()) {
@@ -383,7 +383,7 @@
}
@Override
- public void read(org.apache.thrift.protocol.TProtocol prot, MapFileInfo struct) throws org.apache.thrift.TException {
+ public void read(org.apache.thrift.protocol.TProtocol prot, DataFileInfo struct) throws org.apache.thrift.TException {
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
java.util.BitSet incoming = iprot.readBitSet(1);
if (incoming.get(0)) {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TDurability.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TDurability.java
similarity index 96%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TDurability.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TDurability.java
index b5e8a95..2bab344 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TDurability.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TDurability.java
@@ -22,7 +22,7 @@
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
-package org.apache.accumulo.core.tabletserver.thrift;
+package org.apache.accumulo.core.tabletingest.thrift;
public enum TDurability implements org.apache.thrift.TEnum {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TabletIngestClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TabletIngestClientService.java
new file mode 100644
index 0000000..881fe6e
--- /dev/null
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletingest/thrift/TabletIngestClientService.java
@@ -0,0 +1,10359 @@
+/*
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.17.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ * @generated
+ */
+package org.apache.accumulo.core.tabletingest.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class TabletIngestClientService {
+
+ public interface Iface {
+
+ public long startUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+ public void applyUpdates(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations) throws org.apache.thrift.TException;
+
+ public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors closeUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException;
+
+ public boolean cancelUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.thrift.TException;
+
+ public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+ public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> conditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException;
+
+ public void invalidateConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
+
+ public void closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
+
+ public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime) throws org.apache.thrift.TException;
+
+ }
+
+ public interface AsyncIface {
+
+ public void startUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
+
+ public void applyUpdates(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+ public void closeUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler) throws org.apache.thrift.TException;
+
+ public void cancelUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
+
+ public void startConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler) throws org.apache.thrift.TException;
+
+ public void conditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler) throws org.apache.thrift.TException;
+
+ public void invalidateConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+ public void closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+ public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+ }
+
+ public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+ public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
+ public Factory() {}
+ @Override
+ public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
+ return new Client(prot);
+ }
+ @Override
+ public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+ return new Client(iprot, oprot);
+ }
+ }
+
+ public Client(org.apache.thrift.protocol.TProtocol prot)
+ {
+ super(prot, prot);
+ }
+
+ public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
+ super(iprot, oprot);
+ }
+
+ @Override
+ public long startUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+ {
+ send_startUpdate(tinfo, credentials, durability);
+ return recv_startUpdate();
+ }
+
+ public void send_startUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability) throws org.apache.thrift.TException
+ {
+ startUpdate_args args = new startUpdate_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setDurability(durability);
+ sendBase("startUpdate", args);
+ }
+
+ public long recv_startUpdate() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+ {
+ startUpdate_result result = new startUpdate_result();
+ receiveBase(result, "startUpdate");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ if (result.sec != null) {
+ throw result.sec;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startUpdate failed: unknown result");
+ }
+
+ @Override
+ public void applyUpdates(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations) throws org.apache.thrift.TException
+ {
+ send_applyUpdates(tinfo, updateID, keyExtent, mutations);
+ }
+
+ public void send_applyUpdates(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations) throws org.apache.thrift.TException
+ {
+ applyUpdates_args args = new applyUpdates_args();
+ args.setTinfo(tinfo);
+ args.setUpdateID(updateID);
+ args.setKeyExtent(keyExtent);
+ args.setMutations(mutations);
+ sendBaseOneway("applyUpdates", args);
+ }
+
+ @Override
+ public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors closeUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException
+ {
+ send_closeUpdate(tinfo, updateID);
+ return recv_closeUpdate();
+ }
+
+ public void send_closeUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.thrift.TException
+ {
+ closeUpdate_args args = new closeUpdate_args();
+ args.setTinfo(tinfo);
+ args.setUpdateID(updateID);
+ sendBase("closeUpdate", args);
+ }
+
+ public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors recv_closeUpdate() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException
+ {
+ closeUpdate_result result = new closeUpdate_result();
+ receiveBase(result, "closeUpdate");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ if (result.nssi != null) {
+ throw result.nssi;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "closeUpdate failed: unknown result");
+ }
+
+ @Override
+ public boolean cancelUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.thrift.TException
+ {
+ send_cancelUpdate(tinfo, updateID);
+ return recv_cancelUpdate();
+ }
+
+ public void send_cancelUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID) throws org.apache.thrift.TException
+ {
+ cancelUpdate_args args = new cancelUpdate_args();
+ args.setTinfo(tinfo);
+ args.setUpdateID(updateID);
+ sendBase("cancelUpdate", args);
+ }
+
+ public boolean recv_cancelUpdate() throws org.apache.thrift.TException
+ {
+ cancelUpdate_result result = new cancelUpdate_result();
+ receiveBase(result, "cancelUpdate");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "cancelUpdate failed: unknown result");
+ }
+
+ @Override
+ public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+ {
+ send_startConditionalUpdate(tinfo, credentials, authorizations, tableID, durability, classLoaderContext);
+ return recv_startConditionalUpdate();
+ }
+
+ public void send_startConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext) throws org.apache.thrift.TException
+ {
+ startConditionalUpdate_args args = new startConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setAuthorizations(authorizations);
+ args.setTableID(tableID);
+ args.setDurability(durability);
+ args.setClassLoaderContext(classLoaderContext);
+ sendBase("startConditionalUpdate", args);
+ }
+
+ public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession recv_startConditionalUpdate() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+ {
+ startConditionalUpdate_result result = new startConditionalUpdate_result();
+ receiveBase(result, "startConditionalUpdate");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ if (result.sec != null) {
+ throw result.sec;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startConditionalUpdate failed: unknown result");
+ }
+
+ @Override
+ public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> conditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException
+ {
+ send_conditionalUpdate(tinfo, sessID, mutations, symbols);
+ return recv_conditionalUpdate();
+ }
+
+ public void send_conditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols) throws org.apache.thrift.TException
+ {
+ conditionalUpdate_args args = new conditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ args.setMutations(mutations);
+ args.setSymbols(symbols);
+ sendBase("conditionalUpdate", args);
+ }
+
+ public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> recv_conditionalUpdate() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException
+ {
+ conditionalUpdate_result result = new conditionalUpdate_result();
+ receiveBase(result, "conditionalUpdate");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ if (result.nssi != null) {
+ throw result.nssi;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "conditionalUpdate failed: unknown result");
+ }
+
+ @Override
+ public void invalidateConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+ {
+ send_invalidateConditionalUpdate(tinfo, sessID);
+ recv_invalidateConditionalUpdate();
+ }
+
+ public void send_invalidateConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+ {
+ invalidateConditionalUpdate_args args = new invalidateConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ sendBase("invalidateConditionalUpdate", args);
+ }
+
+ public void recv_invalidateConditionalUpdate() throws org.apache.thrift.TException
+ {
+ invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+ receiveBase(result, "invalidateConditionalUpdate");
+ return;
+ }
+
+ @Override
+ public void closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+ {
+ send_closeConditionalUpdate(tinfo, sessID);
+ }
+
+ public void send_closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+ {
+ closeConditionalUpdate_args args = new closeConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ sendBaseOneway("closeConditionalUpdate", args);
+ }
+
+ @Override
+ public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime) throws org.apache.thrift.TException
+ {
+ send_loadFiles(tinfo, credentials, tid, dir, files, setTime);
+ }
+
+ public void send_loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime) throws org.apache.thrift.TException
+ {
+ loadFiles_args args = new loadFiles_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setTid(tid);
+ args.setDir(dir);
+ args.setFiles(files);
+ args.setSetTime(setTime);
+ sendBaseOneway("loadFiles", args);
+ }
+
+ }
+ public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+ public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
+ private org.apache.thrift.async.TAsyncClientManager clientManager;
+ private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
+ public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
+ this.clientManager = clientManager;
+ this.protocolFactory = protocolFactory;
+ }
+ @Override
+ public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
+ return new AsyncClient(protocolFactory, clientManager, transport);
+ }
+ }
+
+ public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
+ super(protocolFactory, clientManager, transport);
+ }
+
+ @Override
+ public void startUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ startUpdate_call method_call = new startUpdate_call(tinfo, credentials, durability, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class startUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+ private TDurability durability;
+ public startUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.durability = durability;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ startUpdate_args args = new startUpdate_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setDurability(durability);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public java.lang.Long getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_startUpdate();
+ }
+ }
+
+ @Override
+ public void applyUpdates(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ applyUpdates_call method_call = new applyUpdates_call(tinfo, updateID, keyExtent, mutations, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class applyUpdates_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private long updateID;
+ private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent;
+ private java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations;
+ public applyUpdates_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, true);
+ this.tinfo = tinfo;
+ this.updateID = updateID;
+ this.keyExtent = keyExtent;
+ this.mutations = mutations;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("applyUpdates", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+ applyUpdates_args args = new applyUpdates_args();
+ args.setTinfo(tinfo);
+ args.setUpdateID(updateID);
+ args.setKeyExtent(keyExtent);
+ args.setMutations(mutations);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public Void getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return null;
+ }
+ }
+
+ @Override
+ public void closeUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ closeUpdate_call method_call = new closeUpdate_call(tinfo, updateID, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class closeUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private long updateID;
+ public closeUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.tinfo = tinfo;
+ this.updateID = updateID;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("closeUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ closeUpdate_args args = new closeUpdate_args();
+ args.setTinfo(tinfo);
+ args.setUpdateID(updateID);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors getResult() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_closeUpdate();
+ }
+ }
+
+ @Override
+ public void cancelUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ cancelUpdate_call method_call = new cancelUpdate_call(tinfo, updateID, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class cancelUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Boolean> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private long updateID;
+ public cancelUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.tinfo = tinfo;
+ this.updateID = updateID;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("cancelUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ cancelUpdate_args args = new cancelUpdate_args();
+ args.setTinfo(tinfo);
+ args.setUpdateID(updateID);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public java.lang.Boolean getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_cancelUpdate();
+ }
+ }
+
+ @Override
+ public void startConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ startConditionalUpdate_call method_call = new startConditionalUpdate_call(tinfo, credentials, authorizations, tableID, durability, classLoaderContext, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class startConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+ private java.util.List<java.nio.ByteBuffer> authorizations;
+ private java.lang.String tableID;
+ private TDurability durability;
+ private java.lang.String classLoaderContext;
+ public startConditionalUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.nio.ByteBuffer> authorizations, java.lang.String tableID, TDurability durability, java.lang.String classLoaderContext, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.authorizations = authorizations;
+ this.tableID = tableID;
+ this.durability = durability;
+ this.classLoaderContext = classLoaderContext;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ startConditionalUpdate_args args = new startConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setAuthorizations(authorizations);
+ args.setTableID(tableID);
+ args.setDurability(durability);
+ args.setClassLoaderContext(classLoaderContext);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_startConditionalUpdate();
+ }
+ }
+
+ @Override
+ public void conditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ conditionalUpdate_call method_call = new conditionalUpdate_call(tinfo, sessID, mutations, symbols, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class conditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private long sessID;
+ private java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations;
+ private java.util.List<java.lang.String> symbols;
+ public conditionalUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations, java.util.List<java.lang.String> symbols, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.tinfo = tinfo;
+ this.sessID = sessID;
+ this.mutations = mutations;
+ this.symbols = symbols;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("conditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ conditionalUpdate_args args = new conditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ args.setMutations(mutations);
+ args.setSymbols(symbols);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> getResult() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_conditionalUpdate();
+ }
+ }
+
+ @Override
+ public void invalidateConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ invalidateConditionalUpdate_call method_call = new invalidateConditionalUpdate_call(tinfo, sessID, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class invalidateConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private long sessID;
+ public invalidateConditionalUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.tinfo = tinfo;
+ this.sessID = sessID;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("invalidateConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ invalidateConditionalUpdate_args args = new invalidateConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public Void getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ (new Client(prot)).recv_invalidateConditionalUpdate();
+ return null;
+ }
+ }
+
+ @Override
+ public void closeConditionalUpdate(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ closeConditionalUpdate_call method_call = new closeConditionalUpdate_call(tinfo, sessID, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class closeConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private long sessID;
+ public closeConditionalUpdate_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, true);
+ this.tinfo = tinfo;
+ this.sessID = sessID;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("closeConditionalUpdate", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+ closeConditionalUpdate_args args = new closeConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public Void getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return null;
+ }
+ }
+
+ @Override
+ public void loadFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ loadFiles_call method_call = new loadFiles_call(tinfo, credentials, tid, dir, files, setTime, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class loadFiles_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+ private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+ private long tid;
+ private java.lang.String dir;
+ private java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files;
+ private boolean setTime;
+ public loadFiles_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.lang.String dir, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, true);
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.tid = tid;
+ this.dir = dir;
+ this.files = files;
+ this.setTime = setTime;
+ }
+
+ @Override
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("loadFiles", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+ loadFiles_args args = new loadFiles_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setTid(tid);
+ args.setDir(dir);
+ args.setFiles(files);
+ args.setSetTime(setTime);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ @Override
+ public Void getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new java.lang.IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return null;
+ }
+ }
+
+ }
+
+ public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+ private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
+ public Processor(I iface) {
+ super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+ }
+
+ protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
+ super(iface, getProcessMap(processMap));
+ }
+
+ private static <I extends Iface> java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
+ processMap.put("startUpdate", new startUpdate());
+ processMap.put("applyUpdates", new applyUpdates());
+ processMap.put("closeUpdate", new closeUpdate());
+ processMap.put("cancelUpdate", new cancelUpdate());
+ processMap.put("startConditionalUpdate", new startConditionalUpdate());
+ processMap.put("conditionalUpdate", new conditionalUpdate());
+ processMap.put("invalidateConditionalUpdate", new invalidateConditionalUpdate());
+ processMap.put("closeConditionalUpdate", new closeConditionalUpdate());
+ processMap.put("loadFiles", new loadFiles());
+ return processMap;
+ }
+
+ public static class startUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startUpdate_args> {
+ public startUpdate() {
+ super("startUpdate");
+ }
+
+ @Override
+ public startUpdate_args getEmptyArgsInstance() {
+ return new startUpdate_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public startUpdate_result getResult(I iface, startUpdate_args args) throws org.apache.thrift.TException {
+ startUpdate_result result = new startUpdate_result();
+ try {
+ result.success = iface.startUpdate(args.tinfo, args.credentials, args.durability);
+ result.setSuccessIsSet(true);
+ } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+ result.sec = sec;
+ }
+ return result;
+ }
+ }
+
+ public static class applyUpdates<I extends Iface> extends org.apache.thrift.ProcessFunction<I, applyUpdates_args> {
+ public applyUpdates() {
+ super("applyUpdates");
+ }
+
+ @Override
+ public applyUpdates_args getEmptyArgsInstance() {
+ return new applyUpdates_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public org.apache.thrift.TBase getResult(I iface, applyUpdates_args args) throws org.apache.thrift.TException {
+ iface.applyUpdates(args.tinfo, args.updateID, args.keyExtent, args.mutations);
+ return null;
+ }
+ }
+
+ public static class closeUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, closeUpdate_args> {
+ public closeUpdate() {
+ super("closeUpdate");
+ }
+
+ @Override
+ public closeUpdate_args getEmptyArgsInstance() {
+ return new closeUpdate_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public closeUpdate_result getResult(I iface, closeUpdate_args args) throws org.apache.thrift.TException {
+ closeUpdate_result result = new closeUpdate_result();
+ try {
+ result.success = iface.closeUpdate(args.tinfo, args.updateID);
+ } catch (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
+ result.nssi = nssi;
+ }
+ return result;
+ }
+ }
+
+ public static class cancelUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, cancelUpdate_args> {
+ public cancelUpdate() {
+ super("cancelUpdate");
+ }
+
+ @Override
+ public cancelUpdate_args getEmptyArgsInstance() {
+ return new cancelUpdate_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public cancelUpdate_result getResult(I iface, cancelUpdate_args args) throws org.apache.thrift.TException {
+ cancelUpdate_result result = new cancelUpdate_result();
+ result.success = iface.cancelUpdate(args.tinfo, args.updateID);
+ result.setSuccessIsSet(true);
+ return result;
+ }
+ }
+
+ public static class startConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startConditionalUpdate_args> {
+ public startConditionalUpdate() {
+ super("startConditionalUpdate");
+ }
+
+ @Override
+ public startConditionalUpdate_args getEmptyArgsInstance() {
+ return new startConditionalUpdate_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public startConditionalUpdate_result getResult(I iface, startConditionalUpdate_args args) throws org.apache.thrift.TException {
+ startConditionalUpdate_result result = new startConditionalUpdate_result();
+ try {
+ result.success = iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID, args.durability, args.classLoaderContext);
+ } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+ result.sec = sec;
+ }
+ return result;
+ }
+ }
+
+ public static class conditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, conditionalUpdate_args> {
+ public conditionalUpdate() {
+ super("conditionalUpdate");
+ }
+
+ @Override
+ public conditionalUpdate_args getEmptyArgsInstance() {
+ return new conditionalUpdate_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public conditionalUpdate_result getResult(I iface, conditionalUpdate_args args) throws org.apache.thrift.TException {
+ conditionalUpdate_result result = new conditionalUpdate_result();
+ try {
+ result.success = iface.conditionalUpdate(args.tinfo, args.sessID, args.mutations, args.symbols);
+ } catch (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
+ result.nssi = nssi;
+ }
+ return result;
+ }
+ }
+
+ public static class invalidateConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, invalidateConditionalUpdate_args> {
+ public invalidateConditionalUpdate() {
+ super("invalidateConditionalUpdate");
+ }
+
+ @Override
+ public invalidateConditionalUpdate_args getEmptyArgsInstance() {
+ return new invalidateConditionalUpdate_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public invalidateConditionalUpdate_result getResult(I iface, invalidateConditionalUpdate_args args) throws org.apache.thrift.TException {
+ invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+ iface.invalidateConditionalUpdate(args.tinfo, args.sessID);
+ return result;
+ }
+ }
+
+ public static class closeConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, closeConditionalUpdate_args> {
+ public closeConditionalUpdate() {
+ super("closeConditionalUpdate");
+ }
+
+ @Override
+ public closeConditionalUpdate_args getEmptyArgsInstance() {
+ return new closeConditionalUpdate_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public org.apache.thrift.TBase getResult(I iface, closeConditionalUpdate_args args) throws org.apache.thrift.TException {
+ iface.closeConditionalUpdate(args.tinfo, args.sessID);
+ return null;
+ }
+ }
+
+ public static class loadFiles<I extends Iface> extends org.apache.thrift.ProcessFunction<I, loadFiles_args> {
+ public loadFiles() {
+ super("loadFiles");
+ }
+
+ @Override
+ public loadFiles_args getEmptyArgsInstance() {
+ return new loadFiles_args();
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ protected boolean rethrowUnhandledExceptions() {
+ return false;
+ }
+
+ @Override
+ public org.apache.thrift.TBase getResult(I iface, loadFiles_args args) throws org.apache.thrift.TException {
+ iface.loadFiles(args.tinfo, args.credentials, args.tid, args.dir, args.files, args.setTime);
+ return null;
+ }
+ }
+
+ }
+
+ public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
+ private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
+ public AsyncProcessor(I iface) {
+ super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
+ }
+
+ protected AsyncProcessor(I iface, java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>> processMap) {
+ super(iface, getProcessMap(processMap));
+ }
+
+ private static <I extends AsyncIface> java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>> processMap) {
+ processMap.put("startUpdate", new startUpdate());
+ processMap.put("applyUpdates", new applyUpdates());
+ processMap.put("closeUpdate", new closeUpdate());
+ processMap.put("cancelUpdate", new cancelUpdate());
+ processMap.put("startConditionalUpdate", new startConditionalUpdate());
+ processMap.put("conditionalUpdate", new conditionalUpdate());
+ processMap.put("invalidateConditionalUpdate", new invalidateConditionalUpdate());
+ processMap.put("closeConditionalUpdate", new closeConditionalUpdate());
+ processMap.put("loadFiles", new loadFiles());
+ return processMap;
+ }
+
+ public static class startUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startUpdate_args, java.lang.Long> {
+ public startUpdate() {
+ super("startUpdate");
+ }
+
+ @Override
+ public startUpdate_args getEmptyArgsInstance() {
+ return new startUpdate_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Long>() {
+ @Override
+ public void onComplete(java.lang.Long o) {
+ startUpdate_result result = new startUpdate_result();
+ result.success = o;
+ result.setSuccessIsSet(true);
+ try {
+ fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+ } catch (org.apache.thrift.transport.TTransportException e) {
+ _LOGGER.error("TTransportException writing to internal frame buffer", e);
+ fb.close();
+ } catch (java.lang.Exception e) {
+ _LOGGER.error("Exception writing to internal frame buffer", e);
+ onError(e);
+ }
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+ org.apache.thrift.TSerializable msg;
+ startUpdate_result result = new startUpdate_result();
+ if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+ result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+ result.setSecIsSet(true);
+ msg = result;
+ } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ return;
+ } else if (e instanceof org.apache.thrift.TApplicationException) {
+ _LOGGER.error("TApplicationException inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = (org.apache.thrift.TApplicationException)e;
+ } else {
+ _LOGGER.error("Exception inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+ }
+ try {
+ fcall.sendResponse(fb,msg,msgType,seqid);
+ } catch (java.lang.Exception ex) {
+ _LOGGER.error("Exception writing to internal frame buffer", ex);
+ fb.close();
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ public void start(I iface, startUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+ iface.startUpdate(args.tinfo, args.credentials, args.durability,resultHandler);
+ }
+ }
+
+ public static class applyUpdates<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, applyUpdates_args, Void> {
+ public applyUpdates() {
+ super("applyUpdates");
+ }
+
+ @Override
+ public applyUpdates_args getEmptyArgsInstance() {
+ return new applyUpdates_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<Void>() {
+ @Override
+ public void onComplete(Void o) {
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ } else {
+ _LOGGER.error("Exception inside oneway handler", e);
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ public void start(I iface, applyUpdates_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ iface.applyUpdates(args.tinfo, args.updateID, args.keyExtent, args.mutations,resultHandler);
+ }
+ }
+
+ public static class closeUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, closeUpdate_args, org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> {
+ public closeUpdate() {
+ super("closeUpdate");
+ }
+
+ @Override
+ public closeUpdate_args getEmptyArgsInstance() {
+ return new closeUpdate_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors>() {
+ @Override
+ public void onComplete(org.apache.accumulo.core.dataImpl.thrift.UpdateErrors o) {
+ closeUpdate_result result = new closeUpdate_result();
+ result.success = o;
+ try {
+ fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+ } catch (org.apache.thrift.transport.TTransportException e) {
+ _LOGGER.error("TTransportException writing to internal frame buffer", e);
+ fb.close();
+ } catch (java.lang.Exception e) {
+ _LOGGER.error("Exception writing to internal frame buffer", e);
+ onError(e);
+ }
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+ org.apache.thrift.TSerializable msg;
+ closeUpdate_result result = new closeUpdate_result();
+ if (e instanceof org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException) {
+ result.nssi = (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException) e;
+ result.setNssiIsSet(true);
+ msg = result;
+ } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ return;
+ } else if (e instanceof org.apache.thrift.TApplicationException) {
+ _LOGGER.error("TApplicationException inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = (org.apache.thrift.TApplicationException)e;
+ } else {
+ _LOGGER.error("Exception inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+ }
+ try {
+ fcall.sendResponse(fb,msg,msgType,seqid);
+ } catch (java.lang.Exception ex) {
+ _LOGGER.error("Exception writing to internal frame buffer", ex);
+ fb.close();
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ public void start(I iface, closeUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.UpdateErrors> resultHandler) throws org.apache.thrift.TException {
+ iface.closeUpdate(args.tinfo, args.updateID,resultHandler);
+ }
+ }
+
+ public static class cancelUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cancelUpdate_args, java.lang.Boolean> {
+ public cancelUpdate() {
+ super("cancelUpdate");
+ }
+
+ @Override
+ public cancelUpdate_args getEmptyArgsInstance() {
+ return new cancelUpdate_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() {
+ @Override
+ public void onComplete(java.lang.Boolean o) {
+ cancelUpdate_result result = new cancelUpdate_result();
+ result.success = o;
+ result.setSuccessIsSet(true);
+ try {
+ fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+ } catch (org.apache.thrift.transport.TTransportException e) {
+ _LOGGER.error("TTransportException writing to internal frame buffer", e);
+ fb.close();
+ } catch (java.lang.Exception e) {
+ _LOGGER.error("Exception writing to internal frame buffer", e);
+ onError(e);
+ }
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+ org.apache.thrift.TSerializable msg;
+ cancelUpdate_result result = new cancelUpdate_result();
+ if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ return;
+ } else if (e instanceof org.apache.thrift.TApplicationException) {
+ _LOGGER.error("TApplicationException inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = (org.apache.thrift.TApplicationException)e;
+ } else {
+ _LOGGER.error("Exception inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+ }
+ try {
+ fcall.sendResponse(fb,msg,msgType,seqid);
+ } catch (java.lang.Exception ex) {
+ _LOGGER.error("Exception writing to internal frame buffer", ex);
+ fb.close();
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ public void start(I iface, cancelUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+ iface.cancelUpdate(args.tinfo, args.updateID,resultHandler);
+ }
+ }
+
+ public static class startConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startConditionalUpdate_args, org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> {
+ public startConditionalUpdate() {
+ super("startConditionalUpdate");
+ }
+
+ @Override
+ public startConditionalUpdate_args getEmptyArgsInstance() {
+ return new startConditionalUpdate_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession>() {
+ @Override
+ public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TConditionalSession o) {
+ startConditionalUpdate_result result = new startConditionalUpdate_result();
+ result.success = o;
+ try {
+ fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+ } catch (org.apache.thrift.transport.TTransportException e) {
+ _LOGGER.error("TTransportException writing to internal frame buffer", e);
+ fb.close();
+ } catch (java.lang.Exception e) {
+ _LOGGER.error("Exception writing to internal frame buffer", e);
+ onError(e);
+ }
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+ org.apache.thrift.TSerializable msg;
+ startConditionalUpdate_result result = new startConditionalUpdate_result();
+ if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) {
+ result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
+ result.setSecIsSet(true);
+ msg = result;
+ } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ return;
+ } else if (e instanceof org.apache.thrift.TApplicationException) {
+ _LOGGER.error("TApplicationException inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = (org.apache.thrift.TApplicationException)e;
+ } else {
+ _LOGGER.error("Exception inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+ }
+ try {
+ fcall.sendResponse(fb,msg,msgType,seqid);
+ } catch (java.lang.Exception ex) {
+ _LOGGER.error("Exception writing to internal frame buffer", ex);
+ fb.close();
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ public void start(I iface, startConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TConditionalSession> resultHandler) throws org.apache.thrift.TException {
+ iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID, args.durability, args.classLoaderContext,resultHandler);
+ }
+ }
+
+ public static class conditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, conditionalUpdate_args, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> {
+ public conditionalUpdate() {
+ super("conditionalUpdate");
+ }
+
+ @Override
+ public conditionalUpdate_args getEmptyArgsInstance() {
+ return new conditionalUpdate_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>>() {
+ @Override
+ public void onComplete(java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> o) {
+ conditionalUpdate_result result = new conditionalUpdate_result();
+ result.success = o;
+ try {
+ fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+ } catch (org.apache.thrift.transport.TTransportException e) {
+ _LOGGER.error("TTransportException writing to internal frame buffer", e);
+ fb.close();
+ } catch (java.lang.Exception e) {
+ _LOGGER.error("Exception writing to internal frame buffer", e);
+ onError(e);
+ }
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+ org.apache.thrift.TSerializable msg;
+ conditionalUpdate_result result = new conditionalUpdate_result();
+ if (e instanceof org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException) {
+ result.nssi = (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException) e;
+ result.setNssiIsSet(true);
+ msg = result;
+ } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ return;
+ } else if (e instanceof org.apache.thrift.TApplicationException) {
+ _LOGGER.error("TApplicationException inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = (org.apache.thrift.TApplicationException)e;
+ } else {
+ _LOGGER.error("Exception inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+ }
+ try {
+ fcall.sendResponse(fb,msg,msgType,seqid);
+ } catch (java.lang.Exception ex) {
+ _LOGGER.error("Exception writing to internal frame buffer", ex);
+ fb.close();
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ public void start(I iface, conditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>> resultHandler) throws org.apache.thrift.TException {
+ iface.conditionalUpdate(args.tinfo, args.sessID, args.mutations, args.symbols,resultHandler);
+ }
+ }
+
+ public static class invalidateConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, invalidateConditionalUpdate_args, Void> {
+ public invalidateConditionalUpdate() {
+ super("invalidateConditionalUpdate");
+ }
+
+ @Override
+ public invalidateConditionalUpdate_args getEmptyArgsInstance() {
+ return new invalidateConditionalUpdate_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<Void>() {
+ @Override
+ public void onComplete(Void o) {
+ invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+ try {
+ fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+ } catch (org.apache.thrift.transport.TTransportException e) {
+ _LOGGER.error("TTransportException writing to internal frame buffer", e);
+ fb.close();
+ } catch (java.lang.Exception e) {
+ _LOGGER.error("Exception writing to internal frame buffer", e);
+ onError(e);
+ }
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+ org.apache.thrift.TSerializable msg;
+ invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+ if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ return;
+ } else if (e instanceof org.apache.thrift.TApplicationException) {
+ _LOGGER.error("TApplicationException inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = (org.apache.thrift.TApplicationException)e;
+ } else {
+ _LOGGER.error("Exception inside handler", e);
+ msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+ msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+ }
+ try {
+ fcall.sendResponse(fb,msg,msgType,seqid);
+ } catch (java.lang.Exception ex) {
+ _LOGGER.error("Exception writing to internal frame buffer", ex);
+ fb.close();
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return false;
+ }
+
+ @Override
+ public void start(I iface, invalidateConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ iface.invalidateConditionalUpdate(args.tinfo, args.sessID,resultHandler);
+ }
+ }
+
+ public static class closeConditionalUpdate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, closeConditionalUpdate_args, Void> {
+ public closeConditionalUpdate() {
+ super("closeConditionalUpdate");
+ }
+
+ @Override
+ public closeConditionalUpdate_args getEmptyArgsInstance() {
+ return new closeConditionalUpdate_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<Void>() {
+ @Override
+ public void onComplete(Void o) {
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ } else {
+ _LOGGER.error("Exception inside oneway handler", e);
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ public void start(I iface, closeConditionalUpdate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ iface.closeConditionalUpdate(args.tinfo, args.sessID,resultHandler);
+ }
+ }
+
+ public static class loadFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, loadFiles_args, Void> {
+ public loadFiles() {
+ super("loadFiles");
+ }
+
+ @Override
+ public loadFiles_args getEmptyArgsInstance() {
+ return new loadFiles_args();
+ }
+
+ @Override
+ public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new org.apache.thrift.async.AsyncMethodCallback<Void>() {
+ @Override
+ public void onComplete(Void o) {
+ }
+ @Override
+ public void onError(java.lang.Exception e) {
+ if (e instanceof org.apache.thrift.transport.TTransportException) {
+ _LOGGER.error("TTransportException inside handler", e);
+ fb.close();
+ } else {
+ _LOGGER.error("Exception inside oneway handler", e);
+ }
+ }
+ };
+ }
+
+ @Override
+ protected boolean isOneway() {
+ return true;
+ }
+
+ @Override
+ public void start(I iface, loadFiles_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+ iface.loadFiles(args.tinfo, args.credentials, args.tid, args.dir, args.files, args.setTime,resultHandler);
+ }
+ }
+
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class startUpdate_args implements org.apache.thrift.TBase<startUpdate_args, startUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<startUpdate_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startUpdate_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+ private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)3);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startUpdate_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startUpdate_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+ /**
+ *
+ * @see TDurability
+ */
+ public @org.apache.thrift.annotation.Nullable TDurability durability; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)2, "tinfo"),
+ CREDENTIALS((short)1, "credentials"),
+ /**
+ *
+ * @see TDurability
+ */
+ DURABILITY((short)3, "durability");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 2: // TINFO
+ return TINFO;
+ case 1: // CREDENTIALS
+ return CREDENTIALS;
+ case 3: // DURABILITY
+ return DURABILITY;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+ tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startUpdate_args.class, metaDataMap);
+ }
+
+ public startUpdate_args() {
+ }
+
+ public startUpdate_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+ TDurability durability)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.durability = durability;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public startUpdate_args(startUpdate_args other) {
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ if (other.isSetCredentials()) {
+ this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+ }
+ if (other.isSetDurability()) {
+ this.durability = other.durability;
+ }
+ }
+
+ @Override
+ public startUpdate_args deepCopy() {
+ return new startUpdate_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ this.credentials = null;
+ this.durability = null;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public startUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+ return this.credentials;
+ }
+
+ public startUpdate_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+ this.credentials = credentials;
+ return this;
+ }
+
+ public void unsetCredentials() {
+ this.credentials = null;
+ }
+
+ /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+ public boolean isSetCredentials() {
+ return this.credentials != null;
+ }
+
+ public void setCredentialsIsSet(boolean value) {
+ if (!value) {
+ this.credentials = null;
+ }
+ }
+
+ /**
+ *
+ * @see TDurability
+ */
+ @org.apache.thrift.annotation.Nullable
+ public TDurability getDurability() {
+ return this.durability;
+ }
+
+ /**
+ *
+ * @see TDurability
+ */
+ public startUpdate_args setDurability(@org.apache.thrift.annotation.Nullable TDurability durability) {
+ this.durability = durability;
+ return this;
+ }
+
+ public void unsetDurability() {
+ this.durability = null;
+ }
+
+ /** Returns true if field durability is set (has been assigned a value) and false otherwise */
+ public boolean isSetDurability() {
+ return this.durability != null;
+ }
+
+ public void setDurabilityIsSet(boolean value) {
+ if (!value) {
+ this.durability = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case CREDENTIALS:
+ if (value == null) {
+ unsetCredentials();
+ } else {
+ setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+ }
+ break;
+
+ case DURABILITY:
+ if (value == null) {
+ unsetDurability();
+ } else {
+ setDurability((TDurability)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case CREDENTIALS:
+ return getCredentials();
+
+ case DURABILITY:
+ return getDurability();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case CREDENTIALS:
+ return isSetCredentials();
+ case DURABILITY:
+ return isSetDurability();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof startUpdate_args)
+ return this.equals((startUpdate_args)that);
+ return false;
+ }
+
+ public boolean equals(startUpdate_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_credentials = true && this.isSetCredentials();
+ boolean that_present_credentials = true && that.isSetCredentials();
+ if (this_present_credentials || that_present_credentials) {
+ if (!(this_present_credentials && that_present_credentials))
+ return false;
+ if (!this.credentials.equals(that.credentials))
+ return false;
+ }
+
+ boolean this_present_durability = true && this.isSetDurability();
+ boolean that_present_durability = true && that.isSetDurability();
+ if (this_present_durability || that_present_durability) {
+ if (!(this_present_durability && that_present_durability))
+ return false;
+ if (!this.durability.equals(that.durability))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+ if (isSetCredentials())
+ hashCode = hashCode * 8191 + credentials.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetDurability()) ? 131071 : 524287);
+ if (isSetDurability())
+ hashCode = hashCode * 8191 + durability.getValue();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(startUpdate_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCredentials()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetDurability(), other.isSetDurability());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetDurability()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("startUpdate_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("credentials:");
+ if (this.credentials == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.credentials);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("durability:");
+ if (this.durability == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.durability);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ if (credentials != null) {
+ credentials.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class startUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public startUpdate_argsStandardScheme getScheme() {
+ return new startUpdate_argsStandardScheme();
+ }
+ }
+
+ private static class startUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startUpdate_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, startUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 2: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 1: // CREDENTIALS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 3: // DURABILITY
+ if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+ struct.durability = org.apache.accumulo.core.tabletingest.thrift.TDurability.findByValue(iprot.readI32());
+ struct.setDurabilityIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, startUpdate_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.credentials != null) {
+ oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+ struct.credentials.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.durability != null) {
+ oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
+ oprot.writeI32(struct.durability.getValue());
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class startUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public startUpdate_argsTupleScheme getScheme() {
+ return new startUpdate_argsTupleScheme();
+ }
+ }
+
+ private static class startUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startUpdate_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, startUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetCredentials()) {
+ optionals.set(1);
+ }
+ if (struct.isSetDurability()) {
+ optionals.set(2);
+ }
+ oprot.writeBitSet(optionals, 3);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetCredentials()) {
+ struct.credentials.write(oprot);
+ }
+ if (struct.isSetDurability()) {
+ oprot.writeI32(struct.durability.getValue());
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, startUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(3);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ }
+ if (incoming.get(2)) {
+ struct.durability = org.apache.accumulo.core.tabletingest.thrift.TDurability.findByValue(iprot.readI32());
+ struct.setDurabilityIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class startUpdate_result implements org.apache.thrift.TBase<startUpdate_result, startUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<startUpdate_result> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startUpdate_result");
+
+ private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0);
+ private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startUpdate_resultStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startUpdate_resultTupleSchemeFactory();
+
+ public long success; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ SUCCESS((short)0, "success"),
+ SEC((short)1, "sec");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 0: // SUCCESS
+ return SUCCESS;
+ case 1: // SEC
+ return SEC;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __SUCCESS_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64 , "UpdateID")));
+ tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException.class)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startUpdate_result.class, metaDataMap);
+ }
+
+ public startUpdate_result() {
+ }
+
+ public startUpdate_result(
+ long success,
+ org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec)
+ {
+ this();
+ this.success = success;
+ setSuccessIsSet(true);
+ this.sec = sec;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public startUpdate_result(startUpdate_result other) {
+ __isset_bitfield = other.__isset_bitfield;
+ this.success = other.success;
+ if (other.isSetSec()) {
+ this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+ }
+ }
+
+ @Override
+ public startUpdate_result deepCopy() {
+ return new startUpdate_result(this);
+ }
+
+ @Override
+ public void clear() {
+ setSuccessIsSet(false);
+ this.success = 0;
+ this.sec = null;
+ }
+
+ public long getSuccess() {
+ return this.success;
+ }
+
+ public startUpdate_result setSuccess(long success) {
+ this.success = success;
+ setSuccessIsSet(true);
+ return this;
+ }
+
+ public void unsetSuccess() {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+ }
+
+ /** Returns true if field success is set (has been assigned a value) and false otherwise */
+ public boolean isSetSuccess() {
+ return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+ }
+
+ public void setSuccessIsSet(boolean value) {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
+ return this.sec;
+ }
+
+ public startUpdate_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+ this.sec = sec;
+ return this;
+ }
+
+ public void unsetSec() {
+ this.sec = null;
+ }
+
+ /** Returns true if field sec is set (has been assigned a value) and false otherwise */
+ public boolean isSetSec() {
+ return this.sec != null;
+ }
+
+ public void setSecIsSet(boolean value) {
+ if (!value) {
+ this.sec = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case SUCCESS:
+ if (value == null) {
+ unsetSuccess();
+ } else {
+ setSuccess((java.lang.Long)value);
+ }
+ break;
+
+ case SEC:
+ if (value == null) {
+ unsetSec();
+ } else {
+ setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case SUCCESS:
+ return getSuccess();
+
+ case SEC:
+ return getSec();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case SUCCESS:
+ return isSetSuccess();
+ case SEC:
+ return isSetSec();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof startUpdate_result)
+ return this.equals((startUpdate_result)that);
+ return false;
+ }
+
+ public boolean equals(startUpdate_result that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_success = true;
+ boolean that_present_success = true;
+ if (this_present_success || that_present_success) {
+ if (!(this_present_success && that_present_success))
+ return false;
+ if (this.success != that.success)
+ return false;
+ }
+
+ boolean this_present_sec = true && this.isSetSec();
+ boolean that_present_sec = true && that.isSetSec();
+ if (this_present_sec || that_present_sec) {
+ if (!(this_present_sec && that_present_sec))
+ return false;
+ if (!this.sec.equals(that.sec))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(success);
+
+ hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
+ if (isSetSec())
+ hashCode = hashCode * 8191 + sec.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(startUpdate_result other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSuccess()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetSec(), other.isSetSec());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSec()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("startUpdate_result(");
+ boolean first = true;
+
+ sb.append("success:");
+ sb.append(this.success);
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("sec:");
+ if (this.sec == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.sec);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class startUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public startUpdate_resultStandardScheme getScheme() {
+ return new startUpdate_resultStandardScheme();
+ }
+ }
+
+ private static class startUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startUpdate_result> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, startUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 0: // SUCCESS
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.success = iprot.readI64();
+ struct.setSuccessIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 1: // SEC
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+ struct.sec.read(iprot);
+ struct.setSecIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, startUpdate_result struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.isSetSuccess()) {
+ oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+ oprot.writeI64(struct.success);
+ oprot.writeFieldEnd();
+ }
+ if (struct.sec != null) {
+ oprot.writeFieldBegin(SEC_FIELD_DESC);
+ struct.sec.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class startUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public startUpdate_resultTupleScheme getScheme() {
+ return new startUpdate_resultTupleScheme();
+ }
+ }
+
+ private static class startUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startUpdate_result> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, startUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetSuccess()) {
+ optionals.set(0);
+ }
+ if (struct.isSetSec()) {
+ optionals.set(1);
+ }
+ oprot.writeBitSet(optionals, 2);
+ if (struct.isSetSuccess()) {
+ oprot.writeI64(struct.success);
+ }
+ if (struct.isSetSec()) {
+ struct.sec.write(oprot);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, startUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(2);
+ if (incoming.get(0)) {
+ struct.success = iprot.readI64();
+ struct.setSuccessIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+ struct.sec.read(iprot);
+ struct.setSecIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class applyUpdates_args implements org.apache.thrift.TBase<applyUpdates_args, applyUpdates_args._Fields>, java.io.Serializable, Cloneable, Comparable<applyUpdates_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("applyUpdates_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField UPDATE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("updateID", org.apache.thrift.protocol.TType.I64, (short)2);
+ private static final org.apache.thrift.protocol.TField KEY_EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("keyExtent", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+ private static final org.apache.thrift.protocol.TField MUTATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("mutations", org.apache.thrift.protocol.TType.LIST, (short)4);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new applyUpdates_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new applyUpdates_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public long updateID; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent; // required
+ public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)1, "tinfo"),
+ UPDATE_ID((short)2, "updateID"),
+ KEY_EXTENT((short)3, "keyExtent"),
+ MUTATIONS((short)4, "mutations");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // TINFO
+ return TINFO;
+ case 2: // UPDATE_ID
+ return UPDATE_ID;
+ case 3: // KEY_EXTENT
+ return KEY_EXTENT;
+ case 4: // MUTATIONS
+ return MUTATIONS;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __UPDATEID_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.UPDATE_ID, new org.apache.thrift.meta_data.FieldMetaData("updateID", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64 , "UpdateID")));
+ tmpMap.put(_Fields.KEY_EXTENT, new org.apache.thrift.meta_data.FieldMetaData("keyExtent", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class)));
+ tmpMap.put(_Fields.MUTATIONS, new org.apache.thrift.meta_data.FieldMetaData("mutations", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TMutation.class))));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(applyUpdates_args.class, metaDataMap);
+ }
+
+ public applyUpdates_args() {
+ }
+
+ public applyUpdates_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ long updateID,
+ org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent,
+ java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.updateID = updateID;
+ setUpdateIDIsSet(true);
+ this.keyExtent = keyExtent;
+ this.mutations = mutations;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public applyUpdates_args(applyUpdates_args other) {
+ __isset_bitfield = other.__isset_bitfield;
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ this.updateID = other.updateID;
+ if (other.isSetKeyExtent()) {
+ this.keyExtent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.keyExtent);
+ }
+ if (other.isSetMutations()) {
+ java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> __this__mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(other.mutations.size());
+ for (org.apache.accumulo.core.dataImpl.thrift.TMutation other_element : other.mutations) {
+ __this__mutations.add(new org.apache.accumulo.core.dataImpl.thrift.TMutation(other_element));
+ }
+ this.mutations = __this__mutations;
+ }
+ }
+
+ @Override
+ public applyUpdates_args deepCopy() {
+ return new applyUpdates_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ setUpdateIDIsSet(false);
+ this.updateID = 0;
+ this.keyExtent = null;
+ this.mutations = null;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public applyUpdates_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ public long getUpdateID() {
+ return this.updateID;
+ }
+
+ public applyUpdates_args setUpdateID(long updateID) {
+ this.updateID = updateID;
+ setUpdateIDIsSet(true);
+ return this;
+ }
+
+ public void unsetUpdateID() {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+ }
+
+ /** Returns true if field updateID is set (has been assigned a value) and false otherwise */
+ public boolean isSetUpdateID() {
+ return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+ }
+
+ public void setUpdateIDIsSet(boolean value) {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __UPDATEID_ISSET_ID, value);
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getKeyExtent() {
+ return this.keyExtent;
+ }
+
+ public applyUpdates_args setKeyExtent(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent) {
+ this.keyExtent = keyExtent;
+ return this;
+ }
+
+ public void unsetKeyExtent() {
+ this.keyExtent = null;
+ }
+
+ /** Returns true if field keyExtent is set (has been assigned a value) and false otherwise */
+ public boolean isSetKeyExtent() {
+ return this.keyExtent != null;
+ }
+
+ public void setKeyExtentIsSet(boolean value) {
+ if (!value) {
+ this.keyExtent = null;
+ }
+ }
+
+ public int getMutationsSize() {
+ return (this.mutations == null) ? 0 : this.mutations.size();
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.util.Iterator<org.apache.accumulo.core.dataImpl.thrift.TMutation> getMutationsIterator() {
+ return (this.mutations == null) ? null : this.mutations.iterator();
+ }
+
+ public void addToMutations(org.apache.accumulo.core.dataImpl.thrift.TMutation elem) {
+ if (this.mutations == null) {
+ this.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>();
+ }
+ this.mutations.add(elem);
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> getMutations() {
+ return this.mutations;
+ }
+
+ public applyUpdates_args setMutations(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation> mutations) {
+ this.mutations = mutations;
+ return this;
+ }
+
+ public void unsetMutations() {
+ this.mutations = null;
+ }
+
+ /** Returns true if field mutations is set (has been assigned a value) and false otherwise */
+ public boolean isSetMutations() {
+ return this.mutations != null;
+ }
+
+ public void setMutationsIsSet(boolean value) {
+ if (!value) {
+ this.mutations = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case UPDATE_ID:
+ if (value == null) {
+ unsetUpdateID();
+ } else {
+ setUpdateID((java.lang.Long)value);
+ }
+ break;
+
+ case KEY_EXTENT:
+ if (value == null) {
+ unsetKeyExtent();
+ } else {
+ setKeyExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+ }
+ break;
+
+ case MUTATIONS:
+ if (value == null) {
+ unsetMutations();
+ } else {
+ setMutations((java.util.List<org.apache.accumulo.core.dataImpl.thrift.TMutation>)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case UPDATE_ID:
+ return getUpdateID();
+
+ case KEY_EXTENT:
+ return getKeyExtent();
+
+ case MUTATIONS:
+ return getMutations();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case UPDATE_ID:
+ return isSetUpdateID();
+ case KEY_EXTENT:
+ return isSetKeyExtent();
+ case MUTATIONS:
+ return isSetMutations();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof applyUpdates_args)
+ return this.equals((applyUpdates_args)that);
+ return false;
+ }
+
+ public boolean equals(applyUpdates_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_updateID = true;
+ boolean that_present_updateID = true;
+ if (this_present_updateID || that_present_updateID) {
+ if (!(this_present_updateID && that_present_updateID))
+ return false;
+ if (this.updateID != that.updateID)
+ return false;
+ }
+
+ boolean this_present_keyExtent = true && this.isSetKeyExtent();
+ boolean that_present_keyExtent = true && that.isSetKeyExtent();
+ if (this_present_keyExtent || that_present_keyExtent) {
+ if (!(this_present_keyExtent && that_present_keyExtent))
+ return false;
+ if (!this.keyExtent.equals(that.keyExtent))
+ return false;
+ }
+
+ boolean this_present_mutations = true && this.isSetMutations();
+ boolean that_present_mutations = true && that.isSetMutations();
+ if (this_present_mutations || that_present_mutations) {
+ if (!(this_present_mutations && that_present_mutations))
+ return false;
+ if (!this.mutations.equals(that.mutations))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(updateID);
+
+ hashCode = hashCode * 8191 + ((isSetKeyExtent()) ? 131071 : 524287);
+ if (isSetKeyExtent())
+ hashCode = hashCode * 8191 + keyExtent.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetMutations()) ? 131071 : 524287);
+ if (isSetMutations())
+ hashCode = hashCode * 8191 + mutations.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(applyUpdates_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetUpdateID(), other.isSetUpdateID());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetUpdateID()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.updateID, other.updateID);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetKeyExtent(), other.isSetKeyExtent());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetKeyExtent()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keyExtent, other.keyExtent);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetMutations(), other.isSetMutations());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetMutations()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutations, other.mutations);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("applyUpdates_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("updateID:");
+ sb.append(this.updateID);
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("keyExtent:");
+ if (this.keyExtent == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.keyExtent);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("mutations:");
+ if (this.mutations == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.mutations);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ if (keyExtent != null) {
+ keyExtent.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class applyUpdates_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public applyUpdates_argsStandardScheme getScheme() {
+ return new applyUpdates_argsStandardScheme();
+ }
+ }
+
+ private static class applyUpdates_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<applyUpdates_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, applyUpdates_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // UPDATE_ID
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.updateID = iprot.readI64();
+ struct.setUpdateIDIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 3: // KEY_EXTENT
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.keyExtent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ struct.keyExtent.read(iprot);
+ struct.setKeyExtentIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 4: // MUTATIONS
+ if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+ {
+ org.apache.thrift.protocol.TList _list8 = iprot.readListBegin();
+ struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(_list8.size);
+ @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation _elem9;
+ for (int _i10 = 0; _i10 < _list8.size; ++_i10)
+ {
+ _elem9 = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
+ _elem9.read(iprot);
+ struct.mutations.add(_elem9);
+ }
+ iprot.readListEnd();
+ }
+ struct.setMutationsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, applyUpdates_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldBegin(UPDATE_ID_FIELD_DESC);
+ oprot.writeI64(struct.updateID);
+ oprot.writeFieldEnd();
+ if (struct.keyExtent != null) {
+ oprot.writeFieldBegin(KEY_EXTENT_FIELD_DESC);
+ struct.keyExtent.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.mutations != null) {
+ oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
+ {
+ oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mutations.size()));
+ for (org.apache.accumulo.core.dataImpl.thrift.TMutation _iter11 : struct.mutations)
+ {
+ _iter11.write(oprot);
+ }
+ oprot.writeListEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class applyUpdates_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public applyUpdates_argsTupleScheme getScheme() {
+ return new applyUpdates_argsTupleScheme();
+ }
+ }
+
+ private static class applyUpdates_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<applyUpdates_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, applyUpdates_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetUpdateID()) {
+ optionals.set(1);
+ }
+ if (struct.isSetKeyExtent()) {
+ optionals.set(2);
+ }
+ if (struct.isSetMutations()) {
+ optionals.set(3);
+ }
+ oprot.writeBitSet(optionals, 4);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetUpdateID()) {
+ oprot.writeI64(struct.updateID);
+ }
+ if (struct.isSetKeyExtent()) {
+ struct.keyExtent.write(oprot);
+ }
+ if (struct.isSetMutations()) {
+ {
+ oprot.writeI32(struct.mutations.size());
+ for (org.apache.accumulo.core.dataImpl.thrift.TMutation _iter12 : struct.mutations)
+ {
+ _iter12.write(oprot);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, applyUpdates_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(4);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.updateID = iprot.readI64();
+ struct.setUpdateIDIsSet(true);
+ }
+ if (incoming.get(2)) {
+ struct.keyExtent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ struct.keyExtent.read(iprot);
+ struct.setKeyExtentIsSet(true);
+ }
+ if (incoming.get(3)) {
+ {
+ org.apache.thrift.protocol.TList _list13 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+ struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(_list13.size);
+ @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation _elem14;
+ for (int _i15 = 0; _i15 < _list13.size; ++_i15)
+ {
+ _elem14 = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
+ _elem14.read(iprot);
+ struct.mutations.add(_elem14);
+ }
+ }
+ struct.setMutationsIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class closeUpdate_args implements org.apache.thrift.TBase<closeUpdate_args, closeUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<closeUpdate_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("closeUpdate_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+ private static final org.apache.thrift.protocol.TField UPDATE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("updateID", org.apache.thrift.protocol.TType.I64, (short)1);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new closeUpdate_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new closeUpdate_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public long updateID; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)2, "tinfo"),
+ UPDATE_ID((short)1, "updateID");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 2: // TINFO
+ return TINFO;
+ case 1: // UPDATE_ID
+ return UPDATE_ID;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __UPDATEID_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.UPDATE_ID, new org.apache.thrift.meta_data.FieldMetaData("updateID", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64 , "UpdateID")));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(closeUpdate_args.class, metaDataMap);
+ }
+
+ public closeUpdate_args() {
+ }
+
+ public closeUpdate_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ long updateID)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.updateID = updateID;
+ setUpdateIDIsSet(true);
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public closeUpdate_args(closeUpdate_args other) {
+ __isset_bitfield = other.__isset_bitfield;
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ this.updateID = other.updateID;
+ }
+
+ @Override
+ public closeUpdate_args deepCopy() {
+ return new closeUpdate_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ setUpdateIDIsSet(false);
+ this.updateID = 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public closeUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ public long getUpdateID() {
+ return this.updateID;
+ }
+
+ public closeUpdate_args setUpdateID(long updateID) {
+ this.updateID = updateID;
+ setUpdateIDIsSet(true);
+ return this;
+ }
+
+ public void unsetUpdateID() {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+ }
+
+ /** Returns true if field updateID is set (has been assigned a value) and false otherwise */
+ public boolean isSetUpdateID() {
+ return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+ }
+
+ public void setUpdateIDIsSet(boolean value) {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __UPDATEID_ISSET_ID, value);
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case UPDATE_ID:
+ if (value == null) {
+ unsetUpdateID();
+ } else {
+ setUpdateID((java.lang.Long)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case UPDATE_ID:
+ return getUpdateID();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case UPDATE_ID:
+ return isSetUpdateID();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof closeUpdate_args)
+ return this.equals((closeUpdate_args)that);
+ return false;
+ }
+
+ public boolean equals(closeUpdate_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_updateID = true;
+ boolean that_present_updateID = true;
+ if (this_present_updateID || that_present_updateID) {
+ if (!(this_present_updateID && that_present_updateID))
+ return false;
+ if (this.updateID != that.updateID)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(updateID);
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(closeUpdate_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetUpdateID(), other.isSetUpdateID());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetUpdateID()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.updateID, other.updateID);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("closeUpdate_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("updateID:");
+ sb.append(this.updateID);
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class closeUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public closeUpdate_argsStandardScheme getScheme() {
+ return new closeUpdate_argsStandardScheme();
+ }
+ }
+
+ private static class closeUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<closeUpdate_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, closeUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 2: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 1: // UPDATE_ID
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.updateID = iprot.readI64();
+ struct.setUpdateIDIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, closeUpdate_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ oprot.writeFieldBegin(UPDATE_ID_FIELD_DESC);
+ oprot.writeI64(struct.updateID);
+ oprot.writeFieldEnd();
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class closeUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public closeUpdate_argsTupleScheme getScheme() {
+ return new closeUpdate_argsTupleScheme();
+ }
+ }
+
+ private static class closeUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<closeUpdate_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, closeUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetUpdateID()) {
+ optionals.set(1);
+ }
+ oprot.writeBitSet(optionals, 2);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetUpdateID()) {
+ oprot.writeI64(struct.updateID);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, closeUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(2);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.updateID = iprot.readI64();
+ struct.setUpdateIDIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class closeUpdate_result implements org.apache.thrift.TBase<closeUpdate_result, closeUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<closeUpdate_result> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("closeUpdate_result");
+
+ private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+ private static final org.apache.thrift.protocol.TField NSSI_FIELD_DESC = new org.apache.thrift.protocol.TField("nssi", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new closeUpdate_resultStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new closeUpdate_resultTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.UpdateErrors success; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ SUCCESS((short)0, "success"),
+ NSSI((short)1, "nssi");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 0: // SUCCESS
+ return SUCCESS;
+ case 1: // NSSI
+ return NSSI;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.UpdateErrors.class)));
+ tmpMap.put(_Fields.NSSI, new org.apache.thrift.meta_data.FieldMetaData("nssi", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException.class)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(closeUpdate_result.class, metaDataMap);
+ }
+
+ public closeUpdate_result() {
+ }
+
+ public closeUpdate_result(
+ org.apache.accumulo.core.dataImpl.thrift.UpdateErrors success,
+ org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi)
+ {
+ this();
+ this.success = success;
+ this.nssi = nssi;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public closeUpdate_result(closeUpdate_result other) {
+ if (other.isSetSuccess()) {
+ this.success = new org.apache.accumulo.core.dataImpl.thrift.UpdateErrors(other.success);
+ }
+ if (other.isSetNssi()) {
+ this.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException(other.nssi);
+ }
+ }
+
+ @Override
+ public closeUpdate_result deepCopy() {
+ return new closeUpdate_result(this);
+ }
+
+ @Override
+ public void clear() {
+ this.success = null;
+ this.nssi = null;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.dataImpl.thrift.UpdateErrors getSuccess() {
+ return this.success;
+ }
+
+ public closeUpdate_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.UpdateErrors success) {
+ this.success = success;
+ return this;
+ }
+
+ public void unsetSuccess() {
+ this.success = null;
+ }
+
+ /** Returns true if field success is set (has been assigned a value) and false otherwise */
+ public boolean isSetSuccess() {
+ return this.success != null;
+ }
+
+ public void setSuccessIsSet(boolean value) {
+ if (!value) {
+ this.success = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException getNssi() {
+ return this.nssi;
+ }
+
+ public closeUpdate_result setNssi(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
+ this.nssi = nssi;
+ return this;
+ }
+
+ public void unsetNssi() {
+ this.nssi = null;
+ }
+
+ /** Returns true if field nssi is set (has been assigned a value) and false otherwise */
+ public boolean isSetNssi() {
+ return this.nssi != null;
+ }
+
+ public void setNssiIsSet(boolean value) {
+ if (!value) {
+ this.nssi = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case SUCCESS:
+ if (value == null) {
+ unsetSuccess();
+ } else {
+ setSuccess((org.apache.accumulo.core.dataImpl.thrift.UpdateErrors)value);
+ }
+ break;
+
+ case NSSI:
+ if (value == null) {
+ unsetNssi();
+ } else {
+ setNssi((org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case SUCCESS:
+ return getSuccess();
+
+ case NSSI:
+ return getNssi();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case SUCCESS:
+ return isSetSuccess();
+ case NSSI:
+ return isSetNssi();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof closeUpdate_result)
+ return this.equals((closeUpdate_result)that);
+ return false;
+ }
+
+ public boolean equals(closeUpdate_result that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_success = true && this.isSetSuccess();
+ boolean that_present_success = true && that.isSetSuccess();
+ if (this_present_success || that_present_success) {
+ if (!(this_present_success && that_present_success))
+ return false;
+ if (!this.success.equals(that.success))
+ return false;
+ }
+
+ boolean this_present_nssi = true && this.isSetNssi();
+ boolean that_present_nssi = true && that.isSetNssi();
+ if (this_present_nssi || that_present_nssi) {
+ if (!(this_present_nssi && that_present_nssi))
+ return false;
+ if (!this.nssi.equals(that.nssi))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+ if (isSetSuccess())
+ hashCode = hashCode * 8191 + success.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetNssi()) ? 131071 : 524287);
+ if (isSetNssi())
+ hashCode = hashCode * 8191 + nssi.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(closeUpdate_result other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSuccess()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetNssi(), other.isSetNssi());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetNssi()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nssi, other.nssi);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("closeUpdate_result(");
+ boolean first = true;
+
+ sb.append("success:");
+ if (this.success == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.success);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("nssi:");
+ if (this.nssi == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.nssi);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (success != null) {
+ success.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class closeUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public closeUpdate_resultStandardScheme getScheme() {
+ return new closeUpdate_resultStandardScheme();
+ }
+ }
+
+ private static class closeUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<closeUpdate_result> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, closeUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 0: // SUCCESS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.success = new org.apache.accumulo.core.dataImpl.thrift.UpdateErrors();
+ struct.success.read(iprot);
+ struct.setSuccessIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 1: // NSSI
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
+ struct.nssi.read(iprot);
+ struct.setNssiIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, closeUpdate_result struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.success != null) {
+ oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+ struct.success.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.nssi != null) {
+ oprot.writeFieldBegin(NSSI_FIELD_DESC);
+ struct.nssi.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class closeUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public closeUpdate_resultTupleScheme getScheme() {
+ return new closeUpdate_resultTupleScheme();
+ }
+ }
+
+ private static class closeUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<closeUpdate_result> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, closeUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetSuccess()) {
+ optionals.set(0);
+ }
+ if (struct.isSetNssi()) {
+ optionals.set(1);
+ }
+ oprot.writeBitSet(optionals, 2);
+ if (struct.isSetSuccess()) {
+ struct.success.write(oprot);
+ }
+ if (struct.isSetNssi()) {
+ struct.nssi.write(oprot);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, closeUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(2);
+ if (incoming.get(0)) {
+ struct.success = new org.apache.accumulo.core.dataImpl.thrift.UpdateErrors();
+ struct.success.read(iprot);
+ struct.setSuccessIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
+ struct.nssi.read(iprot);
+ struct.setNssiIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class cancelUpdate_args implements org.apache.thrift.TBase<cancelUpdate_args, cancelUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<cancelUpdate_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cancelUpdate_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField UPDATE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("updateID", org.apache.thrift.protocol.TType.I64, (short)2);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelUpdate_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelUpdate_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public long updateID; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)1, "tinfo"),
+ UPDATE_ID((short)2, "updateID");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // TINFO
+ return TINFO;
+ case 2: // UPDATE_ID
+ return UPDATE_ID;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __UPDATEID_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.UPDATE_ID, new org.apache.thrift.meta_data.FieldMetaData("updateID", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64 , "UpdateID")));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelUpdate_args.class, metaDataMap);
+ }
+
+ public cancelUpdate_args() {
+ }
+
+ public cancelUpdate_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ long updateID)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.updateID = updateID;
+ setUpdateIDIsSet(true);
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public cancelUpdate_args(cancelUpdate_args other) {
+ __isset_bitfield = other.__isset_bitfield;
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ this.updateID = other.updateID;
+ }
+
+ @Override
+ public cancelUpdate_args deepCopy() {
+ return new cancelUpdate_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ setUpdateIDIsSet(false);
+ this.updateID = 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public cancelUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ public long getUpdateID() {
+ return this.updateID;
+ }
+
+ public cancelUpdate_args setUpdateID(long updateID) {
+ this.updateID = updateID;
+ setUpdateIDIsSet(true);
+ return this;
+ }
+
+ public void unsetUpdateID() {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+ }
+
+ /** Returns true if field updateID is set (has been assigned a value) and false otherwise */
+ public boolean isSetUpdateID() {
+ return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __UPDATEID_ISSET_ID);
+ }
+
+ public void setUpdateIDIsSet(boolean value) {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __UPDATEID_ISSET_ID, value);
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case UPDATE_ID:
+ if (value == null) {
+ unsetUpdateID();
+ } else {
+ setUpdateID((java.lang.Long)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case UPDATE_ID:
+ return getUpdateID();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case UPDATE_ID:
+ return isSetUpdateID();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof cancelUpdate_args)
+ return this.equals((cancelUpdate_args)that);
+ return false;
+ }
+
+ public boolean equals(cancelUpdate_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_updateID = true;
+ boolean that_present_updateID = true;
+ if (this_present_updateID || that_present_updateID) {
+ if (!(this_present_updateID && that_present_updateID))
+ return false;
+ if (this.updateID != that.updateID)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(updateID);
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(cancelUpdate_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetUpdateID(), other.isSetUpdateID());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetUpdateID()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.updateID, other.updateID);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("cancelUpdate_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("updateID:");
+ sb.append(this.updateID);
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class cancelUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public cancelUpdate_argsStandardScheme getScheme() {
+ return new cancelUpdate_argsStandardScheme();
+ }
+ }
+
+ private static class cancelUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<cancelUpdate_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, cancelUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // UPDATE_ID
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.updateID = iprot.readI64();
+ struct.setUpdateIDIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, cancelUpdate_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldBegin(UPDATE_ID_FIELD_DESC);
+ oprot.writeI64(struct.updateID);
+ oprot.writeFieldEnd();
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class cancelUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public cancelUpdate_argsTupleScheme getScheme() {
+ return new cancelUpdate_argsTupleScheme();
+ }
+ }
+
+ private static class cancelUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<cancelUpdate_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, cancelUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetUpdateID()) {
+ optionals.set(1);
+ }
+ oprot.writeBitSet(optionals, 2);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetUpdateID()) {
+ oprot.writeI64(struct.updateID);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, cancelUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(2);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.updateID = iprot.readI64();
+ struct.setUpdateIDIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class cancelUpdate_result implements org.apache.thrift.TBase<cancelUpdate_result, cancelUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<cancelUpdate_result> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cancelUpdate_result");
+
+ private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelUpdate_resultStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelUpdate_resultTupleSchemeFactory();
+
+ public boolean success; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ SUCCESS((short)0, "success");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 0: // SUCCESS
+ return SUCCESS;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __SUCCESS_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelUpdate_result.class, metaDataMap);
+ }
+
+ public cancelUpdate_result() {
+ }
+
+ public cancelUpdate_result(
+ boolean success)
+ {
+ this();
+ this.success = success;
+ setSuccessIsSet(true);
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public cancelUpdate_result(cancelUpdate_result other) {
+ __isset_bitfield = other.__isset_bitfield;
+ this.success = other.success;
+ }
+
+ @Override
+ public cancelUpdate_result deepCopy() {
+ return new cancelUpdate_result(this);
+ }
+
+ @Override
+ public void clear() {
+ setSuccessIsSet(false);
+ this.success = false;
+ }
+
+ public boolean isSuccess() {
+ return this.success;
+ }
+
+ public cancelUpdate_result setSuccess(boolean success) {
+ this.success = success;
+ setSuccessIsSet(true);
+ return this;
+ }
+
+ public void unsetSuccess() {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+ }
+
+ /** Returns true if field success is set (has been assigned a value) and false otherwise */
+ public boolean isSetSuccess() {
+ return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+ }
+
+ public void setSuccessIsSet(boolean value) {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case SUCCESS:
+ if (value == null) {
+ unsetSuccess();
+ } else {
+ setSuccess((java.lang.Boolean)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case SUCCESS:
+ return isSuccess();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case SUCCESS:
+ return isSetSuccess();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof cancelUpdate_result)
+ return this.equals((cancelUpdate_result)that);
+ return false;
+ }
+
+ public boolean equals(cancelUpdate_result that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_success = true;
+ boolean that_present_success = true;
+ if (this_present_success || that_present_success) {
+ if (!(this_present_success && that_present_success))
+ return false;
+ if (this.success != that.success)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((success) ? 131071 : 524287);
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(cancelUpdate_result other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSuccess()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("cancelUpdate_result(");
+ boolean first = true;
+
+ sb.append("success:");
+ sb.append(this.success);
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class cancelUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public cancelUpdate_resultStandardScheme getScheme() {
+ return new cancelUpdate_resultStandardScheme();
+ }
+ }
+
+ private static class cancelUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<cancelUpdate_result> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, cancelUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 0: // SUCCESS
+ if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+ struct.success = iprot.readBool();
+ struct.setSuccessIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, cancelUpdate_result struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.isSetSuccess()) {
+ oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+ oprot.writeBool(struct.success);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class cancelUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public cancelUpdate_resultTupleScheme getScheme() {
+ return new cancelUpdate_resultTupleScheme();
+ }
+ }
+
+ private static class cancelUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<cancelUpdate_result> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, cancelUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetSuccess()) {
+ optionals.set(0);
+ }
+ oprot.writeBitSet(optionals, 1);
+ if (struct.isSetSuccess()) {
+ oprot.writeBool(struct.success);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, cancelUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(1);
+ if (incoming.get(0)) {
+ struct.success = iprot.readBool();
+ struct.setSuccessIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class startConditionalUpdate_args implements org.apache.thrift.TBase<startConditionalUpdate_args, startConditionalUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<startConditionalUpdate_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startConditionalUpdate_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+ private static final org.apache.thrift.protocol.TField AUTHORIZATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizations", org.apache.thrift.protocol.TType.LIST, (short)3);
+ private static final org.apache.thrift.protocol.TField TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableID", org.apache.thrift.protocol.TType.STRING, (short)4);
+ private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)5);
+ private static final org.apache.thrift.protocol.TField CLASS_LOADER_CONTEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("classLoaderContext", org.apache.thrift.protocol.TType.STRING, (short)6);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startConditionalUpdate_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startConditionalUpdate_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
+ public @org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> authorizations; // required
+ public @org.apache.thrift.annotation.Nullable java.lang.String tableID; // required
+ /**
+ *
+ * @see TDurability
+ */
+ public @org.apache.thrift.annotation.Nullable TDurability durability; // required
+ public @org.apache.thrift.annotation.Nullable java.lang.String classLoaderContext; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)1, "tinfo"),
+ CREDENTIALS((short)2, "credentials"),
+ AUTHORIZATIONS((short)3, "authorizations"),
+ TABLE_ID((short)4, "tableID"),
+ /**
+ *
+ * @see TDurability
+ */
+ DURABILITY((short)5, "durability"),
+ CLASS_LOADER_CONTEXT((short)6, "classLoaderContext");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // TINFO
+ return TINFO;
+ case 2: // CREDENTIALS
+ return CREDENTIALS;
+ case 3: // AUTHORIZATIONS
+ return AUTHORIZATIONS;
+ case 4: // TABLE_ID
+ return TABLE_ID;
+ case 5: // DURABILITY
+ return DURABILITY;
+ case 6: // CLASS_LOADER_CONTEXT
+ return CLASS_LOADER_CONTEXT;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
+ tmpMap.put(_Fields.AUTHORIZATIONS, new org.apache.thrift.meta_data.FieldMetaData("authorizations", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))));
+ tmpMap.put(_Fields.TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("tableID", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+ tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
+ tmpMap.put(_Fields.CLASS_LOADER_CONTEXT, new org.apache.thrift.meta_data.FieldMetaData("classLoaderContext", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startConditionalUpdate_args.class, metaDataMap);
+ }
+
+ public startConditionalUpdate_args() {
+ }
+
+ public startConditionalUpdate_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+ java.util.List<java.nio.ByteBuffer> authorizations,
+ java.lang.String tableID,
+ TDurability durability,
+ java.lang.String classLoaderContext)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.authorizations = authorizations;
+ this.tableID = tableID;
+ this.durability = durability;
+ this.classLoaderContext = classLoaderContext;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public startConditionalUpdate_args(startConditionalUpdate_args other) {
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ if (other.isSetCredentials()) {
+ this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
+ }
+ if (other.isSetAuthorizations()) {
+ java.util.List<java.nio.ByteBuffer> __this__authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(other.authorizations);
+ this.authorizations = __this__authorizations;
+ }
+ if (other.isSetTableID()) {
+ this.tableID = other.tableID;
+ }
+ if (other.isSetDurability()) {
+ this.durability = other.durability;
+ }
+ if (other.isSetClassLoaderContext()) {
+ this.classLoaderContext = other.classLoaderContext;
+ }
+ }
+
+ @Override
+ public startConditionalUpdate_args deepCopy() {
+ return new startConditionalUpdate_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ this.credentials = null;
+ this.authorizations = null;
+ this.tableID = null;
+ this.durability = null;
+ this.classLoaderContext = null;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public startConditionalUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() {
+ return this.credentials;
+ }
+
+ public startConditionalUpdate_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) {
+ this.credentials = credentials;
+ return this;
+ }
+
+ public void unsetCredentials() {
+ this.credentials = null;
+ }
+
+ /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+ public boolean isSetCredentials() {
+ return this.credentials != null;
+ }
+
+ public void setCredentialsIsSet(boolean value) {
+ if (!value) {
+ this.credentials = null;
+ }
+ }
+
+ public int getAuthorizationsSize() {
+ return (this.authorizations == null) ? 0 : this.authorizations.size();
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.util.Iterator<java.nio.ByteBuffer> getAuthorizationsIterator() {
+ return (this.authorizations == null) ? null : this.authorizations.iterator();
+ }
+
+ public void addToAuthorizations(java.nio.ByteBuffer elem) {
+ if (this.authorizations == null) {
+ this.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>();
+ }
+ this.authorizations.add(elem);
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.util.List<java.nio.ByteBuffer> getAuthorizations() {
+ return this.authorizations;
+ }
+
+ public startConditionalUpdate_args setAuthorizations(@org.apache.thrift.annotation.Nullable java.util.List<java.nio.ByteBuffer> authorizations) {
+ this.authorizations = authorizations;
+ return this;
+ }
+
+ public void unsetAuthorizations() {
+ this.authorizations = null;
+ }
+
+ /** Returns true if field authorizations is set (has been assigned a value) and false otherwise */
+ public boolean isSetAuthorizations() {
+ return this.authorizations != null;
+ }
+
+ public void setAuthorizationsIsSet(boolean value) {
+ if (!value) {
+ this.authorizations = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.lang.String getTableID() {
+ return this.tableID;
+ }
+
+ public startConditionalUpdate_args setTableID(@org.apache.thrift.annotation.Nullable java.lang.String tableID) {
+ this.tableID = tableID;
+ return this;
+ }
+
+ public void unsetTableID() {
+ this.tableID = null;
+ }
+
+ /** Returns true if field tableID is set (has been assigned a value) and false otherwise */
+ public boolean isSetTableID() {
+ return this.tableID != null;
+ }
+
+ public void setTableIDIsSet(boolean value) {
+ if (!value) {
+ this.tableID = null;
+ }
+ }
+
+ /**
+ *
+ * @see TDurability
+ */
+ @org.apache.thrift.annotation.Nullable
+ public TDurability getDurability() {
+ return this.durability;
+ }
+
+ /**
+ *
+ * @see TDurability
+ */
+ public startConditionalUpdate_args setDurability(@org.apache.thrift.annotation.Nullable TDurability durability) {
+ this.durability = durability;
+ return this;
+ }
+
+ public void unsetDurability() {
+ this.durability = null;
+ }
+
+ /** Returns true if field durability is set (has been assigned a value) and false otherwise */
+ public boolean isSetDurability() {
+ return this.durability != null;
+ }
+
+ public void setDurabilityIsSet(boolean value) {
+ if (!value) {
+ this.durability = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.lang.String getClassLoaderContext() {
+ return this.classLoaderContext;
+ }
+
+ public startConditionalUpdate_args setClassLoaderContext(@org.apache.thrift.annotation.Nullable java.lang.String classLoaderContext) {
+ this.classLoaderContext = classLoaderContext;
+ return this;
+ }
+
+ public void unsetClassLoaderContext() {
+ this.classLoaderContext = null;
+ }
+
+ /** Returns true if field classLoaderContext is set (has been assigned a value) and false otherwise */
+ public boolean isSetClassLoaderContext() {
+ return this.classLoaderContext != null;
+ }
+
+ public void setClassLoaderContextIsSet(boolean value) {
+ if (!value) {
+ this.classLoaderContext = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case CREDENTIALS:
+ if (value == null) {
+ unsetCredentials();
+ } else {
+ setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
+ }
+ break;
+
+ case AUTHORIZATIONS:
+ if (value == null) {
+ unsetAuthorizations();
+ } else {
+ setAuthorizations((java.util.List<java.nio.ByteBuffer>)value);
+ }
+ break;
+
+ case TABLE_ID:
+ if (value == null) {
+ unsetTableID();
+ } else {
+ setTableID((java.lang.String)value);
+ }
+ break;
+
+ case DURABILITY:
+ if (value == null) {
+ unsetDurability();
+ } else {
+ setDurability((TDurability)value);
+ }
+ break;
+
+ case CLASS_LOADER_CONTEXT:
+ if (value == null) {
+ unsetClassLoaderContext();
+ } else {
+ setClassLoaderContext((java.lang.String)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case CREDENTIALS:
+ return getCredentials();
+
+ case AUTHORIZATIONS:
+ return getAuthorizations();
+
+ case TABLE_ID:
+ return getTableID();
+
+ case DURABILITY:
+ return getDurability();
+
+ case CLASS_LOADER_CONTEXT:
+ return getClassLoaderContext();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case CREDENTIALS:
+ return isSetCredentials();
+ case AUTHORIZATIONS:
+ return isSetAuthorizations();
+ case TABLE_ID:
+ return isSetTableID();
+ case DURABILITY:
+ return isSetDurability();
+ case CLASS_LOADER_CONTEXT:
+ return isSetClassLoaderContext();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof startConditionalUpdate_args)
+ return this.equals((startConditionalUpdate_args)that);
+ return false;
+ }
+
+ public boolean equals(startConditionalUpdate_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_credentials = true && this.isSetCredentials();
+ boolean that_present_credentials = true && that.isSetCredentials();
+ if (this_present_credentials || that_present_credentials) {
+ if (!(this_present_credentials && that_present_credentials))
+ return false;
+ if (!this.credentials.equals(that.credentials))
+ return false;
+ }
+
+ boolean this_present_authorizations = true && this.isSetAuthorizations();
+ boolean that_present_authorizations = true && that.isSetAuthorizations();
+ if (this_present_authorizations || that_present_authorizations) {
+ if (!(this_present_authorizations && that_present_authorizations))
+ return false;
+ if (!this.authorizations.equals(that.authorizations))
+ return false;
+ }
+
+ boolean this_present_tableID = true && this.isSetTableID();
+ boolean that_present_tableID = true && that.isSetTableID();
+ if (this_present_tableID || that_present_tableID) {
+ if (!(this_present_tableID && that_present_tableID))
+ return false;
+ if (!this.tableID.equals(that.tableID))
+ return false;
+ }
+
+ boolean this_present_durability = true && this.isSetDurability();
+ boolean that_present_durability = true && that.isSetDurability();
+ if (this_present_durability || that_present_durability) {
+ if (!(this_present_durability && that_present_durability))
+ return false;
+ if (!this.durability.equals(that.durability))
+ return false;
+ }
+
+ boolean this_present_classLoaderContext = true && this.isSetClassLoaderContext();
+ boolean that_present_classLoaderContext = true && that.isSetClassLoaderContext();
+ if (this_present_classLoaderContext || that_present_classLoaderContext) {
+ if (!(this_present_classLoaderContext && that_present_classLoaderContext))
+ return false;
+ if (!this.classLoaderContext.equals(that.classLoaderContext))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+ if (isSetCredentials())
+ hashCode = hashCode * 8191 + credentials.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetAuthorizations()) ? 131071 : 524287);
+ if (isSetAuthorizations())
+ hashCode = hashCode * 8191 + authorizations.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetTableID()) ? 131071 : 524287);
+ if (isSetTableID())
+ hashCode = hashCode * 8191 + tableID.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetDurability()) ? 131071 : 524287);
+ if (isSetDurability())
+ hashCode = hashCode * 8191 + durability.getValue();
+
+ hashCode = hashCode * 8191 + ((isSetClassLoaderContext()) ? 131071 : 524287);
+ if (isSetClassLoaderContext())
+ hashCode = hashCode * 8191 + classLoaderContext.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(startConditionalUpdate_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCredentials()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetAuthorizations(), other.isSetAuthorizations());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetAuthorizations()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.authorizations, other.authorizations);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetTableID(), other.isSetTableID());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTableID()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableID, other.tableID);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetDurability(), other.isSetDurability());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetDurability()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetClassLoaderContext(), other.isSetClassLoaderContext());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetClassLoaderContext()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.classLoaderContext, other.classLoaderContext);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("startConditionalUpdate_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("credentials:");
+ if (this.credentials == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.credentials);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("authorizations:");
+ if (this.authorizations == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.authorizations, sb);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("tableID:");
+ if (this.tableID == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tableID);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("durability:");
+ if (this.durability == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.durability);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("classLoaderContext:");
+ if (this.classLoaderContext == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.classLoaderContext);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ if (credentials != null) {
+ credentials.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class startConditionalUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public startConditionalUpdate_argsStandardScheme getScheme() {
+ return new startConditionalUpdate_argsStandardScheme();
+ }
+ }
+
+ private static class startConditionalUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startConditionalUpdate_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // CREDENTIALS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 3: // AUTHORIZATIONS
+ if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+ {
+ org.apache.thrift.protocol.TList _list16 = iprot.readListBegin();
+ struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list16.size);
+ @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem17;
+ for (int _i18 = 0; _i18 < _list16.size; ++_i18)
+ {
+ _elem17 = iprot.readBinary();
+ struct.authorizations.add(_elem17);
+ }
+ iprot.readListEnd();
+ }
+ struct.setAuthorizationsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 4: // TABLE_ID
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.tableID = iprot.readString();
+ struct.setTableIDIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 5: // DURABILITY
+ if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+ struct.durability = org.apache.accumulo.core.tabletingest.thrift.TDurability.findByValue(iprot.readI32());
+ struct.setDurabilityIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 6: // CLASS_LOADER_CONTEXT
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.classLoaderContext = iprot.readString();
+ struct.setClassLoaderContextIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.credentials != null) {
+ oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+ struct.credentials.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.authorizations != null) {
+ oprot.writeFieldBegin(AUTHORIZATIONS_FIELD_DESC);
+ {
+ oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.authorizations.size()));
+ for (java.nio.ByteBuffer _iter19 : struct.authorizations)
+ {
+ oprot.writeBinary(_iter19);
+ }
+ oprot.writeListEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ if (struct.tableID != null) {
+ oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
+ oprot.writeString(struct.tableID);
+ oprot.writeFieldEnd();
+ }
+ if (struct.durability != null) {
+ oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
+ oprot.writeI32(struct.durability.getValue());
+ oprot.writeFieldEnd();
+ }
+ if (struct.classLoaderContext != null) {
+ oprot.writeFieldBegin(CLASS_LOADER_CONTEXT_FIELD_DESC);
+ oprot.writeString(struct.classLoaderContext);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class startConditionalUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public startConditionalUpdate_argsTupleScheme getScheme() {
+ return new startConditionalUpdate_argsTupleScheme();
+ }
+ }
+
+ private static class startConditionalUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startConditionalUpdate_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetCredentials()) {
+ optionals.set(1);
+ }
+ if (struct.isSetAuthorizations()) {
+ optionals.set(2);
+ }
+ if (struct.isSetTableID()) {
+ optionals.set(3);
+ }
+ if (struct.isSetDurability()) {
+ optionals.set(4);
+ }
+ if (struct.isSetClassLoaderContext()) {
+ optionals.set(5);
+ }
+ oprot.writeBitSet(optionals, 6);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetCredentials()) {
+ struct.credentials.write(oprot);
+ }
+ if (struct.isSetAuthorizations()) {
+ {
+ oprot.writeI32(struct.authorizations.size());
+ for (java.nio.ByteBuffer _iter20 : struct.authorizations)
+ {
+ oprot.writeBinary(_iter20);
+ }
+ }
+ }
+ if (struct.isSetTableID()) {
+ oprot.writeString(struct.tableID);
+ }
+ if (struct.isSetDurability()) {
+ oprot.writeI32(struct.durability.getValue());
+ }
+ if (struct.isSetClassLoaderContext()) {
+ oprot.writeString(struct.classLoaderContext);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(6);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ }
+ if (incoming.get(2)) {
+ {
+ org.apache.thrift.protocol.TList _list21 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+ struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list21.size);
+ @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem22;
+ for (int _i23 = 0; _i23 < _list21.size; ++_i23)
+ {
+ _elem22 = iprot.readBinary();
+ struct.authorizations.add(_elem22);
+ }
+ }
+ struct.setAuthorizationsIsSet(true);
+ }
+ if (incoming.get(3)) {
+ struct.tableID = iprot.readString();
+ struct.setTableIDIsSet(true);
+ }
+ if (incoming.get(4)) {
+ struct.durability = org.apache.accumulo.core.tabletingest.thrift.TDurability.findByValue(iprot.readI32());
+ struct.setDurabilityIsSet(true);
+ }
+ if (incoming.get(5)) {
+ struct.classLoaderContext = iprot.readString();
+ struct.setClassLoaderContextIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class startConditionalUpdate_result implements org.apache.thrift.TBase<startConditionalUpdate_result, startConditionalUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<startConditionalUpdate_result> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startConditionalUpdate_result");
+
+ private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+ private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startConditionalUpdate_resultStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startConditionalUpdate_resultTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalSession success; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ SUCCESS((short)0, "success"),
+ SEC((short)1, "sec");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 0: // SUCCESS
+ return SUCCESS;
+ case 1: // SEC
+ return SEC;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TConditionalSession.class)));
+ tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException.class)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startConditionalUpdate_result.class, metaDataMap);
+ }
+
+ public startConditionalUpdate_result() {
+ }
+
+ public startConditionalUpdate_result(
+ org.apache.accumulo.core.dataImpl.thrift.TConditionalSession success,
+ org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec)
+ {
+ this();
+ this.success = success;
+ this.sec = sec;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public startConditionalUpdate_result(startConditionalUpdate_result other) {
+ if (other.isSetSuccess()) {
+ this.success = new org.apache.accumulo.core.dataImpl.thrift.TConditionalSession(other.success);
+ }
+ if (other.isSetSec()) {
+ this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+ }
+ }
+
+ @Override
+ public startConditionalUpdate_result deepCopy() {
+ return new startConditionalUpdate_result(this);
+ }
+
+ @Override
+ public void clear() {
+ this.success = null;
+ this.sec = null;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession getSuccess() {
+ return this.success;
+ }
+
+ public startConditionalUpdate_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalSession success) {
+ this.success = success;
+ return this;
+ }
+
+ public void unsetSuccess() {
+ this.success = null;
+ }
+
+ /** Returns true if field success is set (has been assigned a value) and false otherwise */
+ public boolean isSetSuccess() {
+ return this.success != null;
+ }
+
+ public void setSuccessIsSet(boolean value) {
+ if (!value) {
+ this.success = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
+ return this.sec;
+ }
+
+ public startConditionalUpdate_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+ this.sec = sec;
+ return this;
+ }
+
+ public void unsetSec() {
+ this.sec = null;
+ }
+
+ /** Returns true if field sec is set (has been assigned a value) and false otherwise */
+ public boolean isSetSec() {
+ return this.sec != null;
+ }
+
+ public void setSecIsSet(boolean value) {
+ if (!value) {
+ this.sec = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case SUCCESS:
+ if (value == null) {
+ unsetSuccess();
+ } else {
+ setSuccess((org.apache.accumulo.core.dataImpl.thrift.TConditionalSession)value);
+ }
+ break;
+
+ case SEC:
+ if (value == null) {
+ unsetSec();
+ } else {
+ setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case SUCCESS:
+ return getSuccess();
+
+ case SEC:
+ return getSec();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case SUCCESS:
+ return isSetSuccess();
+ case SEC:
+ return isSetSec();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof startConditionalUpdate_result)
+ return this.equals((startConditionalUpdate_result)that);
+ return false;
+ }
+
+ public boolean equals(startConditionalUpdate_result that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_success = true && this.isSetSuccess();
+ boolean that_present_success = true && that.isSetSuccess();
+ if (this_present_success || that_present_success) {
+ if (!(this_present_success && that_present_success))
+ return false;
+ if (!this.success.equals(that.success))
+ return false;
+ }
+
+ boolean this_present_sec = true && this.isSetSec();
+ boolean that_present_sec = true && that.isSetSec();
+ if (this_present_sec || that_present_sec) {
+ if (!(this_present_sec && that_present_sec))
+ return false;
+ if (!this.sec.equals(that.sec))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+ if (isSetSuccess())
+ hashCode = hashCode * 8191 + success.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
+ if (isSetSec())
+ hashCode = hashCode * 8191 + sec.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(startConditionalUpdate_result other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSuccess()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetSec(), other.isSetSec());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSec()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("startConditionalUpdate_result(");
+ boolean first = true;
+
+ sb.append("success:");
+ if (this.success == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.success);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("sec:");
+ if (this.sec == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.sec);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (success != null) {
+ success.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class startConditionalUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public startConditionalUpdate_resultStandardScheme getScheme() {
+ return new startConditionalUpdate_resultStandardScheme();
+ }
+ }
+
+ private static class startConditionalUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startConditionalUpdate_result> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 0: // SUCCESS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.success = new org.apache.accumulo.core.dataImpl.thrift.TConditionalSession();
+ struct.success.read(iprot);
+ struct.setSuccessIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 1: // SEC
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+ struct.sec.read(iprot);
+ struct.setSecIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.success != null) {
+ oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+ struct.success.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.sec != null) {
+ oprot.writeFieldBegin(SEC_FIELD_DESC);
+ struct.sec.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class startConditionalUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public startConditionalUpdate_resultTupleScheme getScheme() {
+ return new startConditionalUpdate_resultTupleScheme();
+ }
+ }
+
+ private static class startConditionalUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startConditionalUpdate_result> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetSuccess()) {
+ optionals.set(0);
+ }
+ if (struct.isSetSec()) {
+ optionals.set(1);
+ }
+ oprot.writeBitSet(optionals, 2);
+ if (struct.isSetSuccess()) {
+ struct.success.write(oprot);
+ }
+ if (struct.isSetSec()) {
+ struct.sec.write(oprot);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(2);
+ if (incoming.get(0)) {
+ struct.success = new org.apache.accumulo.core.dataImpl.thrift.TConditionalSession();
+ struct.success.read(iprot);
+ struct.setSuccessIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
+ struct.sec.read(iprot);
+ struct.setSecIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class conditionalUpdate_args implements org.apache.thrift.TBase<conditionalUpdate_args, conditionalUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<conditionalUpdate_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("conditionalUpdate_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField SESS_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessID", org.apache.thrift.protocol.TType.I64, (short)2);
+ private static final org.apache.thrift.protocol.TField MUTATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("mutations", org.apache.thrift.protocol.TType.MAP, (short)3);
+ private static final org.apache.thrift.protocol.TField SYMBOLS_FIELD_DESC = new org.apache.thrift.protocol.TField("symbols", org.apache.thrift.protocol.TType.LIST, (short)4);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new conditionalUpdate_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new conditionalUpdate_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public long sessID; // required
+ public @org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations; // required
+ public @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> symbols; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)1, "tinfo"),
+ SESS_ID((short)2, "sessID"),
+ MUTATIONS((short)3, "mutations"),
+ SYMBOLS((short)4, "symbols");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // TINFO
+ return TINFO;
+ case 2: // SESS_ID
+ return SESS_ID;
+ case 3: // MUTATIONS
+ return MUTATIONS;
+ case 4: // SYMBOLS
+ return SYMBOLS;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __SESSID_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.SESS_ID, new org.apache.thrift.meta_data.FieldMetaData("sessID", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64 , "UpdateID")));
+ tmpMap.put(_Fields.MUTATIONS, new org.apache.thrift.meta_data.FieldMetaData("mutations", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.MAP , "CMBatch")));
+ tmpMap.put(_Fields.SYMBOLS, new org.apache.thrift.meta_data.FieldMetaData("symbols", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(conditionalUpdate_args.class, metaDataMap);
+ }
+
+ public conditionalUpdate_args() {
+ }
+
+ public conditionalUpdate_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ long sessID,
+ java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations,
+ java.util.List<java.lang.String> symbols)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.sessID = sessID;
+ setSessIDIsSet(true);
+ this.mutations = mutations;
+ this.symbols = symbols;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public conditionalUpdate_args(conditionalUpdate_args other) {
+ __isset_bitfield = other.__isset_bitfield;
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ this.sessID = other.sessID;
+ if (other.isSetMutations()) {
+ java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> __this__mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>(other.mutations.size());
+ for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> other_element : other.mutations.entrySet()) {
+
+ org.apache.accumulo.core.dataImpl.thrift.TKeyExtent other_element_key = other_element.getKey();
+ java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> other_element_value = other_element.getValue();
+
+ org.apache.accumulo.core.dataImpl.thrift.TKeyExtent __this__mutations_copy_key = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other_element_key);
+
+ java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> __this__mutations_copy_value = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(other_element_value.size());
+ for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation other_element_value_element : other_element_value) {
+ __this__mutations_copy_value.add(new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation(other_element_value_element));
+ }
+
+ __this__mutations.put(__this__mutations_copy_key, __this__mutations_copy_value);
+ }
+ this.mutations = __this__mutations;
+ }
+ if (other.isSetSymbols()) {
+ java.util.List<java.lang.String> __this__symbols = new java.util.ArrayList<java.lang.String>(other.symbols);
+ this.symbols = __this__symbols;
+ }
+ }
+
+ @Override
+ public conditionalUpdate_args deepCopy() {
+ return new conditionalUpdate_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ setSessIDIsSet(false);
+ this.sessID = 0;
+ this.mutations = null;
+ this.symbols = null;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public conditionalUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ public long getSessID() {
+ return this.sessID;
+ }
+
+ public conditionalUpdate_args setSessID(long sessID) {
+ this.sessID = sessID;
+ setSessIDIsSet(true);
+ return this;
+ }
+
+ public void unsetSessID() {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SESSID_ISSET_ID);
+ }
+
+ /** Returns true if field sessID is set (has been assigned a value) and false otherwise */
+ public boolean isSetSessID() {
+ return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SESSID_ISSET_ID);
+ }
+
+ public void setSessIDIsSet(boolean value) {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SESSID_ISSET_ID, value);
+ }
+
+ public int getMutationsSize() {
+ return (this.mutations == null) ? 0 : this.mutations.size();
+ }
+
+ public void putToMutations(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent key, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> val) {
+ if (this.mutations == null) {
+ this.mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>();
+ }
+ this.mutations.put(key, val);
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> getMutations() {
+ return this.mutations;
+ }
+
+ public conditionalUpdate_args setMutations(@org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> mutations) {
+ this.mutations = mutations;
+ return this;
+ }
+
+ public void unsetMutations() {
+ this.mutations = null;
+ }
+
+ /** Returns true if field mutations is set (has been assigned a value) and false otherwise */
+ public boolean isSetMutations() {
+ return this.mutations != null;
+ }
+
+ public void setMutationsIsSet(boolean value) {
+ if (!value) {
+ this.mutations = null;
+ }
+ }
+
+ public int getSymbolsSize() {
+ return (this.symbols == null) ? 0 : this.symbols.size();
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.util.Iterator<java.lang.String> getSymbolsIterator() {
+ return (this.symbols == null) ? null : this.symbols.iterator();
+ }
+
+ public void addToSymbols(java.lang.String elem) {
+ if (this.symbols == null) {
+ this.symbols = new java.util.ArrayList<java.lang.String>();
+ }
+ this.symbols.add(elem);
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.util.List<java.lang.String> getSymbols() {
+ return this.symbols;
+ }
+
+ public conditionalUpdate_args setSymbols(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> symbols) {
+ this.symbols = symbols;
+ return this;
+ }
+
+ public void unsetSymbols() {
+ this.symbols = null;
+ }
+
+ /** Returns true if field symbols is set (has been assigned a value) and false otherwise */
+ public boolean isSetSymbols() {
+ return this.symbols != null;
+ }
+
+ public void setSymbolsIsSet(boolean value) {
+ if (!value) {
+ this.symbols = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case SESS_ID:
+ if (value == null) {
+ unsetSessID();
+ } else {
+ setSessID((java.lang.Long)value);
+ }
+ break;
+
+ case MUTATIONS:
+ if (value == null) {
+ unsetMutations();
+ } else {
+ setMutations((java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>)value);
+ }
+ break;
+
+ case SYMBOLS:
+ if (value == null) {
+ unsetSymbols();
+ } else {
+ setSymbols((java.util.List<java.lang.String>)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case SESS_ID:
+ return getSessID();
+
+ case MUTATIONS:
+ return getMutations();
+
+ case SYMBOLS:
+ return getSymbols();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case SESS_ID:
+ return isSetSessID();
+ case MUTATIONS:
+ return isSetMutations();
+ case SYMBOLS:
+ return isSetSymbols();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof conditionalUpdate_args)
+ return this.equals((conditionalUpdate_args)that);
+ return false;
+ }
+
+ public boolean equals(conditionalUpdate_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_sessID = true;
+ boolean that_present_sessID = true;
+ if (this_present_sessID || that_present_sessID) {
+ if (!(this_present_sessID && that_present_sessID))
+ return false;
+ if (this.sessID != that.sessID)
+ return false;
+ }
+
+ boolean this_present_mutations = true && this.isSetMutations();
+ boolean that_present_mutations = true && that.isSetMutations();
+ if (this_present_mutations || that_present_mutations) {
+ if (!(this_present_mutations && that_present_mutations))
+ return false;
+ if (!this.mutations.equals(that.mutations))
+ return false;
+ }
+
+ boolean this_present_symbols = true && this.isSetSymbols();
+ boolean that_present_symbols = true && that.isSetSymbols();
+ if (this_present_symbols || that_present_symbols) {
+ if (!(this_present_symbols && that_present_symbols))
+ return false;
+ if (!this.symbols.equals(that.symbols))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(sessID);
+
+ hashCode = hashCode * 8191 + ((isSetMutations()) ? 131071 : 524287);
+ if (isSetMutations())
+ hashCode = hashCode * 8191 + mutations.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetSymbols()) ? 131071 : 524287);
+ if (isSetSymbols())
+ hashCode = hashCode * 8191 + symbols.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(conditionalUpdate_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetSessID(), other.isSetSessID());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSessID()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessID, other.sessID);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetMutations(), other.isSetMutations());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetMutations()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutations, other.mutations);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetSymbols(), other.isSetSymbols());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSymbols()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.symbols, other.symbols);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("conditionalUpdate_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("sessID:");
+ sb.append(this.sessID);
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("mutations:");
+ if (this.mutations == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.mutations);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("symbols:");
+ if (this.symbols == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.symbols);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class conditionalUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public conditionalUpdate_argsStandardScheme getScheme() {
+ return new conditionalUpdate_argsStandardScheme();
+ }
+ }
+
+ private static class conditionalUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<conditionalUpdate_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, conditionalUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // SESS_ID
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.sessID = iprot.readI64();
+ struct.setSessIDIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 3: // MUTATIONS
+ if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+ {
+ org.apache.thrift.protocol.TMap _map24 = iprot.readMapBegin();
+ struct.mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>(2*_map24.size);
+ @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key25;
+ @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> _val26;
+ for (int _i27 = 0; _i27 < _map24.size; ++_i27)
+ {
+ _key25 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ _key25.read(iprot);
+ {
+ org.apache.thrift.protocol.TList _list28 = iprot.readListBegin();
+ _val26 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(_list28.size);
+ @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _elem29;
+ for (int _i30 = 0; _i30 < _list28.size; ++_i30)
+ {
+ _elem29 = new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation();
+ _elem29.read(iprot);
+ _val26.add(_elem29);
+ }
+ iprot.readListEnd();
+ }
+ struct.mutations.put(_key25, _val26);
+ }
+ iprot.readMapEnd();
+ }
+ struct.setMutationsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 4: // SYMBOLS
+ if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+ {
+ org.apache.thrift.protocol.TList _list31 = iprot.readListBegin();
+ struct.symbols = new java.util.ArrayList<java.lang.String>(_list31.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _elem32;
+ for (int _i33 = 0; _i33 < _list31.size; ++_i33)
+ {
+ _elem32 = iprot.readString();
+ struct.symbols.add(_elem32);
+ }
+ iprot.readListEnd();
+ }
+ struct.setSymbolsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, conditionalUpdate_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldBegin(SESS_ID_FIELD_DESC);
+ oprot.writeI64(struct.sessID);
+ oprot.writeFieldEnd();
+ if (struct.mutations != null) {
+ oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
+ {
+ oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, struct.mutations.size()));
+ for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> _iter34 : struct.mutations.entrySet())
+ {
+ _iter34.getKey().write(oprot);
+ {
+ oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter34.getValue().size()));
+ for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _iter35 : _iter34.getValue())
+ {
+ _iter35.write(oprot);
+ }
+ oprot.writeListEnd();
+ }
+ }
+ oprot.writeMapEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ if (struct.symbols != null) {
+ oprot.writeFieldBegin(SYMBOLS_FIELD_DESC);
+ {
+ oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.symbols.size()));
+ for (java.lang.String _iter36 : struct.symbols)
+ {
+ oprot.writeString(_iter36);
+ }
+ oprot.writeListEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class conditionalUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public conditionalUpdate_argsTupleScheme getScheme() {
+ return new conditionalUpdate_argsTupleScheme();
+ }
+ }
+
+ private static class conditionalUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<conditionalUpdate_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetSessID()) {
+ optionals.set(1);
+ }
+ if (struct.isSetMutations()) {
+ optionals.set(2);
+ }
+ if (struct.isSetSymbols()) {
+ optionals.set(3);
+ }
+ oprot.writeBitSet(optionals, 4);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetSessID()) {
+ oprot.writeI64(struct.sessID);
+ }
+ if (struct.isSetMutations()) {
+ {
+ oprot.writeI32(struct.mutations.size());
+ for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>> _iter37 : struct.mutations.entrySet())
+ {
+ _iter37.getKey().write(oprot);
+ {
+ oprot.writeI32(_iter37.getValue().size());
+ for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _iter38 : _iter37.getValue())
+ {
+ _iter38.write(oprot);
+ }
+ }
+ }
+ }
+ }
+ if (struct.isSetSymbols()) {
+ {
+ oprot.writeI32(struct.symbols.size());
+ for (java.lang.String _iter39 : struct.symbols)
+ {
+ oprot.writeString(_iter39);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(4);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.sessID = iprot.readI64();
+ struct.setSessIDIsSet(true);
+ }
+ if (incoming.get(2)) {
+ {
+ org.apache.thrift.protocol.TMap _map40 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST);
+ struct.mutations = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>>(2*_map40.size);
+ @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key41;
+ @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> _val42;
+ for (int _i43 = 0; _i43 < _map40.size; ++_i43)
+ {
+ _key41 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+ _key41.read(iprot);
+ {
+ org.apache.thrift.protocol.TList _list44 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+ _val42 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(_list44.size);
+ @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _elem45;
+ for (int _i46 = 0; _i46 < _list44.size; ++_i46)
+ {
+ _elem45 = new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation();
+ _elem45.read(iprot);
+ _val42.add(_elem45);
+ }
+ }
+ struct.mutations.put(_key41, _val42);
+ }
+ }
+ struct.setMutationsIsSet(true);
+ }
+ if (incoming.get(3)) {
+ {
+ org.apache.thrift.protocol.TList _list47 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+ struct.symbols = new java.util.ArrayList<java.lang.String>(_list47.size);
+ @org.apache.thrift.annotation.Nullable java.lang.String _elem48;
+ for (int _i49 = 0; _i49 < _list47.size; ++_i49)
+ {
+ _elem48 = iprot.readString();
+ struct.symbols.add(_elem48);
+ }
+ }
+ struct.setSymbolsIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class conditionalUpdate_result implements org.apache.thrift.TBase<conditionalUpdate_result, conditionalUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<conditionalUpdate_result> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("conditionalUpdate_result");
+
+ private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+ private static final org.apache.thrift.protocol.TField NSSI_FIELD_DESC = new org.apache.thrift.protocol.TField("nssi", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new conditionalUpdate_resultStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new conditionalUpdate_resultTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> success; // required
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ SUCCESS((short)0, "success"),
+ NSSI((short)1, "nssi");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 0: // SUCCESS
+ return SUCCESS;
+ case 1: // NSSI
+ return NSSI;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TCMResult.class))));
+ tmpMap.put(_Fields.NSSI, new org.apache.thrift.meta_data.FieldMetaData("nssi", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException.class)));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(conditionalUpdate_result.class, metaDataMap);
+ }
+
+ public conditionalUpdate_result() {
+ }
+
+ public conditionalUpdate_result(
+ java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> success,
+ org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi)
+ {
+ this();
+ this.success = success;
+ this.nssi = nssi;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public conditionalUpdate_result(conditionalUpdate_result other) {
+ if (other.isSetSuccess()) {
+ java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> __this__success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(other.success.size());
+ for (org.apache.accumulo.core.dataImpl.thrift.TCMResult other_element : other.success) {
+ __this__success.add(new org.apache.accumulo.core.dataImpl.thrift.TCMResult(other_element));
+ }
+ this.success = __this__success;
+ }
+ if (other.isSetNssi()) {
+ this.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException(other.nssi);
+ }
+ }
+
+ @Override
+ public conditionalUpdate_result deepCopy() {
+ return new conditionalUpdate_result(this);
+ }
+
+ @Override
+ public void clear() {
+ this.success = null;
+ this.nssi = null;
+ }
+
+ public int getSuccessSize() {
+ return (this.success == null) ? 0 : this.success.size();
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.util.Iterator<org.apache.accumulo.core.dataImpl.thrift.TCMResult> getSuccessIterator() {
+ return (this.success == null) ? null : this.success.iterator();
+ }
+
+ public void addToSuccess(org.apache.accumulo.core.dataImpl.thrift.TCMResult elem) {
+ if (this.success == null) {
+ this.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>();
+ }
+ this.success.add(elem);
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> getSuccess() {
+ return this.success;
+ }
+
+ public conditionalUpdate_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult> success) {
+ this.success = success;
+ return this;
+ }
+
+ public void unsetSuccess() {
+ this.success = null;
+ }
+
+ /** Returns true if field success is set (has been assigned a value) and false otherwise */
+ public boolean isSetSuccess() {
+ return this.success != null;
+ }
+
+ public void setSuccessIsSet(boolean value) {
+ if (!value) {
+ this.success = null;
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException getNssi() {
+ return this.nssi;
+ }
+
+ public conditionalUpdate_result setNssi(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
+ this.nssi = nssi;
+ return this;
+ }
+
+ public void unsetNssi() {
+ this.nssi = null;
+ }
+
+ /** Returns true if field nssi is set (has been assigned a value) and false otherwise */
+ public boolean isSetNssi() {
+ return this.nssi != null;
+ }
+
+ public void setNssiIsSet(boolean value) {
+ if (!value) {
+ this.nssi = null;
+ }
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case SUCCESS:
+ if (value == null) {
+ unsetSuccess();
+ } else {
+ setSuccess((java.util.List<org.apache.accumulo.core.dataImpl.thrift.TCMResult>)value);
+ }
+ break;
+
+ case NSSI:
+ if (value == null) {
+ unsetNssi();
+ } else {
+ setNssi((org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case SUCCESS:
+ return getSuccess();
+
+ case NSSI:
+ return getNssi();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case SUCCESS:
+ return isSetSuccess();
+ case NSSI:
+ return isSetNssi();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof conditionalUpdate_result)
+ return this.equals((conditionalUpdate_result)that);
+ return false;
+ }
+
+ public boolean equals(conditionalUpdate_result that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_success = true && this.isSetSuccess();
+ boolean that_present_success = true && that.isSetSuccess();
+ if (this_present_success || that_present_success) {
+ if (!(this_present_success && that_present_success))
+ return false;
+ if (!this.success.equals(that.success))
+ return false;
+ }
+
+ boolean this_present_nssi = true && this.isSetNssi();
+ boolean that_present_nssi = true && that.isSetNssi();
+ if (this_present_nssi || that_present_nssi) {
+ if (!(this_present_nssi && that_present_nssi))
+ return false;
+ if (!this.nssi.equals(that.nssi))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+ if (isSetSuccess())
+ hashCode = hashCode * 8191 + success.hashCode();
+
+ hashCode = hashCode * 8191 + ((isSetNssi()) ? 131071 : 524287);
+ if (isSetNssi())
+ hashCode = hashCode * 8191 + nssi.hashCode();
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(conditionalUpdate_result other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSuccess()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetNssi(), other.isSetNssi());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetNssi()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nssi, other.nssi);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("conditionalUpdate_result(");
+ boolean first = true;
+
+ sb.append("success:");
+ if (this.success == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.success);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("nssi:");
+ if (this.nssi == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.nssi);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class conditionalUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public conditionalUpdate_resultStandardScheme getScheme() {
+ return new conditionalUpdate_resultStandardScheme();
+ }
+ }
+
+ private static class conditionalUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<conditionalUpdate_result> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, conditionalUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 0: // SUCCESS
+ if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+ {
+ org.apache.thrift.protocol.TList _list50 = iprot.readListBegin();
+ struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(_list50.size);
+ @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TCMResult _elem51;
+ for (int _i52 = 0; _i52 < _list50.size; ++_i52)
+ {
+ _elem51 = new org.apache.accumulo.core.dataImpl.thrift.TCMResult();
+ _elem51.read(iprot);
+ struct.success.add(_elem51);
+ }
+ iprot.readListEnd();
+ }
+ struct.setSuccessIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 1: // NSSI
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
+ struct.nssi.read(iprot);
+ struct.setNssiIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, conditionalUpdate_result struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.success != null) {
+ oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+ {
+ oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
+ for (org.apache.accumulo.core.dataImpl.thrift.TCMResult _iter53 : struct.success)
+ {
+ _iter53.write(oprot);
+ }
+ oprot.writeListEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ if (struct.nssi != null) {
+ oprot.writeFieldBegin(NSSI_FIELD_DESC);
+ struct.nssi.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class conditionalUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public conditionalUpdate_resultTupleScheme getScheme() {
+ return new conditionalUpdate_resultTupleScheme();
+ }
+ }
+
+ private static class conditionalUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<conditionalUpdate_result> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetSuccess()) {
+ optionals.set(0);
+ }
+ if (struct.isSetNssi()) {
+ optionals.set(1);
+ }
+ oprot.writeBitSet(optionals, 2);
+ if (struct.isSetSuccess()) {
+ {
+ oprot.writeI32(struct.success.size());
+ for (org.apache.accumulo.core.dataImpl.thrift.TCMResult _iter54 : struct.success)
+ {
+ _iter54.write(oprot);
+ }
+ }
+ }
+ if (struct.isSetNssi()) {
+ struct.nssi.write(oprot);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, conditionalUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(2);
+ if (incoming.get(0)) {
+ {
+ org.apache.thrift.protocol.TList _list55 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+ struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(_list55.size);
+ @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TCMResult _elem56;
+ for (int _i57 = 0; _i57 < _list55.size; ++_i57)
+ {
+ _elem56 = new org.apache.accumulo.core.dataImpl.thrift.TCMResult();
+ _elem56.read(iprot);
+ struct.success.add(_elem56);
+ }
+ }
+ struct.setSuccessIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
+ struct.nssi.read(iprot);
+ struct.setNssiIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class invalidateConditionalUpdate_args implements org.apache.thrift.TBase<invalidateConditionalUpdate_args, invalidateConditionalUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<invalidateConditionalUpdate_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("invalidateConditionalUpdate_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField SESS_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessID", org.apache.thrift.protocol.TType.I64, (short)2);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new invalidateConditionalUpdate_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new invalidateConditionalUpdate_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public long sessID; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)1, "tinfo"),
+ SESS_ID((short)2, "sessID");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // TINFO
+ return TINFO;
+ case 2: // SESS_ID
+ return SESS_ID;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __SESSID_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.SESS_ID, new org.apache.thrift.meta_data.FieldMetaData("sessID", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64 , "UpdateID")));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(invalidateConditionalUpdate_args.class, metaDataMap);
+ }
+
+ public invalidateConditionalUpdate_args() {
+ }
+
+ public invalidateConditionalUpdate_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ long sessID)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.sessID = sessID;
+ setSessIDIsSet(true);
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public invalidateConditionalUpdate_args(invalidateConditionalUpdate_args other) {
+ __isset_bitfield = other.__isset_bitfield;
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+ }
+ this.sessID = other.sessID;
+ }
+
+ @Override
+ public invalidateConditionalUpdate_args deepCopy() {
+ return new invalidateConditionalUpdate_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ setSessIDIsSet(false);
+ this.sessID = 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public invalidateConditionalUpdate_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ public long getSessID() {
+ return this.sessID;
+ }
+
+ public invalidateConditionalUpdate_args setSessID(long sessID) {
+ this.sessID = sessID;
+ setSessIDIsSet(true);
+ return this;
+ }
+
+ public void unsetSessID() {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SESSID_ISSET_ID);
+ }
+
+ /** Returns true if field sessID is set (has been assigned a value) and false otherwise */
+ public boolean isSetSessID() {
+ return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SESSID_ISSET_ID);
+ }
+
+ public void setSessIDIsSet(boolean value) {
+ __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SESSID_ISSET_ID, value);
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value);
+ }
+ break;
+
+ case SESS_ID:
+ if (value == null) {
+ unsetSessID();
+ } else {
+ setSessID((java.lang.Long)value);
+ }
+ break;
+
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case SESS_ID:
+ return getSessID();
+
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case SESS_ID:
+ return isSetSessID();
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof invalidateConditionalUpdate_args)
+ return this.equals((invalidateConditionalUpdate_args)that);
+ return false;
+ }
+
+ public boolean equals(invalidateConditionalUpdate_args that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_sessID = true;
+ boolean that_present_sessID = true;
+ if (this_present_sessID || that_present_sessID) {
+ if (!(this_present_sessID && that_present_sessID))
+ return false;
+ if (this.sessID != that.sessID)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+ if (isSetTinfo())
+ hashCode = hashCode * 8191 + tinfo.hashCode();
+
+ hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(sessID);
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(invalidateConditionalUpdate_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = java.lang.Boolean.compare(isSetSessID(), other.isSetSessID());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSessID()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessID, other.sessID);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("invalidateConditionalUpdate_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("sessID:");
+ sb.append(this.sessID);
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class invalidateConditionalUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public invalidateConditionalUpdate_argsStandardScheme getScheme() {
+ return new invalidateConditionalUpdate_argsStandardScheme();
+ }
+ }
+
+ private static class invalidateConditionalUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<invalidateConditionalUpdate_args> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, invalidateConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // SESS_ID
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.sessID = iprot.readI64();
+ struct.setSessIDIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, invalidateConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldBegin(SESS_ID_FIELD_DESC);
+ oprot.writeI64(struct.sessID);
+ oprot.writeFieldEnd();
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class invalidateConditionalUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public invalidateConditionalUpdate_argsTupleScheme getScheme() {
+ return new invalidateConditionalUpdate_argsTupleScheme();
+ }
+ }
+
+ private static class invalidateConditionalUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<invalidateConditionalUpdate_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, invalidateConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet optionals = new java.util.BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetSessID()) {
+ optionals.set(1);
+ }
+ oprot.writeBitSet(optionals, 2);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetSessID()) {
+ oprot.writeI64(struct.sessID);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, invalidateConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ java.util.BitSet incoming = iprot.readBitSet(2);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.sessID = iprot.readI64();
+ struct.setSessIDIsSet(true);
+ }
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class invalidateConditionalUpdate_result implements org.apache.thrift.TBase<invalidateConditionalUpdate_result, invalidateConditionalUpdate_result._Fields>, java.io.Serializable, Cloneable, Comparable<invalidateConditionalUpdate_result> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("invalidateConditionalUpdate_result");
+
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new invalidateConditionalUpdate_resultStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new invalidateConditionalUpdate_resultTupleSchemeFactory();
+
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(invalidateConditionalUpdate_result.class, metaDataMap);
+ }
+
+ public invalidateConditionalUpdate_result() {
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public invalidateConditionalUpdate_result(invalidateConditionalUpdate_result other) {
+ }
+
+ @Override
+ public invalidateConditionalUpdate_result deepCopy() {
+ return new invalidateConditionalUpdate_result(this);
+ }
+
+ @Override
+ public void clear() {
+ }
+
+ @Override
+ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+ switch (field) {
+ }
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public java.lang.Object getFieldValue(_Fields field) {
+ switch (field) {
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ @Override
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new java.lang.IllegalArgumentException();
+ }
+
+ switch (field) {
+ }
+ throw new java.lang.IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(java.lang.Object that) {
+ if (that instanceof invalidateConditionalUpdate_result)
+ return this.equals((invalidateConditionalUpdate_result)that);
+ return false;
+ }
+
+ public boolean equals(invalidateConditionalUpdate_result that) {
+ if (that == null)
+ return false;
+ if (this == that)
+ return true;
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hashCode = 1;
+
+ return hashCode;
+ }
+
+ @Override
+ public int compareTo(invalidateConditionalUpdate_result other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ return 0;
+ }
+
+ @org.apache.thrift.annotation.Nullable
+ @Override
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ scheme(iprot).read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ scheme(oprot).write(oprot, this);
+ }
+
+ @Override
+ public java.lang.String toString() {
+ java.lang.StringBuilder sb = new java.lang.StringBuilder("invalidateConditionalUpdate_result(");
+ boolean first = true;
+
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class invalidateConditionalUpdate_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public invalidateConditionalUpdate_resultStandardScheme getScheme() {
+ return new invalidateConditionalUpdate_resultStandardScheme();
+ }
+ }
+
+ private static class invalidateConditionalUpdate_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<invalidateConditionalUpdate_result> {
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol iprot, invalidateConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol oprot, invalidateConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class invalidateConditionalUpdate_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+ @Override
+ public invalidateConditionalUpdate_resultTupleScheme getScheme() {
+ return new invalidateConditionalUpdate_resultTupleScheme();
+ }
+ }
+
+ private static class invalidateConditionalUpdate_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<invalidateConditionalUpdate_result> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, invalidateConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, invalidateConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+ }
+ }
+
+ private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+ return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+ }
+ }
+
+ @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+ public static class closeConditionalUpdate_args implements org.apache.thrift.TBase<closeConditionalUpdate_args, closeConditionalUpdate_args._Fields>, java.io.Serializable, Cloneable, Comparable<closeConditionalUpdate_args> {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("closeConditionalUpdate_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField SESS_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessID", org.apache.thrift.protocol.TType.I64, (short)2);
+
+ private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new closeConditionalUpdate_argsStandardSchemeFactory();
+ private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new closeConditionalUpdate_argsTupleSchemeFactory();
+
+ public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+ public long sessID; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)1, "tinfo"),
+ SESS_ID((short)2, "sessID");
+
+ private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+ static {
+ for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // TINFO
+ return TINFO;
+ case 2: // SESS_ID
+ return SESS_ID;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static _Fields findByName(java.lang.String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final java.lang.String _fieldName;
+
+ _Fields(short thriftId, java.lang.String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ @Override
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ @Override
+ public java.lang.String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __SESSID_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class)));
+ tmpMap.put(_Fields.SESS_ID, new org.apache.thrift.meta_data.FieldMetaData("sessID", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64 , "UpdateID")));
+ metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(closeConditionalUpdate_args.class, metaDataMap);
+ }
+
+ public closeConditionalUpdate_args() {
+ }
+
+ public closeConditionalUpdate_args(
+ org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+ long sessID)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.sessID = sessID;
+ setSessIDIsSet(true);
+ }
+