Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E3AEAD7A4 for ; Sat, 26 Jan 2013 01:52:27 +0000 (UTC) Received: (qmail 67257 invoked by uid 500); 26 Jan 2013 01:52:27 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 67235 invoked by uid 500); 26 Jan 2013 01:52:27 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 67224 invoked by uid 99); 26 Jan 2013 01:52:27 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 26 Jan 2013 01:52:27 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_FILL_THIS_FORM_SHORT X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 26 Jan 2013 01:52:23 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id A722623888E7; Sat, 26 Jan 2013 01:52:04 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1438817 - in /accumulo/trunk: core/src/main/java/org/apache/accumulo/core/ core/src/main/java/org/apache/accumulo/core/client/admin/ core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ core/src/main/java/org/apache/accumulo/cor... Date: Sat, 26 Jan 2013 01:52:03 -0000 To: commits@accumulo.apache.org From: kturner@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20130126015204.A722623888E7@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: kturner Date: Sat Jan 26 01:52:02 2013 New Revision: 1438817 URL: http://svn.apache.org/viewvc?rev=1438817&view=rev Log: ACCUMULO-990 added table operation to cancel a compaction Added: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/tableOps/CancelCompactions.java Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/Constants.java accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperation.java accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java accumulo/trunk/core/src/main/java/org/apache/accumulo/core/master/thrift/TableOperation.java accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CompactCommand.java accumulo/trunk/core/src/main/thrift/client.thrift accumulo/trunk/core/src/main/thrift/master.thrift accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java accumulo/trunk/proxy/src/main/thrift/proxy.thrift accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/Constants.java URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/Constants.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/Constants.java (original) +++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/Constants.java Sat Jan 26 01:52:02 2013 @@ -49,6 +49,7 @@ public class Constants { public static final String ZTABLE_STATE = "/state"; public static final String ZTABLE_FLUSH_ID = "/flush-id"; public static final String ZTABLE_COMPACT_ID = "/compact-id"; + public static final String ZTABLE_COMPACT_CANCEL_ID = "/compact-cancel-id"; public static final String ZROOT_TABLET = "/root_tablet"; public static final String ZROOT_TABLET_LOCATION = ZROOT_TABLET + "/location"; Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java (original) +++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java Sat Jan 26 01:52:02 2013 @@ -245,6 +245,22 @@ public interface TableOperations { TableNotFoundException, AccumuloException; /** + * Cancels a user initiated major compaction of a table initiated with {@link #compact(String, Text, Text, boolean, boolean)} or + * {@link #compact(String, Text, Text, List, boolean, boolean)}. Compactions of tablets that are currently running may finish, but new compactions of tablets + * will not start. + * + * @param tableName + * the name of the table + * @throws AccumuloException + * if a general error occurs + * @throws TableNotFoundException + * if the table does not exist + * @throws AccumuloSecurityException + * if the user does not have permission + */ + public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException; + + /** * Delete a table * * @param tableName Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java (original) +++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java Sat Jan 26 01:52:02 2013 @@ -720,6 +720,22 @@ public class TableOperationsImpl extends } } + @Override + public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException { + String tableId = Tables.getTableId(instance, tableName); + + List args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes())); + + Map opts = new HashMap(); + try { + doTableOperation(TableOperation.COMPACT_CANCEL, args, opts, true); + } catch (TableExistsException e) { + // should not happen + throw new RuntimeException(e); + } + + } + private void _flush(String tableId, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { try { Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperation.java URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperation.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperation.java (original) +++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperation.java Sat Jan 26 01:52:02 2013 @@ -43,7 +43,8 @@ import org.apache.thrift.TEnum; BULK_IMPORT(12), COMPACT(13), IMPORT(14), - EXPORT(15); + EXPORT(15), + COMPACT_CANCEL(16); private final int value; @@ -96,6 +97,8 @@ import org.apache.thrift.TEnum; return IMPORT; case 15: return EXPORT; + case 16: + return COMPACT_CANCEL; default: return null; } Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java (original) +++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java Sat Jan 26 01:52:02 2013 @@ -319,6 +319,12 @@ public class MockTableOperations extends } @Override + public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException { + if (!exists(tableName)) + throw new TableNotFoundException(tableName, tableName, ""); + } + + @Override public void clone(String srcTableName, String newTableName, boolean flush, Map propertiesToSet, Set propertiesToExclude) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException { throw new NotImplementedException(); Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/master/thrift/TableOperation.java URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/master/thrift/TableOperation.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/master/thrift/TableOperation.java (original) +++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/master/thrift/TableOperation.java Sat Jan 26 01:52:02 2013 @@ -39,7 +39,8 @@ import org.apache.thrift.TEnum; BULK_IMPORT(8), COMPACT(9), IMPORT(10), - EXPORT(11); + EXPORT(11), + COMPACT_CANCEL(12); private final int value; @@ -84,6 +85,8 @@ import org.apache.thrift.TEnum; return IMPORT; case 11: return EXPORT; + case 12: + return COMPACT_CANCEL; default: return null; } Modified: accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CompactCommand.java URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CompactCommand.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CompactCommand.java (original) +++ accumulo/trunk/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CompactCommand.java Sat Jan 26 01:52:02 2013 @@ -23,6 +23,7 @@ import java.util.List; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.util.shell.Shell; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; @@ -30,7 +31,7 @@ import org.apache.commons.cli.Options; import org.apache.hadoop.io.Text; public class CompactCommand extends TableOperation { - private Option noFlushOption, waitOpt, profileOpt; + private Option noFlushOption, waitOpt, profileOpt, cancelOpt; private boolean flush; private Text startRow; private Text endRow; @@ -39,6 +40,8 @@ public class CompactCommand extends Tabl boolean override = false; private boolean wait; + private boolean cancel = false; + @Override public String description() { return "sets all tablets for a table to major compact as soon as possible (based on current time)"; @@ -46,21 +49,42 @@ public class CompactCommand extends Tabl protected void doTableOp(final Shell shellState, final String tableName) throws AccumuloException, AccumuloSecurityException { // compact the tables - try { - if (wait) { - Shell.log.info("Compacting table ..."); + + if (cancel) { + try { + shellState.getConnector().tableOperations().cancelCompaction(tableName); + Shell.log.info("Compaction canceled for table " + tableName); + } catch (TableNotFoundException e) { + throw new AccumuloException(e); + } + } else { + try { + if (wait) { + Shell.log.info("Compacting table ..."); + } + + shellState.getConnector().tableOperations().compact(tableName, startRow, endRow, iterators, flush, wait); + + Shell.log.info("Compaction of table " + tableName + " " + (wait ? "completed" : "started") + " for given range"); + } catch (Exception ex) { + throw new AccumuloException(ex); } - - shellState.getConnector().tableOperations().compact(tableName, startRow, endRow, iterators, flush, wait); - - Shell.log.info("Compaction of table " + tableName + " " + (wait ? "completed" : "started") + " for given range"); - } catch (Exception ex) { - throw new AccumuloException(ex); } } @Override public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception { + + if (cl.hasOption(cancelOpt.getLongOpt())) { + cancel = true; + + if (cl.getOptions().length > 2) { + throw new IllegalArgumentException("Can not specify other options with cancel"); + } + } else { + cancel = false; + } + flush = !cl.hasOption(noFlushOption.getOpt()); startRow = OptUtil.getStartRow(cl); endRow = OptUtil.getEndRow(cl); @@ -97,6 +121,9 @@ public class CompactCommand extends Tabl profileOpt.setArgName("profile"); opts.addOption(profileOpt); + cancelOpt = new Option(null, "cancel", false, "cancel user initiated compactions"); + opts.addOption(cancelOpt); + return opts; } } Modified: accumulo/trunk/core/src/main/thrift/client.thrift URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/thrift/client.thrift?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/core/src/main/thrift/client.thrift (original) +++ accumulo/trunk/core/src/main/thrift/client.thrift Sat Jan 26 01:52:02 2013 @@ -37,6 +37,7 @@ enum TableOperation { COMPACT IMPORT EXPORT + COMPACT_CANCEL } enum TableOperationExceptionType { Modified: accumulo/trunk/core/src/main/thrift/master.thrift URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/main/thrift/master.thrift?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/core/src/main/thrift/master.thrift (original) +++ accumulo/trunk/core/src/main/thrift/master.thrift Sat Jan 26 01:52:02 2013 @@ -126,6 +126,7 @@ enum TableOperation { COMPACT IMPORT EXPORT + COMPACT_CANCEL } service MasterClientService { Modified: accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java URL: http://svn.apache.org/viewvc/accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java (original) +++ accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java Sat Jan 26 01:52:02 2013 @@ -189,6 +189,9 @@ public class TableOperationsHelperTest { @Override public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {} + + @Override + public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {} } @Test Modified: accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java URL: http://svn.apache.org/viewvc/accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java (original) +++ accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java Sat Jan 26 01:52:02 2013 @@ -68,11 +68,11 @@ import org.apache.accumulo.core.util.Byt import org.apache.accumulo.core.util.TextUtil; import org.apache.accumulo.proxy.thrift.AccumuloProxy; import org.apache.accumulo.proxy.thrift.BatchScanOptions; -import org.apache.accumulo.proxy.thrift.KeyValueAndPeek; import org.apache.accumulo.proxy.thrift.ColumnUpdate; import org.apache.accumulo.proxy.thrift.CompactionReason; import org.apache.accumulo.proxy.thrift.CompactionType; import org.apache.accumulo.proxy.thrift.KeyValue; +import org.apache.accumulo.proxy.thrift.KeyValueAndPeek; import org.apache.accumulo.proxy.thrift.NoMoreEntriesException; import org.apache.accumulo.proxy.thrift.ScanColumn; import org.apache.accumulo.proxy.thrift.ScanOptions; @@ -216,6 +216,16 @@ public class ProxyServer implements Accu } } + @Override + public void cancelCompaction(UserPass userpass, String tableName) throws org.apache.accumulo.proxy.thrift.AccumuloSecurityException, + org.apache.accumulo.proxy.thrift.TableNotFoundException, org.apache.accumulo.proxy.thrift.AccumuloException, TException { + try { + getConnector(userpass).tableOperations().cancelCompaction(tableName); + } catch (Exception e) { + throw translateException(e); + } + } + private List getIteratorSettings(List iterators) { List result = new ArrayList(); if (iterators != null) { @@ -1157,5 +1167,4 @@ public class ProxyServer implements Accu throw translateException(e); } } - } Modified: accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java URL: http://svn.apache.org/viewvc/accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java (original) +++ accumulo/trunk/proxy/src/main/java/org/apache/accumulo/proxy/thrift/AccumuloProxy.java Sat Jan 26 01:52:02 2013 @@ -66,6 +66,8 @@ import org.slf4j.LoggerFactory; public void compactTable(UserPass userpass, String tableName, ByteBuffer startRow, ByteBuffer endRow, List iterators, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException, AccumuloException, org.apache.thrift.TException; + public void cancelCompaction(UserPass userpass, String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException, org.apache.thrift.TException; + public void createTable(UserPass userpass, String tableName, boolean versioningIter, TimeType type) throws AccumuloException, AccumuloSecurityException, TableExistsException, org.apache.thrift.TException; public void deleteTable(UserPass userpass, String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, org.apache.thrift.TException; @@ -210,6 +212,8 @@ import org.slf4j.LoggerFactory; public void compactTable(UserPass userpass, String tableName, ByteBuffer startRow, ByteBuffer endRow, List iterators, boolean flush, boolean wait, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void cancelCompaction(UserPass userpass, String tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void createTable(UserPass userpass, String tableName, boolean versioningIter, TimeType type, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void deleteTable(UserPass userpass, String tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; @@ -601,6 +605,36 @@ import org.slf4j.LoggerFactory; return; } + public void cancelCompaction(UserPass userpass, String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException, org.apache.thrift.TException + { + send_cancelCompaction(userpass, tableName); + recv_cancelCompaction(); + } + + public void send_cancelCompaction(UserPass userpass, String tableName) throws org.apache.thrift.TException + { + cancelCompaction_args args = new cancelCompaction_args(); + args.setUserpass(userpass); + args.setTableName(tableName); + sendBase("cancelCompaction", args); + } + + public void recv_cancelCompaction() throws AccumuloSecurityException, TableNotFoundException, AccumuloException, org.apache.thrift.TException + { + cancelCompaction_result result = new cancelCompaction_result(); + receiveBase(result, "cancelCompaction"); + if (result.ouch1 != null) { + throw result.ouch1; + } + if (result.ouch2 != null) { + throw result.ouch2; + } + if (result.ouch3 != null) { + throw result.ouch3; + } + return; + } + public void createTable(UserPass userpass, String tableName, boolean versioningIter, TimeType type) throws AccumuloException, AccumuloSecurityException, TableExistsException, org.apache.thrift.TException { send_createTable(userpass, tableName, versioningIter, type); @@ -2743,6 +2777,41 @@ import org.slf4j.LoggerFactory; } } + public void cancelCompaction(UserPass userpass, String tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + cancelCompaction_call method_call = new cancelCompaction_call(userpass, tableName, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class cancelCompaction_call extends org.apache.thrift.async.TAsyncMethodCall { + private UserPass userpass; + private String tableName; + public cancelCompaction_call(UserPass userpass, String tableName, org.apache.thrift.async.AsyncMethodCallback 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.userpass = userpass; + this.tableName = tableName; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("cancelCompaction", org.apache.thrift.protocol.TMessageType.CALL, 0)); + cancelCompaction_args args = new cancelCompaction_args(); + args.setUserpass(userpass); + args.setTableName(tableName); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws AccumuloSecurityException, TableNotFoundException, AccumuloException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new 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_cancelCompaction(); + } + } + public void createTable(UserPass userpass, String tableName, boolean versioningIter, TimeType type, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); createTable_call method_call = new createTable_call(userpass, tableName, versioningIter, type, resultHandler, this, ___protocolFactory, ___transport); @@ -5053,6 +5122,7 @@ import org.slf4j.LoggerFactory; processMap.put("clearLocatorCache", new clearLocatorCache()); processMap.put("cloneTable", new cloneTable()); processMap.put("compactTable", new compactTable()); + processMap.put("cancelCompaction", new cancelCompaction()); processMap.put("createTable", new createTable()); processMap.put("deleteTable", new deleteTable()); processMap.put("deleteRows", new deleteRows()); @@ -5332,6 +5402,34 @@ import org.slf4j.LoggerFactory; } } + public static class cancelCompaction extends org.apache.thrift.ProcessFunction { + public cancelCompaction() { + super("cancelCompaction"); + } + + public cancelCompaction_args getEmptyArgsInstance() { + return new cancelCompaction_args(); + } + + protected boolean isOneway() { + return false; + } + + public cancelCompaction_result getResult(I iface, cancelCompaction_args args) throws org.apache.thrift.TException { + cancelCompaction_result result = new cancelCompaction_result(); + try { + iface.cancelCompaction(args.userpass, args.tableName); + } catch (AccumuloSecurityException ouch1) { + result.ouch1 = ouch1; + } catch (TableNotFoundException ouch2) { + result.ouch2 = ouch2; + } catch (AccumuloException ouch3) { + result.ouch3 = ouch3; + } + return result; + } + } + public static class createTable extends org.apache.thrift.ProcessFunction { public createTable() { super("createTable"); @@ -16532,6 +16630,1025 @@ import org.slf4j.LoggerFactory; } + public static class cancelCompaction_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cancelCompaction_args"); + + private static final org.apache.thrift.protocol.TField USERPASS_FIELD_DESC = new org.apache.thrift.protocol.TField("userpass", org.apache.thrift.protocol.TType.STRUCT, (short)1); + 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)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new cancelCompaction_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new cancelCompaction_argsTupleSchemeFactory()); + } + + public UserPass userpass; // required + public String tableName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum { + USERPASS((short)1, "userpass"), + TABLE_NAME((short)2, "tableName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // USERPASS + return USERPASS; + case 2: // TABLE_NAME + return TABLE_NAME; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.USERPASS, new org.apache.thrift.meta_data.FieldMetaData("userpass", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, UserPass.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))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelCompaction_args.class, metaDataMap); + } + + public cancelCompaction_args() { + } + + public cancelCompaction_args( + UserPass userpass, + String tableName) + { + this(); + this.userpass = userpass; + this.tableName = tableName; + } + + /** + * Performs a deep copy on other. + */ + public cancelCompaction_args(cancelCompaction_args other) { + if (other.isSetUserpass()) { + this.userpass = new UserPass(other.userpass); + } + if (other.isSetTableName()) { + this.tableName = other.tableName; + } + } + + public cancelCompaction_args deepCopy() { + return new cancelCompaction_args(this); + } + + @Override + public void clear() { + this.userpass = null; + this.tableName = null; + } + + public UserPass getUserpass() { + return this.userpass; + } + + public cancelCompaction_args setUserpass(UserPass userpass) { + this.userpass = userpass; + return this; + } + + public void unsetUserpass() { + this.userpass = null; + } + + /** Returns true if field userpass is set (has been assigned a value) and false otherwise */ + public boolean isSetUserpass() { + return this.userpass != null; + } + + public void setUserpassIsSet(boolean value) { + if (!value) { + this.userpass = null; + } + } + + public String getTableName() { + return this.tableName; + } + + public cancelCompaction_args setTableName(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 void setFieldValue(_Fields field, Object value) { + switch (field) { + case USERPASS: + if (value == null) { + unsetUserpass(); + } else { + setUserpass((UserPass)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case USERPASS: + return getUserpass(); + + case TABLE_NAME: + return getTableName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case USERPASS: + return isSetUserpass(); + case TABLE_NAME: + return isSetTableName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof cancelCompaction_args) + return this.equals((cancelCompaction_args)that); + return false; + } + + public boolean equals(cancelCompaction_args that) { + if (that == null) + return false; + + boolean this_present_userpass = true && this.isSetUserpass(); + boolean that_present_userpass = true && that.isSetUserpass(); + if (this_present_userpass || that_present_userpass) { + if (!(this_present_userpass && that_present_userpass)) + return false; + if (!this.userpass.equals(that.userpass)) + 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; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + public int compareTo(cancelCompaction_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + cancelCompaction_args typedOther = (cancelCompaction_args)other; + + lastComparison = Boolean.valueOf(isSetUserpass()).compareTo(typedOther.isSetUserpass()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUserpass()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.userpass, typedOther.userpass); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("cancelCompaction_args("); + boolean first = true; + + sb.append("userpass:"); + if (this.userpass == null) { + sb.append("null"); + } else { + sb.append(this.userpass); + } + first = false; + if (!first) sb.append(", "); + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + sb.append(this.tableName); + } + 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 (userpass != null) { + userpass.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, 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 cancelCompaction_argsStandardSchemeFactory implements SchemeFactory { + public cancelCompaction_argsStandardScheme getScheme() { + return new cancelCompaction_argsStandardScheme(); + } + } + + private static class cancelCompaction_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, cancelCompaction_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: // USERPASS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.userpass = new UserPass(); + struct.userpass.read(iprot); + struct.setUserpassIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // 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; + 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(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, cancelCompaction_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.userpass != null) { + oprot.writeFieldBegin(USERPASS_FIELD_DESC); + struct.userpass.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.tableName != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class cancelCompaction_argsTupleSchemeFactory implements SchemeFactory { + public cancelCompaction_argsTupleScheme getScheme() { + return new cancelCompaction_argsTupleScheme(); + } + } + + private static class cancelCompaction_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, cancelCompaction_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetUserpass()) { + optionals.set(0); + } + if (struct.isSetTableName()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetUserpass()) { + struct.userpass.write(oprot); + } + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, cancelCompaction_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.userpass = new UserPass(); + struct.userpass.read(iprot); + struct.setUserpassIsSet(true); + } + if (incoming.get(1)) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } + } + } + + } + + public static class cancelCompaction_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("cancelCompaction_result"); + + private static final org.apache.thrift.protocol.TField OUCH1_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OUCH2_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField OUCH3_FIELD_DESC = new org.apache.thrift.protocol.TField("ouch3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new cancelCompaction_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new cancelCompaction_resultTupleSchemeFactory()); + } + + public AccumuloSecurityException ouch1; // required + public TableNotFoundException ouch2; // required + public AccumuloException ouch3; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OUCH1((short)1, "ouch1"), + OUCH2((short)2, "ouch2"), + OUCH3((short)3, "ouch3"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OUCH1 + return OUCH1; + case 2: // OUCH2 + return OUCH2; + case 3: // OUCH3 + return OUCH3; + 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 IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OUCH1, new org.apache.thrift.meta_data.FieldMetaData("ouch1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.OUCH2, new org.apache.thrift.meta_data.FieldMetaData("ouch2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.OUCH3, new org.apache.thrift.meta_data.FieldMetaData("ouch3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelCompaction_result.class, metaDataMap); + } + + public cancelCompaction_result() { + } + + public cancelCompaction_result( + AccumuloSecurityException ouch1, + TableNotFoundException ouch2, + AccumuloException ouch3) + { + this(); + this.ouch1 = ouch1; + this.ouch2 = ouch2; + this.ouch3 = ouch3; + } + + /** + * Performs a deep copy on other. + */ + public cancelCompaction_result(cancelCompaction_result other) { + if (other.isSetOuch1()) { + this.ouch1 = new AccumuloSecurityException(other.ouch1); + } + if (other.isSetOuch2()) { + this.ouch2 = new TableNotFoundException(other.ouch2); + } + if (other.isSetOuch3()) { + this.ouch3 = new AccumuloException(other.ouch3); + } + } + + public cancelCompaction_result deepCopy() { + return new cancelCompaction_result(this); + } + + @Override + public void clear() { + this.ouch1 = null; + this.ouch2 = null; + this.ouch3 = null; + } + + public AccumuloSecurityException getOuch1() { + return this.ouch1; + } + + public cancelCompaction_result setOuch1(AccumuloSecurityException ouch1) { + this.ouch1 = ouch1; + return this; + } + + public void unsetOuch1() { + this.ouch1 = null; + } + + /** Returns true if field ouch1 is set (has been assigned a value) and false otherwise */ + public boolean isSetOuch1() { + return this.ouch1 != null; + } + + public void setOuch1IsSet(boolean value) { + if (!value) { + this.ouch1 = null; + } + } + + public TableNotFoundException getOuch2() { + return this.ouch2; + } + + public cancelCompaction_result setOuch2(TableNotFoundException ouch2) { + this.ouch2 = ouch2; + return this; + } + + public void unsetOuch2() { + this.ouch2 = null; + } + + /** Returns true if field ouch2 is set (has been assigned a value) and false otherwise */ + public boolean isSetOuch2() { + return this.ouch2 != null; + } + + public void setOuch2IsSet(boolean value) { + if (!value) { + this.ouch2 = null; + } + } + + public AccumuloException getOuch3() { + return this.ouch3; + } + + public cancelCompaction_result setOuch3(AccumuloException ouch3) { + this.ouch3 = ouch3; + return this; + } + + public void unsetOuch3() { + this.ouch3 = null; + } + + /** Returns true if field ouch3 is set (has been assigned a value) and false otherwise */ + public boolean isSetOuch3() { + return this.ouch3 != null; + } + + public void setOuch3IsSet(boolean value) { + if (!value) { + this.ouch3 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OUCH1: + if (value == null) { + unsetOuch1(); + } else { + setOuch1((AccumuloSecurityException)value); + } + break; + + case OUCH2: + if (value == null) { + unsetOuch2(); + } else { + setOuch2((TableNotFoundException)value); + } + break; + + case OUCH3: + if (value == null) { + unsetOuch3(); + } else { + setOuch3((AccumuloException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OUCH1: + return getOuch1(); + + case OUCH2: + return getOuch2(); + + case OUCH3: + return getOuch3(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OUCH1: + return isSetOuch1(); + case OUCH2: + return isSetOuch2(); + case OUCH3: + return isSetOuch3(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof cancelCompaction_result) + return this.equals((cancelCompaction_result)that); + return false; + } + + public boolean equals(cancelCompaction_result that) { + if (that == null) + return false; + + boolean this_present_ouch1 = true && this.isSetOuch1(); + boolean that_present_ouch1 = true && that.isSetOuch1(); + if (this_present_ouch1 || that_present_ouch1) { + if (!(this_present_ouch1 && that_present_ouch1)) + return false; + if (!this.ouch1.equals(that.ouch1)) + return false; + } + + boolean this_present_ouch2 = true && this.isSetOuch2(); + boolean that_present_ouch2 = true && that.isSetOuch2(); + if (this_present_ouch2 || that_present_ouch2) { + if (!(this_present_ouch2 && that_present_ouch2)) + return false; + if (!this.ouch2.equals(that.ouch2)) + return false; + } + + boolean this_present_ouch3 = true && this.isSetOuch3(); + boolean that_present_ouch3 = true && that.isSetOuch3(); + if (this_present_ouch3 || that_present_ouch3) { + if (!(this_present_ouch3 && that_present_ouch3)) + return false; + if (!this.ouch3.equals(that.ouch3)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + public int compareTo(cancelCompaction_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + cancelCompaction_result typedOther = (cancelCompaction_result)other; + + lastComparison = Boolean.valueOf(isSetOuch1()).compareTo(typedOther.isSetOuch1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOuch1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ouch1, typedOther.ouch1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOuch2()).compareTo(typedOther.isSetOuch2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOuch2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ouch2, typedOther.ouch2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOuch3()).compareTo(typedOther.isSetOuch3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOuch3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ouch3, typedOther.ouch3); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("cancelCompaction_result("); + boolean first = true; + + sb.append("ouch1:"); + if (this.ouch1 == null) { + sb.append("null"); + } else { + sb.append(this.ouch1); + } + first = false; + if (!first) sb.append(", "); + sb.append("ouch2:"); + if (this.ouch2 == null) { + sb.append("null"); + } else { + sb.append(this.ouch2); + } + first = false; + if (!first) sb.append(", "); + sb.append("ouch3:"); + if (this.ouch3 == null) { + sb.append("null"); + } else { + sb.append(this.ouch3); + } + 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, 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 cancelCompaction_resultStandardSchemeFactory implements SchemeFactory { + public cancelCompaction_resultStandardScheme getScheme() { + return new cancelCompaction_resultStandardScheme(); + } + } + + private static class cancelCompaction_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, cancelCompaction_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: // OUCH1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.ouch1 = new AccumuloSecurityException(); + struct.ouch1.read(iprot); + struct.setOuch1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OUCH2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.ouch2 = new TableNotFoundException(); + struct.ouch2.read(iprot); + struct.setOuch2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // OUCH3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.ouch3 = new AccumuloException(); + struct.ouch3.read(iprot); + struct.setOuch3IsSet(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(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, cancelCompaction_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.ouch1 != null) { + oprot.writeFieldBegin(OUCH1_FIELD_DESC); + struct.ouch1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.ouch2 != null) { + oprot.writeFieldBegin(OUCH2_FIELD_DESC); + struct.ouch2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.ouch3 != null) { + oprot.writeFieldBegin(OUCH3_FIELD_DESC); + struct.ouch3.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class cancelCompaction_resultTupleSchemeFactory implements SchemeFactory { + public cancelCompaction_resultTupleScheme getScheme() { + return new cancelCompaction_resultTupleScheme(); + } + } + + private static class cancelCompaction_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, cancelCompaction_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetOuch1()) { + optionals.set(0); + } + if (struct.isSetOuch2()) { + optionals.set(1); + } + if (struct.isSetOuch3()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetOuch1()) { + struct.ouch1.write(oprot); + } + if (struct.isSetOuch2()) { + struct.ouch2.write(oprot); + } + if (struct.isSetOuch3()) { + struct.ouch3.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, cancelCompaction_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.ouch1 = new AccumuloSecurityException(); + struct.ouch1.read(iprot); + struct.setOuch1IsSet(true); + } + if (incoming.get(1)) { + struct.ouch2 = new TableNotFoundException(); + struct.ouch2.read(iprot); + struct.setOuch2IsSet(true); + } + if (incoming.get(2)) { + struct.ouch3 = new AccumuloException(); + struct.ouch3.read(iprot); + struct.setOuch3IsSet(true); + } + } + } + + } + public static class createTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createTable_args"); Modified: accumulo/trunk/proxy/src/main/thrift/proxy.thrift URL: http://svn.apache.org/viewvc/accumulo/trunk/proxy/src/main/thrift/proxy.thrift?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/proxy/src/main/thrift/proxy.thrift (original) +++ accumulo/trunk/proxy/src/main/thrift/proxy.thrift Sat Jan 26 01:52:02 2013 @@ -245,8 +245,9 @@ service AccumuloProxy 5:map propertiesToSet, 6:set propertiesToExclude) throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3, 4:TableExistsException ouch4); void compactTable (1:UserPass userpass, 2:string tableName, 3:binary startRow, 4:binary endRow, - 5:list iterators, 6:bool flush, 7:bool wait) - throws (1:AccumuloSecurityException ouch1, 2:TableNotFoundException ouch2, 3:AccumuloException ouch3); + 5:list iterators, 6:bool flush, 7:bool wait) throws (1:AccumuloSecurityException ouch1, 2:TableNotFoundException ouch2, 3:AccumuloException ouch3); + void cancelCompaction(1:UserPass userpass, 2:string tableName) throws (1:AccumuloSecurityException ouch1, 2:TableNotFoundException ouch2, 3:AccumuloException ouch3); + void createTable (1:UserPass userpass, 2:string tableName, 3:bool versioningIter, 4:TimeType type) throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableExistsException ouch3); void deleteTable (1:UserPass userpass, 2:string tableName) throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3); void deleteRows (1:UserPass userpass, 2:string tableName, 3:binary startRow, 4:binary endRow) throws (1:AccumuloException ouch1, 2:AccumuloSecurityException ouch2, 3:TableNotFoundException ouch3); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java Sat Jan 26 01:52:02 2013 @@ -128,6 +128,7 @@ import org.apache.accumulo.server.master import org.apache.accumulo.server.master.state.tables.TableManager; import org.apache.accumulo.server.master.state.tables.TableObserver; import org.apache.accumulo.server.master.tableOps.BulkImport; +import org.apache.accumulo.server.master.tableOps.CancelCompactions; import org.apache.accumulo.server.master.tableOps.ChangeTableState; import org.apache.accumulo.server.master.tableOps.CloneTable; import org.apache.accumulo.server.master.tableOps.CompactRange; @@ -291,6 +292,8 @@ public class Master implements LiveTServ for (String id : Tables.getIdToNameMap(instance).keySet()) { zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_FLUSH_ID, "0".getBytes(), NodeExistsPolicy.SKIP); zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_ID, "0".getBytes(), NodeExistsPolicy.SKIP); + zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_COMPACT_CANCEL_ID, "0".getBytes(), + NodeExistsPolicy.SKIP); for (String prop : tablePropsToDelete) { String propPath = ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_CONF + "/" + prop; @@ -1016,6 +1019,15 @@ public class Master implements LiveTServ fate.seedTransaction(opid, new TraceRepo(new CompactRange(tableId, startRow, endRow, iterators)), autoCleanup); break; } + case COMPACT_CANCEL: { + String tableId = ByteBufferUtil.toString(arguments.get(0)); + + if (!security.canCompact(itw, tableId)) + throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED); + + fate.seedTransaction(opid, new TraceRepo(new CancelCompactions(tableId)), autoCleanup); + break; + } case IMPORT: { String tableName = ByteBufferUtil.toString(arguments.get(0)); String exportDir = ByteBufferUtil.toString(arguments.get(1)); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java Sat Jan 26 01:52:02 2013 @@ -65,6 +65,7 @@ public class TableManager { zoo.putPersistentData(zTablePath + Constants.ZTABLE_STATE, state.name().getBytes(), existsPolicy); zoo.putPersistentData(zTablePath + Constants.ZTABLE_FLUSH_ID, "0".getBytes(), existsPolicy); zoo.putPersistentData(zTablePath + Constants.ZTABLE_COMPACT_ID, "0".getBytes(), existsPolicy); + zoo.putPersistentData(zTablePath + Constants.ZTABLE_COMPACT_CANCEL_ID, "0".getBytes(), existsPolicy); } public synchronized static TableManager getInstance() { Added: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/tableOps/CancelCompactions.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/tableOps/CancelCompactions.java?rev=1438817&view=auto ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/tableOps/CancelCompactions.java (added) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/tableOps/CancelCompactions.java Sat Jan 26 01:52:02 2013 @@ -0,0 +1,99 @@ +/** + * 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.server.master.tableOps; + +import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.impl.thrift.TableOperation; +import org.apache.accumulo.fate.Repo; +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter; +import org.apache.accumulo.fate.zookeeper.ZooReaderWriter.Mutator; +import org.apache.accumulo.server.client.HdfsZooInstance; +import org.apache.accumulo.server.master.Master; +import org.apache.accumulo.server.zookeeper.ZooReaderWriter; + +class FinishCancelCompaction extends MasterRepo { + private static final long serialVersionUID = 1L; + private String tableId; + + public FinishCancelCompaction(String tableId) { + this.tableId = tableId; + } + + @Override + public Repo call(long tid, Master environment) throws Exception { + Utils.getReadLock(tableId, tid).unlock(); + return null; + } + + @Override + public void undo(long tid, Master environment) throws Exception { + + } +} + +/** + * + */ +public class CancelCompactions extends MasterRepo { + + private static final long serialVersionUID = 1L; + private String tableId; + + public CancelCompactions(String tableId) { + this.tableId = tableId; + } + + @Override + public long isReady(long tid, Master environment) throws Exception { + return Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT_CANCEL); + } + + @Override + public Repo call(long tid, Master environment) throws Exception { + String zCompactID = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_COMPACT_ID; + String zCancelID = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + tableId + + Constants.ZTABLE_COMPACT_CANCEL_ID; + + IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance(); + + byte[] currentValue = zoo.getData(zCompactID, null); + + String cvs = new String(currentValue); + String[] tokens = cvs.split(","); + final long flushID = Long.parseLong(new String(tokens[0])); + + zoo.mutate(zCancelID, null, null, new Mutator() { + @Override + public byte[] mutate(byte[] currentValue) throws Exception { + long cid = Long.parseLong(new String(currentValue)); + + if (cid < flushID) + return (flushID + "").getBytes(); + else + return (cid + "").getBytes(); + + } + }); + + return new FinishCancelCompaction(tableId); + } + + @Override + public void undo(long tid, Master environment) throws Exception { + Utils.unreserveTable(tableId, tid, false); + } +} Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java Sat Jan 26 01:52:02 2013 @@ -75,6 +75,16 @@ class CompactionDriver extends MasterRep @Override public long isReady(long tid, Master master) throws Exception { + String zCancelID = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + tableId + + Constants.ZTABLE_COMPACT_CANCEL_ID; + + IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance(); + + if (Long.parseLong(new String(zoo.getData(zCancelID, null))) >= compactId) { + // compaction was canceled + throw new ThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.OTHER, "Compaction canceled"); + } + MapCounter serversToFlush = new MapCounter(); Connector conn = master.getConnector(); Scanner scanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS)); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java?rev=1438817&r1=1438816&r2=1438817&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java Sat Jan 26 01:52:02 2013 @@ -2394,6 +2394,19 @@ public class Tablet { } } + long getCompactionCancelID() { + String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId() + + Constants.ZTABLE_COMPACT_CANCEL_ID; + + try { + return Long.parseLong(new String(ZooReaderWriter.getRetryingInstance().getData(zTablePath, null))); + } catch (KeeperException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + Pair> getCompactionID() throws NoNodeException { try { String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId() @@ -3204,6 +3217,13 @@ public class Tablet { List compactionIterators = new ArrayList(); if (compactionId != null) { + if (reason == MajorCompactionReason.USER) { + if (getCompactionCancelID() >= compactionId.getFirst()) { + // compaction was canceled + return majCStats; + } + } + compactionIterators = compactionId.getSecond(); }