You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mm...@apache.org on 2020/01/28 18:59:16 UTC

[accumulo] branch 1.9 updated: fixes #1474: Option to leave cloned tables offline (#1475)

This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch 1.9
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/1.9 by this push:
     new 9cc673a  fixes #1474: Option to leave cloned tables offline (#1475)
9cc673a is described below

commit 9cc673a4d7096207ff3ff619bc5c9f718faa225a
Author: Ivan Bella <iv...@bella.name>
AuthorDate: Tue Jan 28 13:59:09 2020 -0500

    fixes #1474: Option to leave cloned tables offline (#1475)
    
    * Added an optional keepOffline option to TableOperations.clone via a CloneConfiguration
    * Updated the shell clone command with a -o option
    * Updated the CloneTestIT to test the keepOffline option
    * Updated the ShellServerIT to test the -o option
---
 .../core/client/admin/CloneConfiguration.java      | 118 +++++++++++++++++++++
 .../core/client/admin/TableOperations.java         |  20 ++++
 .../core/client/impl/CloneConfigurationImpl.java   | 118 +++++++++++++++++++++
 .../core/client/impl/TableOperationsImpl.java      |  17 ++-
 .../core/client/mock/MockTableOperations.java      |   8 ++
 .../client/impl/TableOperationsHelperTest.java     |   6 ++
 .../apache/accumulo/master/FateServiceHandler.java |   9 +-
 .../apache/accumulo/master/tableOps/CloneInfo.java |   1 +
 .../accumulo/master/tableOps/CloneTable.java       |   3 +-
 .../accumulo/master/tableOps/FinishCloneTable.java |   6 +-
 .../accumulo/shell/commands/CloneTableCommand.java |  15 ++-
 .../org/apache/accumulo/test/ShellServerIT.java    |  23 ++++
 .../accumulo/test/functional/CloneTestIT.java      |  44 ++++++++
 13 files changed, 380 insertions(+), 8 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/CloneConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/admin/CloneConfiguration.java
new file mode 100644
index 0000000..4f9ea89
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/CloneConfiguration.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.client.admin;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.impl.CloneConfigurationImpl;
+
+/**
+ * A configuration object that can be used with the table clone command in the
+ * {@link TableOperations}.
+ *
+ * @since 1.10 and 2.1
+ */
+public interface CloneConfiguration {
+  /**
+   * Determines if memory is flushed in the source table before cloning.
+   *
+   * @return true if memory is flushed in the source table before cloning.
+   */
+  public boolean isFlush();
+
+  /**
+   * The source table properties are copied. This allows overriding of some of those properties.
+   *
+   * @return The source table properties to override.
+   */
+  public Map<String,String> getPropertiesToSet();
+
+  /**
+   * The source table properties are copied, this allows reverting to system defaults for some of
+   * those properties.
+   *
+   * @return The properties that are to be reverted to system defaults.
+   */
+  public Set<String> getPropertiesToExclude();
+
+  /**
+   * The new table is normally brought online after the cloning process. This allows leaving the new
+   * table offline
+   *
+   * @return true if the new table is to be kept offline after cloning.
+   */
+  public boolean isKeepOffline();
+
+  /**
+   * A CloneConfiguration builder
+   *
+   * @since 1.10 and 2.1
+   */
+  public static interface Builder {
+    /**
+     * Determines if memory is flushed in the source table before cloning.
+     *
+     * @param flush
+     *          true if memory is flushed in the source table before cloning.
+     */
+    public Builder setFlush(boolean flush);
+
+    /**
+     * The source table properties are copied. This allows overriding of some of those properties.
+     *
+     * @param propertiesToSet
+     *          The source table properties to override.
+     */
+    public Builder setPropertiesToSet(Map<String,String> propertiesToSet);
+
+    /**
+     * The source table properties are copied, this allows reverting to system defaults for some of
+     * those properties.
+     *
+     * @param propertiesToExclude
+     *          The properties that are to be reverted to system defaults.
+     */
+    public Builder setPropertiesToExclude(Set<String> propertiesToExclude);
+
+    /**
+     * The new table is normally brought online after the cloning process. This allows leaving the
+     * new table offline
+     *
+     * @param keepOffline
+     *          true if the new table is to be kept offline after cloning.
+     */
+    public Builder setKeepOffline(boolean keepOffline);
+
+    /**
+     * Build the clone configuration
+     *
+     * @return the built immutable clone configuration
+     */
+    public CloneConfiguration build();
+  }
+
+  /**
+   * @return a {@link CloneConfiguration} builder
+   */
+  public static CloneConfiguration.Builder builder() {
+    return new CloneConfigurationImpl();
+  }
+
+}
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 7ff76c9..5b9e5ac 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
@@ -434,6 +434,26 @@ public interface TableOperations {
       AccumuloSecurityException, TableNotFoundException, TableExistsException;
 
   /**
+   * Clone a table from an existing table. The cloned table will have the same data as the source
+   * table it was created from. After cloning, the two tables can mutate independently. Initially
+   * the cloned table should not use any extra space, however as the source table and cloned table
+   * major compact extra space will be used by the clone.
+   *
+   * Initially the cloned table is only readable and writable by the user who created it.
+   *
+   * @param srcTableName
+   *          the table to clone
+   * @param newTableName
+   *          the name of the clone
+   * @param config
+   *          the clone command configuration
+   * @since 1.10 and 2.1
+   */
+  void clone(String srcTableName, String newTableName, CloneConfiguration config)
+      throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
+      TableExistsException;
+
+  /**
    * Rename a table
    *
    * @param oldTableName
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/CloneConfigurationImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/CloneConfigurationImpl.java
new file mode 100644
index 0000000..3c01690
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/CloneConfigurationImpl.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.client.impl;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.admin.CloneConfiguration;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A {@link CloneConfiguration} implementation which also implements the builder thereof
+ *
+ * @since 1.10 and 2.1
+ */
+public class CloneConfigurationImpl implements CloneConfiguration, CloneConfiguration.Builder {
+
+  // The purpose of this is to allow building an immutable CloneConfiguration object without
+  // creating
+  // separate Builder and CloneConfiguration objects. This is done to reduce object creation and
+  // copying. This could easily be changed to two objects without changing the interfaces.
+  private boolean built = false;
+
+  // determines if memory is flushed in the source table before cloning.
+  private boolean flush = true;
+
+  // the sources table properties are copied, this allows overriding of those properties
+  private Map<String,String> propertiesToSet = null;
+
+  // do not copy these properties from the source table, just revert to system defaults
+  private Set<String> propertiesToExclude = null;
+
+  // do not bring the table online after cloning
+  private boolean keepOffline = false;
+
+  public CloneConfigurationImpl() {}
+
+  public boolean isFlush() {
+    Preconditions.checkState(built);
+    return flush;
+  }
+
+  public Map<String,String> getPropertiesToSet() {
+    Preconditions.checkState(built);
+    return (propertiesToSet == null ? Collections.<String,String>emptyMap()
+        : Collections.unmodifiableMap(propertiesToSet));
+  }
+
+  public Set<String> getPropertiesToExclude() {
+    Preconditions.checkState(built);
+    return (propertiesToExclude == null ? Collections.<String>emptySet()
+        : Collections.unmodifiableSet(propertiesToExclude));
+  }
+
+  public boolean isKeepOffline() {
+    Preconditions.checkState(built);
+    return keepOffline;
+  }
+
+  @Override
+  public Builder setFlush(boolean flush) {
+    Preconditions.checkState(!built);
+    this.flush = flush;
+    return this;
+  }
+
+  @Override
+  public Builder setPropertiesToSet(Map<String,String> propertiesToSet) {
+    Preconditions.checkState(!built);
+    this.propertiesToSet = propertiesToSet;
+    return this;
+  }
+
+  @Override
+  public Builder setPropertiesToExclude(Set<String> propertiesToExclude) {
+    Preconditions.checkState(!built);
+    this.propertiesToExclude = propertiesToExclude;
+    return this;
+  }
+
+  @Override
+  public Builder setKeepOffline(boolean keepOffline) {
+    Preconditions.checkState(!built);
+    this.keepOffline = keepOffline;
+    return this;
+  }
+
+  @Override
+  public CloneConfiguration build() {
+    Preconditions.checkState(!built);
+    built = true;
+    return this;
+  }
+
+  @Override
+  public String toString() {
+    return "{flush=" + flush + ", propertiesToSet=" + propertiesToSet + ", propertiesToExclude="
+        + propertiesToExclude + ", keepOffline=" + keepOffline + ", built=" + built + "}";
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
index 4a7d22d..10e14ac 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
@@ -66,6 +66,7 @@ import org.apache.accumulo.core.client.TableDeletedException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.admin.CloneConfiguration;
 import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.FindMax;
@@ -716,23 +717,35 @@ public class TableOperationsImpl extends TableOperationsHelper {
       Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
       TableExistsException {
+    clone(srcTableName, newTableName,
+        CloneConfiguration.builder().setFlush(flush).setPropertiesToSet(propertiesToSet)
+            .setPropertiesToExclude(propertiesToExclude).setKeepOffline(false).build());
+  }
+
+  @Override
+  public void clone(String srcTableName, String newTableName, CloneConfiguration config)
+      throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
+      TableExistsException {
 
     checkArgument(srcTableName != null, "srcTableName is null");
     checkArgument(newTableName != null, "newTableName is null");
 
     String srcTableId = Tables.getTableId(context.getInstance(), srcTableName);
 
-    if (flush)
+    if (config.isFlush())
       _flush(srcTableId, null, null, true);
 
+    Set<String> propertiesToExclude = config.getPropertiesToExclude();
     if (propertiesToExclude == null)
       propertiesToExclude = Collections.emptySet();
 
+    Map<String,String> propertiesToSet = config.getPropertiesToSet();
     if (propertiesToSet == null)
       propertiesToSet = Collections.emptyMap();
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes(UTF_8)),
-        ByteBuffer.wrap(newTableName.getBytes(UTF_8)));
+        ByteBuffer.wrap(newTableName.getBytes(UTF_8)),
+        ByteBuffer.wrap(Boolean.toString(config.isKeepOffline()).getBytes(UTF_8)));
     Map<String,String> opts = new HashMap<>();
     for (Entry<String,String> entry : propertiesToSet.entrySet()) {
       if (entry.getKey().startsWith(CLONE_EXCLUDE_PREFIX))
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index d76d29c..6e92b7b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -38,6 +38,7 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.CloneConfiguration;
 import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.FindMax;
@@ -475,6 +476,13 @@ class MockTableOperations extends TableOperationsHelper {
   }
 
   @Override
+  public void clone(String srcTableName, String newTableName, CloneConfiguration config)
+      throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
+      TableExistsException {
+    throw new NotImplementedException();
+  }
+
+  @Override
   public void flush(String tableName, Text start, Text end, boolean wait)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     if (!exists(tableName))
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
index e732686..c1ae4da 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
@@ -38,6 +38,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.CloneConfiguration;
 import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.Locations;
@@ -154,6 +155,11 @@ public class TableOperationsHelperTest {
         TableExistsException {}
 
     @Override
+    public void clone(String srcTableName, String newTableName, CloneConfiguration config)
+        throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
+        TableExistsException {}
+
+    @Override
     public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException,
         TableNotFoundException, AccumuloException, TableExistsException {}
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
index 7949ac2..08237f8 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
@@ -224,9 +224,14 @@ class FateServiceHandler implements FateService.Iface {
       }
       case TABLE_CLONE: {
         TableOperation tableOp = TableOperation.CLONE;
-        validateArgumentCount(arguments, tableOp, 2);
+        validateArgumentCount(arguments, tableOp, 3);
         String srcTableId = validateTableIdArgument(arguments.get(0), tableOp, CAN_CLONE);
         String tableName = validateTableNameArgument(arguments.get(1), tableOp, NOT_SYSTEM);
+        boolean keepOffline = false;
+        if (arguments.get(2) != null) {
+          keepOffline = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(2)));
+        }
+
         String namespaceId;
         try {
           namespaceId =
@@ -269,7 +274,7 @@ class FateServiceHandler implements FateService.Iface {
         }
 
         master.fate.seedTransaction(opid, new TraceRepo<>(new CloneTable(c.getPrincipal(),
-            namespaceId, srcTableId, tableName, propertiesToSet, propertiesToExclude)),
+            namespaceId, srcTableId, tableName, propertiesToSet, propertiesToExclude, keepOffline)),
             autoCleanup);
 
         break;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneInfo.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneInfo.java
index 96ff954..7e822c8 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneInfo.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneInfo.java
@@ -31,6 +31,7 @@ class CloneInfo implements Serializable {
   String srcNamespaceId;
   Map<String,String> propertiesToSet;
   Set<String> propertiesToExclude;
+  boolean keepOffline;
 
   public String user;
 }
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
index 22e8a68..c45afdb 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
@@ -29,7 +29,7 @@ public class CloneTable extends MasterRepo {
   private CloneInfo cloneInfo;
 
   public CloneTable(String user, String namespaceId, String srcTableId, String tableName,
-      Map<String,String> propertiesToSet, Set<String> propertiesToExclude) {
+      Map<String,String> propertiesToSet, Set<String> propertiesToExclude, boolean keepOffline) {
     cloneInfo = new CloneInfo();
     cloneInfo.user = user;
     cloneInfo.srcTableId = srcTableId;
@@ -37,6 +37,7 @@ public class CloneTable extends MasterRepo {
     cloneInfo.propertiesToExclude = propertiesToExclude;
     cloneInfo.propertiesToSet = propertiesToSet;
     cloneInfo.srcNamespaceId = namespaceId;
+    cloneInfo.keepOffline = keepOffline;
   }
 
   @Override
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/FinishCloneTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/FinishCloneTable.java
index d089ba1..213f254 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/FinishCloneTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/FinishCloneTable.java
@@ -44,7 +44,11 @@ class FinishCloneTable extends MasterRepo {
     // may never create files.. therefore there is no need to consume namenode space w/ directories
     // that are not used... tablet will create directories as needed
 
-    TableManager.getInstance().transitionTableState(cloneInfo.tableId, TableState.ONLINE);
+    if (!cloneInfo.keepOffline) {
+      TableManager.getInstance().transitionTableState(cloneInfo.tableId, TableState.ONLINE);
+    } else {
+      TableManager.getInstance().transitionTableState(cloneInfo.tableId, TableState.OFFLINE);
+    }
 
     Utils.unreserveNamespace(cloneInfo.srcNamespaceId, tid, false);
     if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId))
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/CloneTableCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/CloneTableCommand.java
index 33850a4..d7045ed 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/CloneTableCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/CloneTableCommand.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.CloneConfiguration;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.shell.Shell.Command;
 import org.apache.accumulo.shell.Token;
@@ -37,6 +38,7 @@ public class CloneTableCommand extends Command {
   private Option setPropsOption;
   private Option excludePropsOption;
   private Option noFlushOption;
+  private Option keepOfflineOption;
 
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState)
@@ -46,6 +48,7 @@ public class CloneTableCommand extends Command {
     final HashMap<String,String> props = new HashMap<>();
     final HashSet<String> exclude = new HashSet<>();
     boolean flush = true;
+    boolean keepOffline = false;
 
     if (cl.hasOption(setPropsOption.getOpt())) {
       String[] keyVals = cl.getOptionValue(setPropsOption.getOpt()).split(",");
@@ -66,8 +69,13 @@ public class CloneTableCommand extends Command {
       flush = false;
     }
 
-    shellState.getConnector().tableOperations().clone(cl.getArgs()[0], cl.getArgs()[1], flush,
-        props, exclude);
+    if (cl.hasOption(keepOfflineOption.getOpt())) {
+      keepOffline = true;
+    }
+
+    shellState.getConnector().tableOperations().clone(cl.getArgs()[0], cl.getArgs()[1],
+        CloneConfiguration.builder().setFlush(flush).setPropertiesToSet(props)
+            .setPropertiesToExclude(exclude).setKeepOffline(keepOffline).build());
     return 0;
   }
 
@@ -99,6 +107,9 @@ public class CloneTableCommand extends Command {
     noFlushOption =
         new Option("nf", "noFlush", false, "do not flush table data in memory before cloning.");
     o.addOption(noFlushOption);
+    keepOfflineOption =
+        new Option("o", "offline", false, "do not bring the table online after cloning.");
+    o.addOption(keepOfflineOption);
     return o;
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index abf06bf..8f009f3 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@ -828,6 +828,29 @@ public class ShellServerIT extends SharedMiniClusterBase {
     ts.exec("scan", true, "value", true);
     ts.exec("clonetable " + table + " " + clone);
     // verify constraint, config, and splits were cloned
+    ts.exec("table " + clone);
+    ts.exec("scan", true, "value", true);
+    ts.exec("constraint --list -t " + clone, true, "VisibilityConstraint=2", true);
+    ts.exec("config -t " + clone + " -np", true, "123M", true);
+    ts.exec("getsplits -t " + clone, true, "a\nb\nc\n");
+    ts.exec("deletetable -f " + table);
+    ts.exec("deletetable -f " + clone);
+  }
+
+  @Test
+  public void clonetableOffline() throws Exception {
+    final String table = name.getMethodName(), clone = table + "_clone";
+
+    // clonetable
+    ts.exec("createtable " + table + " -evc");
+    ts.exec("config -t " + table + " -s table.split.threshold=123M", true);
+    ts.exec("addsplits -t " + table + " a b c", true);
+    ts.exec("insert a b c value");
+    ts.exec("scan", true, "value", true);
+    ts.exec("clonetable " + table + " " + clone + " -o");
+    // verify constraint, config, and splits were cloned
+    ts.exec("table " + clone);
+    ts.exec("scan", false, "TableOfflineException", true);
     ts.exec("constraint --list -t " + clone, true, "VisibilityConstraint=2", true);
     ts.exec("config -t " + clone + " -np", true, "123M", true);
     ts.exec("getsplits -t " + clone, true, "a\nb\nc\n");
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java
index d76a28a..d072365 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java
@@ -40,12 +40,15 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.CloneConfiguration;
 import org.apache.accumulo.core.client.admin.DiskUsage;
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.Property;
 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.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
@@ -57,6 +60,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.apache.thrift.TException;
 import org.junit.Assume;
 import org.junit.Test;
 
@@ -121,6 +125,12 @@ public class CloneTestIT extends AccumuloClusterHarness {
 
   }
 
+  private void assertTableState(String table, Connector c, TableState expected) throws TException {
+    String tableId = c.tableOperations().tableIdMap().get(table);
+    TableState tableState = Tables.getTableState(c.getInstance(), tableId);
+    assertEquals(expected, tableState);
+  }
+
   private void checkData(String table2, Connector c) throws TableNotFoundException {
     Scanner scanner = c.createScanner(table2, Authorizations.EMPTY);
 
@@ -248,6 +258,8 @@ public class CloneTestIT extends AccumuloClusterHarness {
 
     c.tableOperations().clone(table1, table2, true, props, exclude);
 
+    assertTableState(table2, c, TableState.ONLINE);
+
     Mutation m3 = new Mutation("009");
     m3.put("data", "x", "1");
     m3.put("data", "y", "2");
@@ -268,6 +280,38 @@ public class CloneTestIT extends AccumuloClusterHarness {
   }
 
   @Test
+  public void testOfflineClone() throws Exception {
+    String[] tableNames = getUniqueNames(3);
+    String table1 = tableNames[0];
+    String table2 = tableNames[1];
+
+    Connector c = getConnector();
+    AccumuloCluster cluster = getCluster();
+    Assume.assumeTrue(cluster instanceof MiniAccumuloClusterImpl);
+    MiniAccumuloClusterImpl mac = (MiniAccumuloClusterImpl) cluster;
+    String rootPath = mac.getConfig().getDir().getAbsolutePath();
+
+    c.tableOperations().create(table1);
+
+    BatchWriter bw = writeData(table1, c);
+
+    Map<String,String> props = new HashMap<>();
+    props.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), "500K");
+
+    Set<String> exclude = new HashSet<>();
+    exclude.add(Property.TABLE_FILE_MAX.getKey());
+
+    c.tableOperations().clone(table1, table2, CloneConfiguration.builder().setFlush(true)
+        .setPropertiesToSet(props).setPropertiesToExclude(exclude).setKeepOffline(true).build());
+
+    assertTableState(table2, c, TableState.OFFLINE);
+
+    // delete tables
+    c.tableOperations().delete(table1);
+    c.tableOperations().delete(table2);
+  }
+
+  @Test
   public void testCloneWithSplits() throws Exception {
     Connector conn = getConnector();