diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java index 637e0c193b8..09e4174dd7e 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java @@ -132,12 +132,15 @@ import org.apache.accumulo.core.dataImpl.thrift.TSummaries; import org.apache.accumulo.core.dataImpl.thrift.TSummarizerConfiguration; import org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest; +import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.manager.thrift.FateOperation; import org.apache.accumulo.core.manager.thrift.FateService; import org.apache.accumulo.core.manager.thrift.ManagerClientService; +import org.apache.accumulo.core.manager.thrift.TFateId; +import org.apache.accumulo.core.manager.thrift.TFateInstanceType; import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.metadata.TServerInstance; @@ -277,12 +280,13 @@ public void create(String tableName, NewTableConfiguration ntc) } } - private long beginFateOperation() throws ThriftSecurityException, TException { + private TFateId beginFateOperation(TFateInstanceType type) + throws ThriftSecurityException, TException { while (true) { FateService.Client client = null; try { client = ThriftClientTypes.FATE.getConnectionWithRetry(context); - return client.beginFateOperation(TraceUtil.traceInfo(), context.rpcCreds()); + return client.beginFateOperation(TraceUtil.traceInfo(), context.rpcCreds(), type); } catch (TTransportException tte) { log.debug("Failed to call beginFateOperation(), retrying ... ", tte); sleepUninterruptibly(100, MILLISECONDS); @@ -298,7 +302,7 @@ private long beginFateOperation() throws ThriftSecurityException, TException { // This method is for retrying in the case of network failures; // anything else it passes to the caller to deal with - private void executeFateOperation(long opid, FateOperation op, List args, + private void executeFateOperation(TFateId opid, FateOperation op, List args, Map opts, boolean autoCleanUp) throws ThriftSecurityException, TException, ThriftTableOperationException { while (true) { @@ -321,7 +325,7 @@ private void executeFateOperation(long opid, FateOperation op, List } } - private String waitForFateOperation(long opid) + private String waitForFateOperation(TFateId opid) throws ThriftSecurityException, TException, ThriftTableOperationException { while (true) { FateService.Client client = null; @@ -341,7 +345,7 @@ private String waitForFateOperation(long opid) } } - private void finishFateOperation(long opid) throws ThriftSecurityException, TException { + private void finishFateOperation(TFateId opid) throws ThriftSecurityException, TException { while (true) { FateService.Client client = null; try { @@ -387,10 +391,12 @@ String doFateOperation(FateOperation op, List args, Map hlocks; @@ -82,10 +83,11 @@ public static class TransactionStatus { private final String top; private final long timeCreated; - private TransactionStatus(Long tid, TStatus status, String txName, List hlocks, - List wlocks, String top, Long timeCreated) { + private TransactionStatus(Long tid, FateInstanceType instanceType, TStatus status, + String txName, List hlocks, List wlocks, String top, Long timeCreated) { this.txid = tid; + this.instanceType = instanceType; this.status = status; this.txName = txName; this.hlocks = Collections.unmodifiableList(hlocks); @@ -103,6 +105,10 @@ public String getTxid() { return FastFormat.toHexString(txid); } + public FateInstanceType getInstanceType() { + return instanceType; + } + public TStatus getStatus() { return status; } @@ -216,15 +222,16 @@ public Map> getDanglingWaitingLocks() { * method does not process lock information, if lock information is desired, use * {@link #getStatus(ReadOnlyFateStore, ZooReader, ServiceLockPath, Set, EnumSet)} * - * @param zs read-only zoostore + * @param fateStores read-only fate stores * @param filterTxid filter results to include for provided transaction ids. * @param filterStatus filter results to include only provided status types * @return list of FATE transactions that match filter criteria */ - public List getTransactionStatus(ReadOnlyFateStore zs, Set filterTxid, + public List getTransactionStatus( + Map> fateStores, Set filterTxid, EnumSet filterStatus) { - FateStatus status = getTransactionStatus(zs, filterTxid, filterStatus, + FateStatus status = getTransactionStatus(fateStores, filterTxid, filterStatus, Collections.>emptyMap(), Collections.>emptyMap()); return status.getTransactions(); @@ -251,7 +258,26 @@ public FateStatus getStatus(ReadOnlyFateStore zs, ZooReader zk, findLocks(zk, lockPath, heldLocks, waitingLocks); - return getTransactionStatus(zs, filterTxid, filterStatus, heldLocks, waitingLocks); + return getTransactionStatus(Map.of(FateInstanceType.META, zs), filterTxid, filterStatus, + heldLocks, waitingLocks); + } + + public FateStatus getStatus(ReadOnlyFateStore as, Set filterTxid, + EnumSet filterStatus) throws KeeperException, InterruptedException { + + return getTransactionStatus(Map.of(FateInstanceType.USER, as), filterTxid, filterStatus, + new HashMap<>(), new HashMap<>()); + } + + public FateStatus getStatus(Map> fateStores, ZooReader zk, + ServiceLock.ServiceLockPath lockPath, Set filterTxid, EnumSet filterStatus) + throws KeeperException, InterruptedException { + Map> heldLocks = new HashMap<>(); + Map> waitingLocks = new HashMap<>(); + + findLocks(zk, lockPath, heldLocks, waitingLocks); + + return getTransactionStatus(fateStores, filterTxid, filterStatus, heldLocks, waitingLocks); } /** @@ -327,7 +353,7 @@ private void findLocks(ZooReader zk, final ServiceLock.ServiceLockPath lockPath, /** * Returns fate status, possibly filtered * - * @param zs read-only access to a populated transaction store. + * @param fateStores read-only access to populated transaction stores. * @param filterTxid Optional. List of transactions to filter results - if null, all transactions * are returned * @param filterStatus Optional. List of status types to filter results - if null, all @@ -336,49 +362,49 @@ private void findLocks(ZooReader zk, final ServiceLock.ServiceLockPath lockPath, * @param waitingLocks populated list of locks held by transaction - or an empty map if none. * @return current fate and lock status */ - private FateStatus getTransactionStatus(ReadOnlyFateStore zs, Set filterTxid, - EnumSet filterStatus, Map> heldLocks, + private FateStatus getTransactionStatus(Map> fateStores, + Set filterTxid, EnumSet filterStatus, Map> heldLocks, Map> waitingLocks) { + final List statuses = new ArrayList<>(); - try (Stream tids = zs.list()) { - List statuses = new ArrayList<>(); + fateStores.forEach((type, store) -> { + try (Stream tids = store.list()) { + tids.forEach(tid -> { - tids.forEach(tid -> { + ReadOnlyFateTxStore txStore = store.read(tid); - ReadOnlyFateTxStore txStore = zs.read(tid); + String txName = (String) txStore.getTransactionInfo(Fate.TxInfo.TX_NAME); - String txName = (String) txStore.getTransactionInfo(Fate.TxInfo.TX_NAME); + List hlocks = heldLocks.remove(tid); - List hlocks = heldLocks.remove(tid); - - if (hlocks == null) { - hlocks = Collections.emptyList(); - } - - List wlocks = waitingLocks.remove(tid); + if (hlocks == null) { + hlocks = Collections.emptyList(); + } - if (wlocks == null) { - wlocks = Collections.emptyList(); - } + List wlocks = waitingLocks.remove(tid); - String top = null; - ReadOnlyRepo repo = txStore.top(); - if (repo != null) { - top = repo.getName(); - } + if (wlocks == null) { + wlocks = Collections.emptyList(); + } - TStatus status = txStore.getStatus(); + String top = null; + ReadOnlyRepo repo = txStore.top(); + if (repo != null) { + top = repo.getName(); + } - long timeCreated = txStore.timeCreated(); + TStatus status = txStore.getStatus(); - if (includeByStatus(status, filterStatus) && includeByTxid(tid, filterTxid)) { - statuses - .add(new TransactionStatus(tid, status, txName, hlocks, wlocks, top, timeCreated)); - } - }); + long timeCreated = txStore.timeCreated(); - return new FateStatus(statuses, heldLocks, waitingLocks); - } + if (includeByStatus(status, filterStatus) && includeByTxid(tid, filterTxid)) { + statuses.add( + new TransactionStatus(tid, type, status, txName, hlocks, wlocks, top, timeCreated)); + } + }); + } + }); + return new FateStatus(statuses, heldLocks, waitingLocks); } private boolean includeByStatus(TStatus status, EnumSet filterStatus) { @@ -389,15 +415,15 @@ private boolean includeByTxid(Long tid, Set filterTxid) { return (filterTxid == null) || filterTxid.isEmpty() || filterTxid.contains(tid); } - public void printAll(ReadOnlyFateStore zs, ZooReader zk, + public void printAll(Map> fateStores, ZooReader zk, ServiceLock.ServiceLockPath tableLocksPath) throws KeeperException, InterruptedException { - print(zs, zk, tableLocksPath, new Formatter(System.out), null, null); + print(fateStores, zk, tableLocksPath, new Formatter(System.out), null, null); } - public void print(ReadOnlyFateStore zs, ZooReader zk, + public void print(Map> fateStores, ZooReader zk, ServiceLock.ServiceLockPath tableLocksPath, Formatter fmt, Set filterTxid, EnumSet filterStatus) throws KeeperException, InterruptedException { - FateStatus fateStatus = getStatus(zs, zk, tableLocksPath, filterTxid, filterStatus); + FateStatus fateStatus = getStatus(fateStores, zk, tableLocksPath, filterTxid, filterStatus); for (TransactionStatus txStatus : fateStatus.getTransactions()) { fmt.format( diff --git a/core/src/main/java/org/apache/accumulo/core/fate/FateInstanceType.java b/core/src/main/java/org/apache/accumulo/core/fate/FateInstanceType.java new file mode 100644 index 00000000000..43e8e35d7ab --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/fate/FateInstanceType.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.core.fate; + +import java.util.Set; + +import org.apache.accumulo.core.clientImpl.Namespace; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.manager.thrift.TFateInstanceType; +import org.apache.accumulo.core.metadata.FateTable; +import org.apache.accumulo.core.metadata.MetadataTable; +import org.apache.accumulo.core.metadata.RootTable; + +public enum FateInstanceType { + META, USER; + + private static final Set META_TABLES = + Set.of(RootTable.ID, MetadataTable.ID, FateTable.ID); + + public static FateInstanceType fromNamespaceOrTableName(String tableOrNamespaceName) { + return tableOrNamespaceName.startsWith(Namespace.ACCUMULO.name()) ? FateInstanceType.META + : FateInstanceType.USER; + } + + public TFateInstanceType toThrift() { + switch (this) { + case USER: + return TFateInstanceType.USER; + case META: + return TFateInstanceType.META; + default: + throw new IllegalStateException("Unknown FateInstance type " + this); + } + } + + public static FateInstanceType fromThrift(TFateInstanceType tfit) { + switch (tfit) { + case USER: + return FateInstanceType.USER; + case META: + return FateInstanceType.META; + default: + throw new IllegalStateException("Unknown type " + tfit); + } + } + + public static FateInstanceType fromTableId(TableId tableId) { + return META_TABLES.contains(tableId) ? META : USER; + } +} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java index 26a66632e8a..89c1c316c9b 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/accumulo/AccumuloStore.java @@ -40,6 +40,7 @@ import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.RepoColumnFamily; import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.TxColumnFamily; import org.apache.accumulo.core.fate.accumulo.schema.FateSchema.TxInfoColumnFamily; +import org.apache.accumulo.core.metadata.FateTable; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.util.ColumnFQ; import org.apache.accumulo.core.util.FastFormat; @@ -61,6 +62,10 @@ public AccumuloStore(ClientContext context, String tableName) { this.tableName = Objects.requireNonNull(tableName); } + public AccumuloStore(ClientContext context) { + this(context, FateTable.NAME); + } + @Override public long create() { long tid = RANDOM.get().nextLong() & 0x7fffffffffffffffL; diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/FateTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/FateTable.java new file mode 100644 index 00000000000..0b6d99d4534 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/metadata/FateTable.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.core.metadata; + +import org.apache.accumulo.core.clientImpl.Namespace; +import org.apache.accumulo.core.data.TableId; + +public class FateTable { + public static final TableId ID = TableId.of("+fate"); + public static final String NAME = Namespace.ACCUMULO.name() + ".fate"; +} diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java index cc16dd4a316..b38fb423bcb 100644 --- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/FateService.java @@ -29,29 +29,29 @@ public class FateService { public interface Iface { - public long beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; + public TFateId beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; - public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; + public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; - public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; + public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; - public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; + public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; - public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; + public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException; } public interface AsyncIface { - public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; } @@ -78,21 +78,22 @@ public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.prot } @Override - public long beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException + public TFateId beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException { - send_beginFateOperation(tinfo, credentials); + send_beginFateOperation(tinfo, credentials, type); return recv_beginFateOperation(); } - public void send_beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException + public void send_beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type) throws org.apache.thrift.TException { beginFateOperation_args args = new beginFateOperation_args(); args.setTinfo(tinfo); args.setCredentials(credentials); + args.setType(type); sendBase("beginFateOperation", args); } - public long recv_beginFateOperation() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException + public TFateId recv_beginFateOperation() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException { beginFateOperation_result result = new beginFateOperation_result(); receiveBase(result, "beginFateOperation"); @@ -109,13 +110,13 @@ public long recv_beginFateOperation() throws org.apache.accumulo.core.clientImpl } @Override - public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException + public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException { send_executeFateOperation(tinfo, credentials, opid, op, arguments, options, autoClean); recv_executeFateOperation(); } - public void send_executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean) throws org.apache.thrift.TException + public void send_executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean) throws org.apache.thrift.TException { executeFateOperation_args args = new executeFateOperation_args(); args.setTinfo(tinfo); @@ -145,13 +146,13 @@ public void recv_executeFateOperation() throws org.apache.accumulo.core.clientIm } @Override - public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException + public java.lang.String waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException { send_waitForFateOperation(tinfo, credentials, opid); return recv_waitForFateOperation(); } - public void send_waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException + public void send_waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.thrift.TException { waitForFateOperation_args args = new waitForFateOperation_args(); args.setTinfo(tinfo); @@ -180,13 +181,13 @@ public java.lang.String recv_waitForFateOperation() throws org.apache.accumulo.c } @Override - public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException + public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException { send_finishFateOperation(tinfo, credentials, opid); recv_finishFateOperation(); } - public void send_finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException + public void send_finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.thrift.TException { finishFateOperation_args args = new finishFateOperation_args(); args.setTinfo(tinfo); @@ -209,13 +210,13 @@ public void recv_finishFateOperation() throws org.apache.accumulo.core.clientImp } @Override - public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException + public boolean cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException { send_cancelFateOperation(tinfo, credentials, opid); return recv_cancelFateOperation(); } - public void send_cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException + public void send_cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid) throws org.apache.thrift.TException { cancelFateOperation_args args = new cancelFateOperation_args(); args.setTinfo(tinfo); @@ -260,20 +261,22 @@ public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, } @Override - public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void beginFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - beginFateOperation_call method_call = new beginFateOperation_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport); + beginFateOperation_call method_call = new beginFateOperation_call(tinfo, credentials, type, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class beginFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class beginFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall { private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; - public beginFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, 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 { + private TFateInstanceType type; + public beginFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateInstanceType type, 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.tinfo = tinfo; this.credentials = credentials; + this.type = type; } @Override @@ -282,12 +285,13 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa beginFateOperation_args args = new beginFateOperation_args(); args.setTinfo(tinfo); args.setCredentials(credentials); + args.setType(type); args.write(prot); prot.writeMessageEnd(); } @Override - public java.lang.Long getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException { + public TFateId getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new java.lang.IllegalStateException("Method call not finished!"); } @@ -298,7 +302,7 @@ public java.lang.Long getResult() throws org.apache.accumulo.core.clientImpl.thr } @Override - public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); executeFateOperation_call method_call = new executeFateOperation_call(tinfo, credentials, opid, op, arguments, options, autoClean, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -308,12 +312,12 @@ public void executeFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInf public static class executeFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall { private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; - private long opid; + private TFateId opid; private FateOperation op; private java.util.List arguments; private java.util.Map options; private boolean autoClean; - public executeFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean, 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 { + public executeFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, FateOperation op, java.util.List arguments, java.util.Map options, boolean autoClean, 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.tinfo = tinfo; this.credentials = credentials; @@ -352,7 +356,7 @@ public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.Thrift } @Override - public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); waitForFateOperation_call method_call = new waitForFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -362,8 +366,8 @@ public void waitForFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInf public static class waitForFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall { private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; - private long opid; - public waitForFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, 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 { + private TFateId opid; + public waitForFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, 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.tinfo = tinfo; this.credentials = credentials; @@ -393,7 +397,7 @@ public java.lang.String getResult() throws org.apache.accumulo.core.clientImpl.t } @Override - public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); finishFateOperation_call method_call = new finishFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -403,8 +407,8 @@ public void finishFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo public static class finishFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall { private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; - private long opid; - public finishFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, 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 { + private TFateId opid; + public finishFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, 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.tinfo = tinfo; this.credentials = credentials; @@ -435,7 +439,7 @@ public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.Thrift } @Override - public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); cancelFateOperation_call method_call = new cancelFateOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -445,8 +449,8 @@ public void cancelFateOperation(org.apache.accumulo.core.clientImpl.thrift.TInfo public static class cancelFateOperation_call extends org.apache.thrift.async.TAsyncMethodCall { private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; - private long opid; - public cancelFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long opid, 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 { + private TFateId opid; + public cancelFateOperation_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, TFateId opid, 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.tinfo = tinfo; this.credentials = credentials; @@ -520,8 +524,7 @@ protected boolean rethrowUnhandledExceptions() { public beginFateOperation_result getResult(I iface, beginFateOperation_args args) throws org.apache.thrift.TException { beginFateOperation_result result = new beginFateOperation_result(); try { - result.success = iface.beginFateOperation(args.tinfo, args.credentials); - result.setSuccessIsSet(true); + result.success = iface.beginFateOperation(args.tinfo, args.credentials, args.type); } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { result.sec = sec; } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) { @@ -693,7 +696,7 @@ protected AsyncProcessor(I iface, java.util.Map extends org.apache.thrift.AsyncProcessFunction { + public static class beginFateOperation extends org.apache.thrift.AsyncProcessFunction { public beginFateOperation() { super("beginFateOperation"); } @@ -704,14 +707,13 @@ public beginFateOperation_args getEmptyArgsInstance() { } @Override - public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + public org.apache.thrift.async.AsyncMethodCallback getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { final org.apache.thrift.AsyncProcessFunction fcall = this; - return new org.apache.thrift.async.AsyncMethodCallback() { + return new org.apache.thrift.async.AsyncMethodCallback() { @Override - public void onComplete(java.lang.Long o) { + public void onComplete(TFateId o) { beginFateOperation_result result = new beginFateOperation_result(); result.success = o; - result.setSuccessIsSet(true); try { fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); } catch (org.apache.thrift.transport.TTransportException e) { @@ -764,8 +766,8 @@ protected boolean isOneway() { } @Override - public void start(I iface, beginFateOperation_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - iface.beginFateOperation(args.tinfo, args.credentials,resultHandler); + public void start(I iface, beginFateOperation_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + iface.beginFateOperation(args.tinfo, args.credentials, args.type,resultHandler); } } @@ -1084,17 +1086,28 @@ public static class beginFateOperation_args implements org.apache.thrift.TBase byName = new java.util.HashMap(); @@ -1114,6 +1127,8 @@ public static _Fields findByThriftId(int fieldId) { return TINFO; case 2: // CREDENTIALS return CREDENTIALS; + case 3: // TYPE + return TYPE; default: return null; } @@ -1164,6 +1179,8 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class))); tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class))); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TFateInstanceType.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginFateOperation_args.class, metaDataMap); } @@ -1173,11 +1190,13 @@ public beginFateOperation_args() { public beginFateOperation_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, - org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, + TFateInstanceType type) { this(); this.tinfo = tinfo; this.credentials = credentials; + this.type = type; } /** @@ -1190,6 +1209,9 @@ public beginFateOperation_args(beginFateOperation_args other) { if (other.isSetCredentials()) { this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials); } + if (other.isSetType()) { + this.type = other.type; + } } @Override @@ -1201,6 +1223,7 @@ public beginFateOperation_args deepCopy() { public void clear() { this.tinfo = null; this.credentials = null; + this.type = null; } @org.apache.thrift.annotation.Nullable @@ -1253,6 +1276,39 @@ public void setCredentialsIsSet(boolean value) { } } + /** + * + * @see TFateInstanceType + */ + @org.apache.thrift.annotation.Nullable + public TFateInstanceType getType() { + return this.type; + } + + /** + * + * @see TFateInstanceType + */ + public beginFateOperation_args setType(@org.apache.thrift.annotation.Nullable TFateInstanceType type) { + this.type = type; + return this; + } + + public void unsetType() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean isSetType() { + return this.type != null; + } + + public void setTypeIsSet(boolean value) { + if (!value) { + this.type = null; + } + } + @Override public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { @@ -1272,6 +1328,14 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case TYPE: + if (value == null) { + unsetType(); + } else { + setType((TFateInstanceType)value); + } + break; + } } @@ -1285,6 +1349,9 @@ public java.lang.Object getFieldValue(_Fields field) { case CREDENTIALS: return getCredentials(); + case TYPE: + return getType(); + } throw new java.lang.IllegalStateException(); } @@ -1301,6 +1368,8 @@ public boolean isSet(_Fields field) { return isSetTinfo(); case CREDENTIALS: return isSetCredentials(); + case TYPE: + return isSetType(); } throw new java.lang.IllegalStateException(); } @@ -1336,6 +1405,15 @@ public boolean equals(beginFateOperation_args that) { return false; } + boolean this_present_type = true && this.isSetType(); + boolean that_present_type = true && that.isSetType(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + return true; } @@ -1351,6 +1429,10 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); + hashCode = hashCode * 8191 + ((isSetType()) ? 131071 : 524287); + if (isSetType()) + hashCode = hashCode * 8191 + type.getValue(); + return hashCode; } @@ -1382,6 +1464,16 @@ public int compareTo(beginFateOperation_args other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetType(), other.isSetType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -1421,6 +1513,14 @@ public java.lang.String toString() { sb.append(this.credentials); } first = false; + if (!first) sb.append(", "); + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; sb.append(")"); return sb.toString(); } @@ -1490,6 +1590,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, beginFateOperation_ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 3: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.type = org.apache.accumulo.core.manager.thrift.TFateInstanceType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -1516,6 +1624,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, beginFateOperation struct.credentials.write(oprot); oprot.writeFieldEnd(); } + if (struct.type != null) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeI32(struct.type.getValue()); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -1541,19 +1654,25 @@ public void write(org.apache.thrift.protocol.TProtocol prot, beginFateOperation_ if (struct.isSetCredentials()) { optionals.set(1); } - oprot.writeBitSet(optionals, 2); + if (struct.isSetType()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); if (struct.isSetTinfo()) { struct.tinfo.write(oprot); } if (struct.isSetCredentials()) { struct.credentials.write(oprot); } + if (struct.isSetType()) { + oprot.writeI32(struct.type.getValue()); + } } @Override public void read(org.apache.thrift.protocol.TProtocol prot, beginFateOperation_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; - java.util.BitSet incoming = iprot.readBitSet(2); + java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(); struct.tinfo.read(iprot); @@ -1564,6 +1683,10 @@ public void read(org.apache.thrift.protocol.TProtocol prot, beginFateOperation_a struct.credentials.read(iprot); struct.setCredentialsIsSet(true); } + if (incoming.get(2)) { + struct.type = org.apache.accumulo.core.manager.thrift.TFateInstanceType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } } } @@ -1576,14 +1699,14 @@ private static S scheme(org.apache. public static class beginFateOperation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFateOperation_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0); + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift.protocol.TField TNASE_FIELD_DESC = new org.apache.thrift.protocol.TField("tnase", org.apache.thrift.protocol.TType.STRUCT, (short)2); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new beginFateOperation_resultStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new beginFateOperation_resultTupleSchemeFactory(); - public long success; // required + public @org.apache.thrift.annotation.Nullable TFateId success; // required public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase; // required @@ -1656,13 +1779,11 @@ public java.lang.String getFieldName() { } // isset id assignments - private static final int __SUCCESS_ISSET_ID = 0; - private byte __isset_bitfield = 0; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFateId.class))); tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException.class))); tmpMap.put(_Fields.TNASE, new org.apache.thrift.meta_data.FieldMetaData("tnase", org.apache.thrift.TFieldRequirementType.DEFAULT, @@ -1675,13 +1796,12 @@ public beginFateOperation_result() { } public beginFateOperation_result( - long success, + TFateId success, org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec, org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException tnase) { this(); this.success = success; - setSuccessIsSet(true); this.sec = sec; this.tnase = tnase; } @@ -1690,8 +1810,9 @@ public beginFateOperation_result( * Performs a deep copy on other. */ public beginFateOperation_result(beginFateOperation_result other) { - __isset_bitfield = other.__isset_bitfield; - this.success = other.success; + if (other.isSetSuccess()) { + this.success = new TFateId(other.success); + } if (other.isSetSec()) { this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec); } @@ -1707,33 +1828,34 @@ public beginFateOperation_result deepCopy() { @Override public void clear() { - setSuccessIsSet(false); - this.success = 0; + this.success = null; this.sec = null; this.tnase = null; } - public long getSuccess() { + @org.apache.thrift.annotation.Nullable + public TFateId getSuccess() { return this.success; } - public beginFateOperation_result setSuccess(long success) { + public beginFateOperation_result setSuccess(@org.apache.thrift.annotation.Nullable TFateId success) { this.success = success; - setSuccessIsSet(true); return this; } public void unsetSuccess() { - __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); + this.success = null; } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); + return this.success != null; } public void setSuccessIsSet(boolean value) { - __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); + if (!value) { + this.success = null; + } } @org.apache.thrift.annotation.Nullable @@ -1793,7 +1915,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetSuccess(); } else { - setSuccess((java.lang.Long)value); + setSuccess((TFateId)value); } break; @@ -1864,12 +1986,12 @@ public boolean equals(beginFateOperation_result that) { if (this == that) return true; - boolean this_present_success = true; - boolean that_present_success = true; + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); if (this_present_success || that_present_success) { if (!(this_present_success && that_present_success)) return false; - if (this.success != that.success) + if (!this.success.equals(that.success)) return false; } @@ -1898,7 +2020,9 @@ public boolean equals(beginFateOperation_result that) { public int hashCode() { int hashCode = 1; - hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(success); + hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287); + if (isSetSuccess()) + hashCode = hashCode * 8191 + success.hashCode(); hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287); if (isSetSec()) @@ -1973,7 +2097,11 @@ public java.lang.String toString() { boolean first = true; sb.append("success:"); - sb.append(this.success); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } first = false; if (!first) sb.append(", "); sb.append("sec:"); @@ -1998,6 +2126,9 @@ public java.lang.String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields // check for sub-struct validity + if (success != null) { + success.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -2010,8 +2141,6 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); } catch (org.apache.thrift.TException te) { throw new java.io.IOException(te); @@ -2039,8 +2168,9 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, beginFateOperation_ } switch (schemeField.id) { case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.success = iprot.readI64(); + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TFateId(); + struct.success.read(iprot); struct.setSuccessIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); @@ -2080,9 +2210,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, beginFateOperation struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetSuccess()) { + if (struct.success != null) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI64(struct.success); + struct.success.write(oprot); oprot.writeFieldEnd(); } if (struct.sec != null) { @@ -2125,7 +2255,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, beginFateOperation_ } oprot.writeBitSet(optionals, 3); if (struct.isSetSuccess()) { - oprot.writeI64(struct.success); + struct.success.write(oprot); } if (struct.isSetSec()) { struct.sec.write(oprot); @@ -2140,7 +2270,8 @@ public void read(org.apache.thrift.protocol.TProtocol prot, beginFateOperation_r org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { - struct.success = iprot.readI64(); + struct.success = new TFateId(); + struct.success.read(iprot); struct.setSuccessIsSet(true); } if (incoming.get(1)) { @@ -2167,7 +2298,7 @@ public static class executeFateOperation_args implements org.apache.thrift.TBase private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.STRUCT, (short)3); private static final org.apache.thrift.protocol.TField OP_FIELD_DESC = new org.apache.thrift.protocol.TField("op", org.apache.thrift.protocol.TType.I32, (short)4); private static final org.apache.thrift.protocol.TField ARGUMENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("arguments", org.apache.thrift.protocol.TType.LIST, (short)5); private static final org.apache.thrift.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("options", org.apache.thrift.protocol.TType.MAP, (short)6); @@ -2178,7 +2309,7 @@ public static class executeFateOperation_args implements org.apache.thrift.TBase public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required - public long opid; // required + public @org.apache.thrift.annotation.Nullable TFateId opid; // required /** * * @see FateOperation @@ -2273,8 +2404,7 @@ public java.lang.String getFieldName() { } // isset id assignments - private static final int __OPID_ISSET_ID = 0; - private static final int __AUTOCLEAN_ISSET_ID = 1; + private static final int __AUTOCLEAN_ISSET_ID = 0; private byte __isset_bitfield = 0; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { @@ -2284,7 +2414,7 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class))); tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFateId.class))); tmpMap.put(_Fields.OP, new org.apache.thrift.meta_data.FieldMetaData("op", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, FateOperation.class))); tmpMap.put(_Fields.ARGUMENTS, new org.apache.thrift.meta_data.FieldMetaData("arguments", org.apache.thrift.TFieldRequirementType.DEFAULT, @@ -2306,7 +2436,7 @@ public executeFateOperation_args() { public executeFateOperation_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, - long opid, + TFateId opid, FateOperation op, java.util.List arguments, java.util.Map options, @@ -2316,7 +2446,6 @@ public executeFateOperation_args( this.tinfo = tinfo; this.credentials = credentials; this.opid = opid; - setOpidIsSet(true); this.op = op; this.arguments = arguments; this.options = options; @@ -2335,7 +2464,9 @@ public executeFateOperation_args(executeFateOperation_args other) { if (other.isSetCredentials()) { this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials); } - this.opid = other.opid; + if (other.isSetOpid()) { + this.opid = new TFateId(other.opid); + } if (other.isSetOp()) { this.op = other.op; } @@ -2359,8 +2490,7 @@ public executeFateOperation_args deepCopy() { public void clear() { this.tinfo = null; this.credentials = null; - setOpidIsSet(false); - this.opid = 0; + this.opid = null; this.op = null; this.arguments = null; this.options = null; @@ -2418,27 +2548,29 @@ public void setCredentialsIsSet(boolean value) { } } - public long getOpid() { + @org.apache.thrift.annotation.Nullable + public TFateId getOpid() { return this.opid; } - public executeFateOperation_args setOpid(long opid) { + public executeFateOperation_args setOpid(@org.apache.thrift.annotation.Nullable TFateId opid) { this.opid = opid; - setOpidIsSet(true); return this; } public void unsetOpid() { - __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __OPID_ISSET_ID); + this.opid = null; } /** Returns true if field opid is set (has been assigned a value) and false otherwise */ public boolean isSetOpid() { - return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __OPID_ISSET_ID); + return this.opid != null; } public void setOpidIsSet(boolean value) { - __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __OPID_ISSET_ID, value); + if (!value) { + this.opid = null; + } } /** @@ -2597,7 +2729,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetOpid(); } else { - setOpid((java.lang.Long)value); + setOpid((TFateId)value); } break; @@ -2722,12 +2854,12 @@ public boolean equals(executeFateOperation_args that) { return false; } - boolean this_present_opid = true; - boolean that_present_opid = true; + boolean this_present_opid = true && this.isSetOpid(); + boolean that_present_opid = true && that.isSetOpid(); if (this_present_opid || that_present_opid) { if (!(this_present_opid && that_present_opid)) return false; - if (this.opid != that.opid) + if (!this.opid.equals(that.opid)) return false; } @@ -2782,7 +2914,9 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); - hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(opid); + hashCode = hashCode * 8191 + ((isSetOpid()) ? 131071 : 524287); + if (isSetOpid()) + hashCode = hashCode * 8191 + opid.hashCode(); hashCode = hashCode * 8191 + ((isSetOp()) ? 131071 : 524287); if (isSetOp()) @@ -2920,7 +3054,11 @@ public java.lang.String toString() { first = false; if (!first) sb.append(", "); sb.append("opid:"); - sb.append(this.opid); + if (this.opid == null) { + sb.append("null"); + } else { + sb.append(this.opid); + } first = false; if (!first) sb.append(", "); sb.append("op:"); @@ -2963,6 +3101,9 @@ public void validate() throws org.apache.thrift.TException { if (credentials != null) { credentials.validate(); } + if (opid != null) { + opid.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -3022,8 +3163,9 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, executeFateOperatio } break; case 3: // OPID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.opid = iprot.readI64(); + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.opid = new TFateId(); + struct.opid.read(iprot); struct.setOpidIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); @@ -3109,9 +3251,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, executeFateOperati struct.credentials.write(oprot); oprot.writeFieldEnd(); } - oprot.writeFieldBegin(OPID_FIELD_DESC); - oprot.writeI64(struct.opid); - oprot.writeFieldEnd(); + if (struct.opid != null) { + oprot.writeFieldBegin(OPID_FIELD_DESC); + struct.opid.write(oprot); + oprot.writeFieldEnd(); + } if (struct.op != null) { oprot.writeFieldBegin(OP_FIELD_DESC); oprot.writeI32(struct.op.getValue()); @@ -3193,7 +3337,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, executeFateOperatio struct.credentials.write(oprot); } if (struct.isSetOpid()) { - oprot.writeI64(struct.opid); + struct.opid.write(oprot); } if (struct.isSetOp()) { oprot.writeI32(struct.op.getValue()); @@ -3237,7 +3381,8 @@ public void read(org.apache.thrift.protocol.TProtocol prot, executeFateOperation struct.setCredentialsIsSet(true); } if (incoming.get(2)) { - struct.opid = iprot.readI64(); + struct.opid = new TFateId(); + struct.opid.read(iprot); struct.setOpidIsSet(true); } if (incoming.get(3)) { @@ -3884,14 +4029,14 @@ public static class waitForFateOperation_args implements org.apache.thrift.TBase private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.STRUCT, (short)3); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new waitForFateOperation_argsStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new waitForFateOperation_argsTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required - public long opid; // required + public @org.apache.thrift.annotation.Nullable TFateId opid; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -3962,8 +4107,6 @@ public java.lang.String getFieldName() { } // isset id assignments - private static final int __OPID_ISSET_ID = 0; - private byte __isset_bitfield = 0; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -3972,7 +4115,7 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class))); tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFateId.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(waitForFateOperation_args.class, metaDataMap); } @@ -3983,27 +4126,27 @@ public waitForFateOperation_args() { public waitForFateOperation_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, - long opid) + TFateId opid) { this(); this.tinfo = tinfo; this.credentials = credentials; this.opid = opid; - setOpidIsSet(true); } /** * Performs a deep copy on other. */ public waitForFateOperation_args(waitForFateOperation_args other) { - __isset_bitfield = other.__isset_bitfield; if (other.isSetTinfo()) { this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo); } if (other.isSetCredentials()) { this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials); } - this.opid = other.opid; + if (other.isSetOpid()) { + this.opid = new TFateId(other.opid); + } } @Override @@ -4015,8 +4158,7 @@ public waitForFateOperation_args deepCopy() { public void clear() { this.tinfo = null; this.credentials = null; - setOpidIsSet(false); - this.opid = 0; + this.opid = null; } @org.apache.thrift.annotation.Nullable @@ -4069,27 +4211,29 @@ public void setCredentialsIsSet(boolean value) { } } - public long getOpid() { + @org.apache.thrift.annotation.Nullable + public TFateId getOpid() { return this.opid; } - public waitForFateOperation_args setOpid(long opid) { + public waitForFateOperation_args setOpid(@org.apache.thrift.annotation.Nullable TFateId opid) { this.opid = opid; - setOpidIsSet(true); return this; } public void unsetOpid() { - __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __OPID_ISSET_ID); + this.opid = null; } /** Returns true if field opid is set (has been assigned a value) and false otherwise */ public boolean isSetOpid() { - return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __OPID_ISSET_ID); + return this.opid != null; } public void setOpidIsSet(boolean value) { - __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __OPID_ISSET_ID, value); + if (!value) { + this.opid = null; + } } @Override @@ -4115,7 +4259,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetOpid(); } else { - setOpid((java.lang.Long)value); + setOpid((TFateId)value); } break; @@ -4188,12 +4332,12 @@ public boolean equals(waitForFateOperation_args that) { return false; } - boolean this_present_opid = true; - boolean that_present_opid = true; + boolean this_present_opid = true && this.isSetOpid(); + boolean that_present_opid = true && that.isSetOpid(); if (this_present_opid || that_present_opid) { if (!(this_present_opid && that_present_opid)) return false; - if (this.opid != that.opid) + if (!this.opid.equals(that.opid)) return false; } @@ -4212,7 +4356,9 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); - hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(opid); + hashCode = hashCode * 8191 + ((isSetOpid()) ? 131071 : 524287); + if (isSetOpid()) + hashCode = hashCode * 8191 + opid.hashCode(); return hashCode; } @@ -4296,7 +4442,11 @@ public java.lang.String toString() { first = false; if (!first) sb.append(", "); sb.append("opid:"); - sb.append(this.opid); + if (this.opid == null) { + sb.append("null"); + } else { + sb.append(this.opid); + } first = false; sb.append(")"); return sb.toString(); @@ -4311,6 +4461,9 @@ public void validate() throws org.apache.thrift.TException { if (credentials != null) { credentials.validate(); } + if (opid != null) { + opid.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -4323,8 +4476,6 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); } catch (org.apache.thrift.TException te) { throw new java.io.IOException(te); @@ -4370,8 +4521,9 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, waitForFateOperatio } break; case 3: // OPID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.opid = iprot.readI64(); + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.opid = new TFateId(); + struct.opid.read(iprot); struct.setOpidIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); @@ -4403,9 +4555,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, waitForFateOperati struct.credentials.write(oprot); oprot.writeFieldEnd(); } - oprot.writeFieldBegin(OPID_FIELD_DESC); - oprot.writeI64(struct.opid); - oprot.writeFieldEnd(); + if (struct.opid != null) { + oprot.writeFieldBegin(OPID_FIELD_DESC); + struct.opid.write(oprot); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -4442,7 +4596,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, waitForFateOperatio struct.credentials.write(oprot); } if (struct.isSetOpid()) { - oprot.writeI64(struct.opid); + struct.opid.write(oprot); } } @@ -4461,7 +4615,8 @@ public void read(org.apache.thrift.protocol.TProtocol prot, waitForFateOperation struct.setCredentialsIsSet(true); } if (incoming.get(2)) { - struct.opid = iprot.readI64(); + struct.opid = new TFateId(); + struct.opid.read(iprot); struct.setOpidIsSet(true); } } @@ -5177,14 +5332,14 @@ public static class finishFateOperation_args implements org.apache.thrift.TBase< private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.STRUCT, (short)3); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new finishFateOperation_argsStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new finishFateOperation_argsTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required - public long opid; // required + public @org.apache.thrift.annotation.Nullable TFateId opid; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -5255,8 +5410,6 @@ public java.lang.String getFieldName() { } // isset id assignments - private static final int __OPID_ISSET_ID = 0; - private byte __isset_bitfield = 0; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -5265,7 +5418,7 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class))); tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFateId.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(finishFateOperation_args.class, metaDataMap); } @@ -5276,27 +5429,27 @@ public finishFateOperation_args() { public finishFateOperation_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, - long opid) + TFateId opid) { this(); this.tinfo = tinfo; this.credentials = credentials; this.opid = opid; - setOpidIsSet(true); } /** * Performs a deep copy on other. */ public finishFateOperation_args(finishFateOperation_args other) { - __isset_bitfield = other.__isset_bitfield; if (other.isSetTinfo()) { this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo); } if (other.isSetCredentials()) { this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials); } - this.opid = other.opid; + if (other.isSetOpid()) { + this.opid = new TFateId(other.opid); + } } @Override @@ -5308,8 +5461,7 @@ public finishFateOperation_args deepCopy() { public void clear() { this.tinfo = null; this.credentials = null; - setOpidIsSet(false); - this.opid = 0; + this.opid = null; } @org.apache.thrift.annotation.Nullable @@ -5362,27 +5514,29 @@ public void setCredentialsIsSet(boolean value) { } } - public long getOpid() { + @org.apache.thrift.annotation.Nullable + public TFateId getOpid() { return this.opid; } - public finishFateOperation_args setOpid(long opid) { + public finishFateOperation_args setOpid(@org.apache.thrift.annotation.Nullable TFateId opid) { this.opid = opid; - setOpidIsSet(true); return this; } public void unsetOpid() { - __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __OPID_ISSET_ID); + this.opid = null; } /** Returns true if field opid is set (has been assigned a value) and false otherwise */ public boolean isSetOpid() { - return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __OPID_ISSET_ID); + return this.opid != null; } public void setOpidIsSet(boolean value) { - __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __OPID_ISSET_ID, value); + if (!value) { + this.opid = null; + } } @Override @@ -5408,7 +5562,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetOpid(); } else { - setOpid((java.lang.Long)value); + setOpid((TFateId)value); } break; @@ -5481,12 +5635,12 @@ public boolean equals(finishFateOperation_args that) { return false; } - boolean this_present_opid = true; - boolean that_present_opid = true; + boolean this_present_opid = true && this.isSetOpid(); + boolean that_present_opid = true && that.isSetOpid(); if (this_present_opid || that_present_opid) { if (!(this_present_opid && that_present_opid)) return false; - if (this.opid != that.opid) + if (!this.opid.equals(that.opid)) return false; } @@ -5505,7 +5659,9 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); - hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(opid); + hashCode = hashCode * 8191 + ((isSetOpid()) ? 131071 : 524287); + if (isSetOpid()) + hashCode = hashCode * 8191 + opid.hashCode(); return hashCode; } @@ -5589,7 +5745,11 @@ public java.lang.String toString() { first = false; if (!first) sb.append(", "); sb.append("opid:"); - sb.append(this.opid); + if (this.opid == null) { + sb.append("null"); + } else { + sb.append(this.opid); + } first = false; sb.append(")"); return sb.toString(); @@ -5604,6 +5764,9 @@ public void validate() throws org.apache.thrift.TException { if (credentials != null) { credentials.validate(); } + if (opid != null) { + opid.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -5616,8 +5779,6 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); } catch (org.apache.thrift.TException te) { throw new java.io.IOException(te); @@ -5663,8 +5824,9 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, finishFateOperation } break; case 3: // OPID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.opid = iprot.readI64(); + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.opid = new TFateId(); + struct.opid.read(iprot); struct.setOpidIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); @@ -5696,9 +5858,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, finishFateOperatio struct.credentials.write(oprot); oprot.writeFieldEnd(); } - oprot.writeFieldBegin(OPID_FIELD_DESC); - oprot.writeI64(struct.opid); - oprot.writeFieldEnd(); + if (struct.opid != null) { + oprot.writeFieldBegin(OPID_FIELD_DESC); + struct.opid.write(oprot); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -5735,7 +5899,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, finishFateOperation struct.credentials.write(oprot); } if (struct.isSetOpid()) { - oprot.writeI64(struct.opid); + struct.opid.write(oprot); } } @@ -5754,7 +5918,8 @@ public void read(org.apache.thrift.protocol.TProtocol prot, finishFateOperation_ struct.setCredentialsIsSet(true); } if (incoming.get(2)) { - struct.opid = iprot.readI64(); + struct.opid = new TFateId(); + struct.opid.read(iprot); struct.setOpidIsSet(true); } } @@ -6258,14 +6423,14 @@ public static class cancelFateOperation_args implements org.apache.thrift.TBase< private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField OPID_FIELD_DESC = new org.apache.thrift.protocol.TField("opid", org.apache.thrift.protocol.TType.STRUCT, (short)3); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new cancelFateOperation_argsStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new cancelFateOperation_argsTupleSchemeFactory(); public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required - public long opid; // required + public @org.apache.thrift.annotation.Nullable TFateId opid; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -6336,8 +6501,6 @@ public java.lang.String getFieldName() { } // isset id assignments - private static final int __OPID_ISSET_ID = 0; - private byte __isset_bitfield = 0; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -6346,7 +6509,7 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class))); tmpMap.put(_Fields.OPID, new org.apache.thrift.meta_data.FieldMetaData("opid", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFateId.class))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(cancelFateOperation_args.class, metaDataMap); } @@ -6357,27 +6520,27 @@ public cancelFateOperation_args() { public cancelFateOperation_args( org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, - long opid) + TFateId opid) { this(); this.tinfo = tinfo; this.credentials = credentials; this.opid = opid; - setOpidIsSet(true); } /** * Performs a deep copy on other. */ public cancelFateOperation_args(cancelFateOperation_args other) { - __isset_bitfield = other.__isset_bitfield; if (other.isSetTinfo()) { this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo); } if (other.isSetCredentials()) { this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials); } - this.opid = other.opid; + if (other.isSetOpid()) { + this.opid = new TFateId(other.opid); + } } @Override @@ -6389,8 +6552,7 @@ public cancelFateOperation_args deepCopy() { public void clear() { this.tinfo = null; this.credentials = null; - setOpidIsSet(false); - this.opid = 0; + this.opid = null; } @org.apache.thrift.annotation.Nullable @@ -6443,27 +6605,29 @@ public void setCredentialsIsSet(boolean value) { } } - public long getOpid() { + @org.apache.thrift.annotation.Nullable + public TFateId getOpid() { return this.opid; } - public cancelFateOperation_args setOpid(long opid) { + public cancelFateOperation_args setOpid(@org.apache.thrift.annotation.Nullable TFateId opid) { this.opid = opid; - setOpidIsSet(true); return this; } public void unsetOpid() { - __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __OPID_ISSET_ID); + this.opid = null; } /** Returns true if field opid is set (has been assigned a value) and false otherwise */ public boolean isSetOpid() { - return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __OPID_ISSET_ID); + return this.opid != null; } public void setOpidIsSet(boolean value) { - __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __OPID_ISSET_ID, value); + if (!value) { + this.opid = null; + } } @Override @@ -6489,7 +6653,7 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable if (value == null) { unsetOpid(); } else { - setOpid((java.lang.Long)value); + setOpid((TFateId)value); } break; @@ -6562,12 +6726,12 @@ public boolean equals(cancelFateOperation_args that) { return false; } - boolean this_present_opid = true; - boolean that_present_opid = true; + boolean this_present_opid = true && this.isSetOpid(); + boolean that_present_opid = true && that.isSetOpid(); if (this_present_opid || that_present_opid) { if (!(this_present_opid && that_present_opid)) return false; - if (this.opid != that.opid) + if (!this.opid.equals(that.opid)) return false; } @@ -6586,7 +6750,9 @@ public int hashCode() { if (isSetCredentials()) hashCode = hashCode * 8191 + credentials.hashCode(); - hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(opid); + hashCode = hashCode * 8191 + ((isSetOpid()) ? 131071 : 524287); + if (isSetOpid()) + hashCode = hashCode * 8191 + opid.hashCode(); return hashCode; } @@ -6670,7 +6836,11 @@ public java.lang.String toString() { first = false; if (!first) sb.append(", "); sb.append("opid:"); - sb.append(this.opid); + if (this.opid == null) { + sb.append("null"); + } else { + sb.append(this.opid); + } first = false; sb.append(")"); return sb.toString(); @@ -6685,6 +6855,9 @@ public void validate() throws org.apache.thrift.TException { if (credentials != null) { credentials.validate(); } + if (opid != null) { + opid.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -6697,8 +6870,6 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); } catch (org.apache.thrift.TException te) { throw new java.io.IOException(te); @@ -6744,8 +6915,9 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, cancelFateOperation } break; case 3: // OPID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.opid = iprot.readI64(); + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.opid = new TFateId(); + struct.opid.read(iprot); struct.setOpidIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); @@ -6777,9 +6949,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, cancelFateOperatio struct.credentials.write(oprot); oprot.writeFieldEnd(); } - oprot.writeFieldBegin(OPID_FIELD_DESC); - oprot.writeI64(struct.opid); - oprot.writeFieldEnd(); + if (struct.opid != null) { + oprot.writeFieldBegin(OPID_FIELD_DESC); + struct.opid.write(oprot); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -6816,7 +6990,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, cancelFateOperation struct.credentials.write(oprot); } if (struct.isSetOpid()) { - oprot.writeI64(struct.opid); + struct.opid.write(oprot); } } @@ -6835,7 +7009,8 @@ public void read(org.apache.thrift.protocol.TProtocol prot, cancelFateOperation_ struct.setCredentialsIsSet(true); } if (incoming.get(2)) { - struct.opid = iprot.readI64(); + struct.opid = new TFateId(); + struct.opid.read(iprot); struct.setOpidIsSet(true); } } diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateId.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateId.java new file mode 100644 index 00000000000..d781407397c --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateId.java @@ -0,0 +1,522 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +/** + * Autogenerated by Thrift Compiler (0.17.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.accumulo.core.manager.thrift; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) +public class TFateId implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFateId"); + + private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField TID_FIELD_DESC = new org.apache.thrift.protocol.TField("tid", org.apache.thrift.protocol.TType.I64, (short)2); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TFateIdStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TFateIdTupleSchemeFactory(); + + /** + * + * @see TFateInstanceType + */ + public @org.apache.thrift.annotation.Nullable TFateInstanceType type; // required + public long tid; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + /** + * + * @see TFateInstanceType + */ + TYPE((short)1, "type"), + TID((short)2, "tid"); + + private static final java.util.Map byName = new java.util.HashMap(); + + static { + for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + @org.apache.thrift.annotation.Nullable + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TYPE + return TYPE; + case 2: // TID + return TID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + @org.apache.thrift.annotation.Nullable + public static _Fields findByName(java.lang.String name) { + return byName.get(name); + } + + private final short _thriftId; + private final java.lang.String _fieldName; + + _Fields(short thriftId, java.lang.String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + @Override + public short getThriftFieldId() { + return _thriftId; + } + + @Override + public java.lang.String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __TID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TFateInstanceType.class))); + tmpMap.put(_Fields.TID, new org.apache.thrift.meta_data.FieldMetaData("tid", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFateId.class, metaDataMap); + } + + public TFateId() { + } + + public TFateId( + TFateInstanceType type, + long tid) + { + this(); + this.type = type; + this.tid = tid; + setTidIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TFateId(TFateId other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetType()) { + this.type = other.type; + } + this.tid = other.tid; + } + + @Override + public TFateId deepCopy() { + return new TFateId(this); + } + + @Override + public void clear() { + this.type = null; + setTidIsSet(false); + this.tid = 0; + } + + /** + * + * @see TFateInstanceType + */ + @org.apache.thrift.annotation.Nullable + public TFateInstanceType getType() { + return this.type; + } + + /** + * + * @see TFateInstanceType + */ + public TFateId setType(@org.apache.thrift.annotation.Nullable TFateInstanceType type) { + this.type = type; + return this; + } + + public void unsetType() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean isSetType() { + return this.type != null; + } + + public void setTypeIsSet(boolean value) { + if (!value) { + this.type = null; + } + } + + public long getTid() { + return this.tid; + } + + public TFateId setTid(long tid) { + this.tid = tid; + setTidIsSet(true); + return this; + } + + public void unsetTid() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TID_ISSET_ID); + } + + /** Returns true if field tid is set (has been assigned a value) and false otherwise */ + public boolean isSetTid() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TID_ISSET_ID); + } + + public void setTidIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TID_ISSET_ID, value); + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case TYPE: + if (value == null) { + unsetType(); + } else { + setType((TFateInstanceType)value); + } + break; + + case TID: + if (value == null) { + unsetTid(); + } else { + setTid((java.lang.Long)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case TYPE: + return getType(); + + case TID: + return getTid(); + + } + throw new java.lang.IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + @Override + public boolean isSet(_Fields field) { + if (field == null) { + throw new java.lang.IllegalArgumentException(); + } + + switch (field) { + case TYPE: + return isSetType(); + case TID: + return isSetTid(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof TFateId) + return this.equals((TFateId)that); + return false; + } + + public boolean equals(TFateId that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_type = true && this.isSetType(); + boolean that_present_type = true && that.isSetType(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + + boolean this_present_tid = true; + boolean that_present_tid = true; + if (this_present_tid || that_present_tid) { + if (!(this_present_tid && that_present_tid)) + return false; + if (this.tid != that.tid) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetType()) ? 131071 : 524287); + if (isSetType()) + hashCode = hashCode * 8191 + type.getValue(); + + hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(tid); + + return hashCode; + } + + @Override + public int compareTo(TFateId other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetType(), other.isSetType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetTid(), other.isSetTid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tid, other.tid); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + @org.apache.thrift.annotation.Nullable + @Override + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + scheme(iprot).read(iprot, this); + } + + @Override + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + scheme(oprot).write(oprot, this); + } + + @Override + public java.lang.String toString() { + java.lang.StringBuilder sb = new java.lang.StringBuilder("TFateId("); + boolean first = true; + + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; + if (!first) sb.append(", "); + sb.append("tid:"); + sb.append(this.tid); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TFateIdStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TFateIdStandardScheme getScheme() { + return new TFateIdStandardScheme(); + } + } + + private static class TFateIdStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, TFateId 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: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.type = org.apache.accumulo.core.manager.thrift.TFateInstanceType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.tid = iprot.readI64(); + struct.setTidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + @Override + public void write(org.apache.thrift.protocol.TProtocol oprot, TFateId struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.type != null) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeI32(struct.type.getValue()); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(TID_FIELD_DESC); + oprot.writeI64(struct.tid); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TFateIdTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TFateIdTupleScheme getScheme() { + return new TFateIdTupleScheme(); + } + } + + private static class TFateIdTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TFateId struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet optionals = new java.util.BitSet(); + if (struct.isSetType()) { + optionals.set(0); + } + if (struct.isSetTid()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetType()) { + oprot.writeI32(struct.type.getValue()); + } + if (struct.isSetTid()) { + oprot.writeI64(struct.tid); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TFateId struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.type = org.apache.accumulo.core.manager.thrift.TFateInstanceType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } + if (incoming.get(1)) { + struct.tid = iprot.readI64(); + struct.setTidIsSet(true); + } + } + } + + private static S scheme(org.apache.thrift.protocol.TProtocol proto) { + return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme(); + } + private static void unusedMethod() {} +} + diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateInstanceType.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateInstanceType.java new file mode 100644 index 00000000000..488e0829063 --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TFateInstanceType.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +/** + * Autogenerated by Thrift Compiler (0.17.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.accumulo.core.manager.thrift; + + +public enum TFateInstanceType implements org.apache.thrift.TEnum { + META(0), + USER(1); + + private final int value; + + private TFateInstanceType(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + @Override + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + @org.apache.thrift.annotation.Nullable + public static TFateInstanceType findByValue(int value) { + switch (value) { + case 0: + return META; + case 1: + return USER; + default: + return null; + } + } +} diff --git a/core/src/main/thrift/manager.thrift b/core/src/main/thrift/manager.thrift index dc9ddbb34ab..8a8b413ce62 100644 --- a/core/src/main/thrift/manager.thrift +++ b/core/src/main/thrift/manager.thrift @@ -164,12 +164,23 @@ struct ManagerMonitorInfo { 9:list bulkImports } +enum TFateInstanceType { + META + USER +} + +struct TFateId { + 1:TFateInstanceType type + 2:i64 tid +} + service FateService { // register a fate operation by reserving an opid - i64 beginFateOperation( + TFateId beginFateOperation( 1:client.TInfo tinfo 2:security.TCredentials credentials + 3:TFateInstanceType type ) throws ( 1:client.ThriftSecurityException sec 2:client.ThriftNotActiveServiceException tnase @@ -179,7 +190,7 @@ service FateService { void executeFateOperation( 1:client.TInfo tinfo 2:security.TCredentials credentials - 3:i64 opid + 3:TFateId opid 4:FateOperation op 5:list arguments 6:map options @@ -194,7 +205,7 @@ service FateService { string waitForFateOperation( 1:client.TInfo tinfo 2:security.TCredentials credentials - 3:i64 opid + 3:TFateId opid ) throws ( 1:client.ThriftSecurityException sec 2:client.ThriftTableOperationException tope @@ -205,7 +216,7 @@ service FateService { void finishFateOperation( 1:client.TInfo tinfo 2:security.TCredentials credentials - 3:i64 opid + 3:TFateId opid ) throws ( 1:client.ThriftSecurityException sec 2:client.ThriftNotActiveServiceException tnase @@ -215,7 +226,7 @@ service FateService { bool cancelFateOperation( 1:client.TInfo tinfo 2:security.TCredentials credentials - 3:i64 opid + 3:TFateId opid ) throws ( 1:client.ThriftSecurityException sec 2:client.ThriftNotActiveServiceException tnase diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java b/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java index 21c38ff8a07..12cfd0ccbfa 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java +++ b/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java @@ -34,6 +34,7 @@ import org.apache.accumulo.core.client.admin.TimeType; import org.apache.accumulo.core.clientImpl.TabletHostingGoalUtil; import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.conf.SiteConfiguration; import org.apache.accumulo.core.crypto.CryptoFactoryLoader; import org.apache.accumulo.core.data.InstanceId; @@ -43,9 +44,11 @@ import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.file.FileOperations; import org.apache.accumulo.core.file.FileSKVWriter; +import org.apache.accumulo.core.metadata.FateTable; import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.ReferencedTabletFile; import org.apache.accumulo.core.metadata.RootTable; +import org.apache.accumulo.core.metadata.StoredTabletFile; import org.apache.accumulo.core.metadata.schema.DataFileValue; import org.apache.accumulo.core.metadata.schema.MetadataSchema; import org.apache.accumulo.core.metadata.schema.MetadataTime; @@ -104,6 +107,13 @@ void initialize(VolumeManager fs, String rootTabletDirUri, String rootTabletFile String tableMetadataTabletDirUri = fs.choose(chooserEnv, context.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + MetadataTable.ID + Path.SEPARATOR + tableMetadataTabletDirName; + chooserEnv = new VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.Scope.INIT, FateTable.ID, + null, context); + String fateTableDefaultTabletDirName = + MetadataSchema.TabletsSection.ServerColumnFamily.DEFAULT_TABLET_DIR_NAME; + String fateTableDefaultTabletDirUri = + fs.choose(chooserEnv, context.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + + FateTable.ID + Path.SEPARATOR + fateTableDefaultTabletDirName; chooserEnv = new VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.Scope.INIT, MetadataTable.ID, null, context); String defaultMetadataTabletDirName = @@ -113,11 +123,19 @@ void initialize(VolumeManager fs, String rootTabletDirUri, String rootTabletFile + MetadataTable.ID + Path.SEPARATOR + defaultMetadataTabletDirName; // create table and default tablets directories - createDirectories(fs, rootTabletDirUri, tableMetadataTabletDirUri, defaultMetadataTabletDirUri); + createDirectories(fs, rootTabletDirUri, tableMetadataTabletDirUri, defaultMetadataTabletDirUri, + fateTableDefaultTabletDirUri); + + String ext = FileOperations.getNewFileExtension(DefaultConfiguration.getInstance()); + + // populate the metadata tablet with info about the fate tablet + String metadataFileName = tableMetadataTabletDirUri + Path.SEPARATOR + "0_1." + ext; + Tablet fateTablet = new Tablet(FateTable.ID, fateTableDefaultTabletDirName, null, null); + createMetadataFile(fs, metadataFileName, siteConfig, fateTablet); // populate the root tablet with info about the metadata table's two initial tablets - Tablet tablesTablet = - new Tablet(MetadataTable.ID, tableMetadataTabletDirName, null, splitPoint); + Tablet tablesTablet = new Tablet(MetadataTable.ID, tableMetadataTabletDirName, null, splitPoint, + StoredTabletFile.of(new Path(metadataFileName)).getMetadata()); Tablet defaultTablet = new Tablet(MetadataTable.ID, defaultMetadataTabletDirName, splitPoint, null); createMetadataFile(fs, rootTabletFileUri, siteConfig, tablesTablet, defaultTablet); @@ -148,6 +166,7 @@ private void initSystemTablesConfig(final ServerContext context) setTableProperties(context, RootTable.ID, initConfig.getRootMetaConf()); setTableProperties(context, MetadataTable.ID, initConfig.getRootMetaConf()); setTableProperties(context, MetadataTable.ID, initConfig.getMetaTableConf()); + setTableProperties(context, FateTable.ID, initConfig.getFateTableConf()); } private void setTableProperties(final ServerContext context, TableId tableId, diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java index bd083b00c24..720140356e4 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java +++ b/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java @@ -41,6 +41,8 @@ class InitialConfiguration { private final HashMap initialRootMetaConf = new HashMap<>(); // config for only metadata table private final HashMap initialMetaConf = new HashMap<>(); + // config for only fate table + private final HashMap initialFateTableConf = new HashMap<>(); private final Configuration hadoopConf; private final SiteConfiguration siteConf; @@ -88,6 +90,8 @@ class InitialConfiguration { SimpleCompactionDispatcher.class.getName()); initialMetaConf.put(Property.TABLE_COMPACTION_DISPATCHER_OPTS.getKey() + "service", "meta"); + // TODO configure initial fate table config.. probably needs compaction config?? + int max = hadoopConf.getInt("dfs.replication.max", 512); // Hadoop 0.23 switched the min value configuration name int min = Math.max(hadoopConf.getInt("dfs.replication.min", 1), @@ -126,6 +130,10 @@ HashMap getMetaTableConf() { return initialMetaConf; } + HashMap getFateTableConf() { + return initialFateTableConf; + } + Configuration getHadoopConf() { return hadoopConf; } diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java index 96e4a541ec6..cbf59bc52cb 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java +++ b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java @@ -34,6 +34,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.manager.thrift.ManagerGoalState; +import org.apache.accumulo.core.metadata.FateTable; import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.metadata.StoredTabletFile; @@ -125,6 +126,8 @@ void initialize(final ServerContext context, final boolean clearInstanceName, RootTable.NAME, TableState.ONLINE, ZooUtil.NodeExistsPolicy.FAIL); TableManager.prepareNewTableState(context, MetadataTable.ID, Namespace.ACCUMULO.id(), MetadataTable.NAME, TableState.ONLINE, ZooUtil.NodeExistsPolicy.FAIL); + TableManager.prepareNewTableState(context, FateTable.ID, Namespace.ACCUMULO.id(), + FateTable.NAME, TableState.ONLINE, ZooUtil.NodeExistsPolicy.FAIL); zoo.putPersistentData(zkInstanceRoot + Constants.ZTSERVERS, EMPTY_BYTE_ARRAY, ZooUtil.NodeExistsPolicy.FAIL); zoo.putPersistentData(zkInstanceRoot + Constants.ZPROBLEMS, EMPTY_BYTE_ARRAY, diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java index 144013dc1eb..a26ed3f205b 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java @@ -54,13 +54,17 @@ import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.AdminUtil; +import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.FateTxId; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.ZooStore; +import org.apache.accumulo.core.fate.accumulo.AccumuloStore; import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.manager.thrift.FateService; +import org.apache.accumulo.core.manager.thrift.TFateId; +import org.apache.accumulo.core.manager.thrift.TFateInstanceType; import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; @@ -762,6 +766,9 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps String fateZkPath = zkRoot + Constants.ZFATE; ZooReaderWriter zk = context.getZooReaderWriter(); ZooStore zs = new ZooStore<>(fateZkPath, zk); + AccumuloStore as = new AccumuloStore<>(context); + Map> fateStores = + Map.of(FateInstanceType.META, zs, FateInstanceType.USER, as); if (fateOpsCommand.cancel) { cancelSubmittedFateTxs(context, fateOpsCommand.txList); @@ -785,13 +792,14 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps fateOpsCommand.txList.forEach(s -> sortedTxs.add(parseTidFromUserInput(s))); EnumSet statusFilter = getCmdLineStatusFilters(fateOpsCommand.states); - admin.print(zs, zk, zTableLocksPath, new Formatter(System.out), sortedTxs, statusFilter); + admin.print(fateStores, zk, zTableLocksPath, new Formatter(System.out), sortedTxs, + statusFilter); // print line break at the end System.out.println(); } if (fateOpsCommand.summarize) { - summarizeFateTx(context, fateOpsCommand, admin, zs, zTableLocksPath); + summarizeFateTx(context, fateOpsCommand, admin, fateStores, zTableLocksPath); } } @@ -809,8 +817,10 @@ private void validateFateUserInput(FateOpsCommand cmd) { private void cancelSubmittedFateTxs(ServerContext context, List txList) throws AccumuloException { for (String txStr : txList) { + // TODO: We need to pass and then parse the instance type to create TFateId, + // maybe something like :txid long txid = Long.parseLong(txStr, 16); - boolean cancelled = cancelFateOperation(context, txid); + boolean cancelled = cancelFateOperation(context, new TFateId(TFateInstanceType.META, txid)); if (cancelled) { System.out.println("FaTE transaction " + FateTxId.formatTid(txid) + " was cancelled or already completed."); @@ -821,7 +831,8 @@ private void cancelSubmittedFateTxs(ServerContext context, List txList) } } - private boolean cancelFateOperation(ClientContext context, long txid) throws AccumuloException { + private boolean cancelFateOperation(ClientContext context, TFateId txid) + throws AccumuloException { FateService.Client client = null; try { client = ThriftClientTypes.FATE.getConnectionWithRetry(context); @@ -836,11 +847,12 @@ private boolean cancelFateOperation(ClientContext context, long txid) throws Acc } private void summarizeFateTx(ServerContext context, FateOpsCommand cmd, AdminUtil admin, - ReadOnlyFateStore zs, ServiceLock.ServiceLockPath tableLocksPath) + Map> fateStores, + ServiceLock.ServiceLockPath tableLocksPath) throws InterruptedException, AccumuloException, AccumuloSecurityException, KeeperException { ZooReaderWriter zk = context.getZooReaderWriter(); - var transactions = admin.getStatus(zs, zk, tableLocksPath, null, null); + var transactions = admin.getStatus(fateStores, zk, tableLocksPath, null, null); // build id map - relies on unique ids for tables and namespaces // used to look up the names of either table or namespace by id. diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java index 08ee56db306..d8567ca2a28 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java @@ -68,10 +68,13 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.dataImpl.thrift.TRange; +import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus; import org.apache.accumulo.core.manager.thrift.BulkImportState; import org.apache.accumulo.core.manager.thrift.FateOperation; import org.apache.accumulo.core.manager.thrift.FateService; +import org.apache.accumulo.core.manager.thrift.TFateId; +import org.apache.accumulo.core.manager.thrift.TFateInstanceType; import org.apache.accumulo.core.manager.thrift.ThriftPropertyException; import org.apache.accumulo.core.securityImpl.thrift.TCredentials; import org.apache.accumulo.core.util.ByteBufferUtil; @@ -118,18 +121,20 @@ public FateServiceHandler(Manager manager) { } @Override - public long beginFateOperation(TInfo tinfo, TCredentials credentials) + public TFateId beginFateOperation(TInfo tinfo, TCredentials credentials, TFateInstanceType type) throws ThriftSecurityException { authenticate(credentials); - return manager.fate().startTransaction(); + return new TFateId(type, manager.fate(FateInstanceType.fromThrift(type)).startTransaction()); } @Override - public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOperation op, + public void executeFateOperation(TInfo tinfo, TCredentials c, TFateId opid, FateOperation op, List arguments, Map options, boolean autoCleanup) throws ThriftSecurityException, ThriftTableOperationException, ThriftPropertyException { authenticate(c); String goalMessage = op.toString() + " "; + long tid = opid.getTid(); + FateInstanceType type = FateInstanceType.fromThrift(opid.getType()); switch (op) { case NAMESPACE_CREATE: { @@ -142,7 +147,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage += "Create " + namespace + " namespace."; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new CreateNamespace(c.getPrincipal(), namespace, options)), autoCleanup, goalMessage); break; @@ -161,7 +166,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage += "Rename " + oldName + " namespace to " + newName; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new RenameNamespace(namespaceId, oldName, newName)), autoCleanup, goalMessage); break; @@ -179,7 +184,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage += "Delete namespace Id: " + namespaceId; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new DeleteNamespace(namespaceId)), autoCleanup, goalMessage); break; } @@ -242,7 +247,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe goalMessage += "Create table " + tableName + " " + initialTableState + " with " + splitCount + " splits and initial hosting goal of " + initialHostingGoal; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new CreateTable(c.getPrincipal(), tableName, timeType, options, splitsPath, splitCount, splitsDirsPath, initialTableState, initialHostingGoal, namespaceId)), @@ -278,7 +283,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe goalMessage += "Rename table " + oldTableName + "(" + tableId + ") to " + oldTableName; try { - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new RenameTable(namespaceId, tableId, oldTableName, newTableName)), autoCleanup, goalMessage); } catch (NamespaceNotFoundException e) { @@ -349,8 +354,8 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe goalMessage += " and keep offline."; } - manager.fate().seedTransaction( - op.toString(), opid, new TraceRepo<>(new CloneTable(c.getPrincipal(), namespaceId, + manager.fate(type).seedTransaction( + op.toString(), tid, new TraceRepo<>(new CloneTable(c.getPrincipal(), namespaceId, srcTableId, tableName, propertiesToSet, propertiesToExclude, keepOffline)), autoCleanup, goalMessage); @@ -379,7 +384,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage += "Delete table " + tableName + "(" + tableId + ")"; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new PreDeleteTable(namespaceId, tableId)), autoCleanup, goalMessage); break; } @@ -402,7 +407,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage += "Online table " + tableId; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new ChangeTableState(namespaceId, tableId, tableOp)), autoCleanup, goalMessage); break; @@ -427,7 +432,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage += "Offline table " + tableId; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new ChangeTableState(namespaceId, tableId, tableOp)), autoCleanup, goalMessage); break; @@ -462,7 +467,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe startRowStr, endRowStr); goalMessage += "Merge table " + tableName + "(" + tableId + ") splits from " + startRowStr + " to " + endRowStr; - manager.fate().seedTransaction(op.toString(), opid, new TraceRepo<>( + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>( new TableRangeOp(MergeInfo.Operation.MERGE, namespaceId, tableId, startRow, endRow)), autoCleanup, goalMessage); break; @@ -494,7 +499,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe goalMessage += "Delete table " + tableName + "(" + tableId + ") range " + startRow + " to " + endRow; - manager.fate().seedTransaction(op.toString(), opid, new TraceRepo<>( + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>( new TableRangeOp(MergeInfo.Operation.DELETE, namespaceId, tableId, startRow, endRow)), autoCleanup, goalMessage); break; @@ -520,7 +525,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage += "Compact table (" + tableId + ") with config " + compactionConfig; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new CompactRange(namespaceId, tableId, compactionConfig)), autoCleanup, goalMessage); break; @@ -544,7 +549,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage += "Cancel compaction of table (" + tableId + ")"; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new CancelCompactions(namespaceId, tableId)), autoCleanup, goalMessage); break; } @@ -585,8 +590,8 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage += "Import table with new name: " + tableName + " from " + exportDirs; - manager.fate() - .seedTransaction(op.toString(), opid, new TraceRepo<>(new ImportTable(c.getPrincipal(), + manager.fate(type) + .seedTransaction(op.toString(), tid, new TraceRepo<>(new ImportTable(c.getPrincipal(), tableName, exportDirs, namespaceId, keepMappings, keepOffline)), autoCleanup, goalMessage); break; @@ -615,7 +620,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage += "Export table " + tableName + "(" + tableId + ") to " + exportDir; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new ExportTable(namespaceId, tableName, tableId, exportDir)), autoCleanup, goalMessage); break; @@ -652,7 +657,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe manager.updateBulkImportStatus(dir, BulkImportState.INITIAL); goalMessage += "Bulk import (v2) " + dir + " to " + tableName + "(" + tableId + ")"; - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new PrepBulkImport(tableId, dir, setTime)), autoCleanup, goalMessage); break; } @@ -694,7 +699,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe goalMessage += "Set Hosting Goal for table: " + tableName + "(" + tableId + ") range: " + tRange + " to: " + goal.name(); - manager.fate().seedTransaction(op.toString(), opid, + manager.fate(type).seedTransaction(op.toString(), tid, new TraceRepo<>(new SetHostingGoal(tableId, namespaceId, tRange, goal)), autoCleanup, goalMessage); break; @@ -771,7 +776,7 @@ public void executeFateOperation(TInfo tinfo, TCredentials c, long opid, FateOpe } goalMessage = "Splitting " + extent + " for user into " + (splits.size() + 1) + " tablets"; - manager.fate().seedTransaction(op.toString(), opid, new PreSplit(extent, splits), + manager.fate(type).seedTransaction(op.toString(), tid, new PreSplit(extent, splits), autoCleanup, goalMessage); break; } @@ -818,13 +823,14 @@ private void throwIfTableMissingSecurityException(ThriftSecurityException e, Tab } @Override - public String waitForFateOperation(TInfo tinfo, TCredentials credentials, long opid) + public String waitForFateOperation(TInfo tinfo, TCredentials credentials, TFateId opid) throws ThriftSecurityException, ThriftTableOperationException { authenticate(credentials); - TStatus status = manager.fate().waitForCompletion(opid); + FateInstanceType type = FateInstanceType.fromThrift(opid.getType()); + TStatus status = manager.fate(type).waitForCompletion(opid.getTid()); if (status == TStatus.FAILED) { - Exception e = manager.fate().getException(opid); + Exception e = manager.fate(type).getException(opid.getTid()); if (e instanceof ThriftTableOperationException) { throw (ThriftTableOperationException) e; } else if (e instanceof ThriftSecurityException) { @@ -836,7 +842,7 @@ public String waitForFateOperation(TInfo tinfo, TCredentials credentials, long o } } - String ret = manager.fate().getReturn(opid); + String ret = manager.fate(type).getReturn(opid.getTid()); if (ret == null) { ret = ""; // thrift does not like returning null } @@ -844,10 +850,10 @@ public String waitForFateOperation(TInfo tinfo, TCredentials credentials, long o } @Override - public void finishFateOperation(TInfo tinfo, TCredentials credentials, long opid) + public void finishFateOperation(TInfo tinfo, TCredentials credentials, TFateId opid) throws ThriftSecurityException { authenticate(credentials); - manager.fate().delete(opid); + manager.fate(FateInstanceType.fromThrift(opid.getType())).delete(opid.getTid()); } protected void authenticate(TCredentials credentials) throws ThriftSecurityException { @@ -926,9 +932,10 @@ private void writeSplitsToFile(Path splitsPath, final List arguments * * @return the path of the created directory */ - public Path mkTempDir(long opid) throws IOException { + public Path mkTempDir(TFateId opid) throws IOException { Volume vol = manager.getVolumeManager().getFirst(); - Path p = vol.prefixChild("/tmp/fate-" + FastFormat.toHexString(opid)); + Path p = vol + .prefixChild("/tmp/fate-" + opid.getType() + "-" + FastFormat.toHexString(opid.getTid())); FileSystem fs = vol.getFileSystem(); if (fs.exists(p)) { fs.delete(p, true); @@ -938,7 +945,7 @@ public Path mkTempDir(long opid) throws IOException { } @Override - public boolean cancelFateOperation(TInfo tinfo, TCredentials credentials, long opid) + public boolean cancelFateOperation(TInfo tinfo, TCredentials credentials, TFateId opid) throws ThriftSecurityException, ThriftNotActiveServiceException { if (!manager.security.canPerformSystemActions(credentials)) { @@ -946,7 +953,6 @@ public boolean cancelFateOperation(TInfo tinfo, TCredentials credentials, long o SecurityErrorCode.PERMISSION_DENIED); } - return manager.fate().cancel(opid); + return manager.fate(FateInstanceType.fromThrift(opid.getType())).cancel(opid.getTid()); } - } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java index 2418dba1fee..cfb311f3833 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java @@ -73,6 +73,10 @@ import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.fate.AgeOffStore; import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.FateStore; +import org.apache.accumulo.core.fate.ZooStore; +import org.apache.accumulo.core.fate.accumulo.AccumuloStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.lock.ServiceLock; @@ -94,6 +98,7 @@ import org.apache.accumulo.core.manager.thrift.ManagerState; import org.apache.accumulo.core.manager.thrift.TableInfo; import org.apache.accumulo.core.manager.thrift.TabletServerStatus; +import org.apache.accumulo.core.metadata.FateTable; import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.metadata.TServerInstance; @@ -155,6 +160,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSortedMap; import com.google.common.collect.Maps; import com.google.common.util.concurrent.RateLimiter; @@ -210,11 +216,12 @@ public class Manager extends AbstractServer private ManagerState state = ManagerState.INITIAL; - // fateReadyLatch and fateRef go together; when this latch is ready, then the fate reference - // should already have been set; still need to use atomic reference or volatile for fateRef, so no - // thread's cached view shows that fateRef is still null after the latch is ready + // fateReadyLatch and fateRefs go together; when this latch is ready, then the fate references + // should already have been set; ConcurrentHashMap will guarantee that all threads will see + // the initialized fate references after the latch is ready private final CountDownLatch fateReadyLatch = new CountDownLatch(1); - private final AtomicReference> fateRef = new AtomicReference<>(null); + private final AtomicReference>> fateRefs = + new AtomicReference<>(); volatile SortedMap tserverStatus = emptySortedMap(); volatile SortedMap tserverStatusForBalancer = emptySortedMap(); @@ -265,7 +272,7 @@ public boolean stillManager() { * * @return the Fate object, only after the fate components are running and ready */ - public Fate fate() { + public Fate fate(FateInstanceType type) { try { // block up to 30 seconds until it's ready; if it's still not ready, introduce some logging if (!fateReadyLatch.await(30, SECONDS)) { @@ -286,7 +293,7 @@ public Fate fate() { Thread.currentThread().interrupt(); throw new IllegalStateException("Thread was interrupted; cannot proceed"); } - return fateRef.get(); + return getFateRefs().get(type); } static final boolean X = true; @@ -329,7 +336,7 @@ synchronized void setManagerState(ManagerState newState) { } if (oldState != newState && (newState == ManagerState.NORMAL)) { - if (fateRef.get() != null) { + if (!getFateRefs().isEmpty()) { throw new IllegalStateException("Access to Fate should not have been" + " initialized prior to the Manager finishing upgrades. Please save" + " all logs and file a bug."); @@ -1068,17 +1075,17 @@ boolean canSuspendTablets() { } try { - final AgeOffStore store = new AgeOffStore<>( - new org.apache.accumulo.core.fate.ZooStore<>(getZooKeeperRoot() + Constants.ZFATE, - context.getZooReaderWriter()), - HOURS.toMillis(8), System::currentTimeMillis); - - Fate f = new Fate<>(this, store, TraceRepo::toLogString, getConfiguration()); - fateRef.set(f); + var metaInstance = initializeFateInstance(context, FateInstanceType.META, + new ZooStore<>(getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter())); + var userInstance = initializeFateInstance(context, FateInstanceType.USER, + new AccumuloStore<>(context, FateTable.NAME)); + + if (!fateRefs.compareAndSet(null, + Map.of(FateInstanceType.META, metaInstance, FateInstanceType.USER, userInstance))) { + throw new IllegalStateException( + "Unexpected previous fate reference map already initialized"); + } fateReadyLatch.countDown(); - - ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() - .scheduleWithFixedDelay(store::ageOff, 63000, 63000, MILLISECONDS)); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException("Exception setting up FaTE cleanup thread", e); } @@ -1139,7 +1146,7 @@ boolean canSuspendTablets() { sleepUninterruptibly(500, MILLISECONDS); } log.info("Shutting down fate."); - fate().shutdown(); + getFateRefs().keySet().forEach(type -> fate(type).shutdown()); splitter.stop(); @@ -1178,6 +1185,20 @@ boolean canSuspendTablets() { log.info("exiting"); } + private Fate initializeFateInstance(ServerContext context, FateInstanceType type, + FateStore store) { + final AgeOffStore ageOffStore = + new AgeOffStore<>(store, HOURS.toMillis(8), System::currentTimeMillis); + + final Fate fateInstance = + new Fate<>(this, ageOffStore, TraceRepo::toLogString, getConfiguration()); + + ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor() + .scheduleWithFixedDelay(ageOffStore::ageOff, 63000, 63000, MILLISECONDS)); + + return fateInstance; + } + /** * Allows property configuration to block manager start-up waiting for a minimum number of * tservers to register in zookeeper. It also accepts a maximum time to wait - if the time @@ -1656,4 +1677,10 @@ public void registerMetrics(MeterRegistry registry) { super.registerMetrics(registry); compactionCoordinator.registerMetrics(registry); } + + private Map> getFateRefs() { + var fateRefs = this.fateRefs.get(); + Preconditions.checkState(fateRefs != null, "Unexpected null fate references map"); + return fateRefs; + } } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java index 97b9a07682f..4d3980a9ed1 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java @@ -57,6 +57,7 @@ import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent; import org.apache.accumulo.core.fate.Fate; +import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.manager.thrift.ManagerClientService; import org.apache.accumulo.core.manager.thrift.ManagerGoalState; @@ -329,7 +330,7 @@ public void shutdownTabletServer(TInfo info, TCredentials c, String tabletServer } } - Fate fate = manager.fate(); + Fate fate = manager.fate(FateInstanceType.META); long tid = fate.startTransaction(); String msg = "Shutdown tserver " + tabletServer; diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java index e31ff65398e..4f99d015861 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java @@ -24,6 +24,7 @@ import java.util.TreeMap; import org.apache.accumulo.core.fate.AdminUtil; +import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.server.ServerContext; import org.apache.zookeeper.KeeperException; @@ -110,7 +111,7 @@ public static FateMetricValues getFromZooKeeper(final ServerContext context, try { List currFates = - admin.getTransactionStatus(zooStore, null, null); + admin.getTransactionStatus(Map.of(FateInstanceType.META, zooStore), null, null); builder.withCurrentFateOps(currFates.size()); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/split/SplitTask.java b/server/manager/src/main/java/org/apache/accumulo/manager/split/SplitTask.java index 0c60e9347f2..18958c92be0 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/split/SplitTask.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/split/SplitTask.java @@ -21,6 +21,7 @@ import java.time.Duration; import java.util.SortedSet; +import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.tableOps.split.PreSplit; @@ -83,9 +84,11 @@ public void run() { return; } - long fateTxId = manager.fate().startTransaction(); + var fateInstanceType = FateInstanceType.fromTableId((tablet.getTableId())); + long fateTxId = manager.fate(fateInstanceType).startTransaction(); - manager.fate().seedTransaction("SYSTEM_SPLIT", fateTxId, new PreSplit(extent, splits), true, + manager.fate(fateInstanceType).seedTransaction("SYSTEM_SPLIT", fateTxId, + new PreSplit(extent, splits), true, "System initiated split of tablet " + extent + " into " + splits.size() + " splits"); } catch (Exception e) { log.error("Failed to split {}", tablet.getExtent(), e); diff --git a/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java b/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java index 013a1e7dbd6..dad27822884 100644 --- a/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java +++ b/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java @@ -188,10 +188,11 @@ public void testMetadataTableSplit() throws Exception { private void verifyMetadataTableScan(AccumuloClient client) throws Exception { var tables = client.tableOperations().tableIdMap(); var expectedExtents = tables.entrySet().stream() - .filter(e -> !e.getKey().startsWith("accumulo.")).map(Map.Entry::getValue).map(TableId::of) - .map(tid -> new KeyExtent(tid, null, null)).collect(Collectors.toSet()); - // Verify we have 10 tablets for metadata - assertEquals(10, expectedExtents.size()); + .filter(e -> !e.getKey().equals(RootTable.NAME) && !e.getKey().equals(MetadataTable.NAME)) + .map(Map.Entry::getValue).map(TableId::of).map(tid -> new KeyExtent(tid, null, null)) + .collect(Collectors.toSet()); + // Verify we have 11 tablets for metadata (Includes FateTable) + assertEquals(11, expectedExtents.size()); // Scan each tablet to verify data exists var ample = ((ClientContext) client).getAmple(); diff --git a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java index 63b3240e58e..6df062d2fbe 100644 --- a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java +++ b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java @@ -135,7 +135,7 @@ public void setupConnectorAndNamespace() { c.tableOperations().delete(t); } } - assertEquals(2, c.tableOperations().list().size()); + assertEquals(3, c.tableOperations().list().size()); for (String n : c.namespaceOperations().list()) { if (!n.equals(Namespace.ACCUMULO.name()) && !n.equals(Namespace.DEFAULT.name())) { c.namespaceOperations().delete(n); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java index a9e0a437a82..68c24ff3cb0 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java @@ -30,6 +30,7 @@ import java.time.Duration; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -43,11 +44,15 @@ import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.Namespace; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.AdminUtil; +import org.apache.accumulo.core.fate.FateInstanceType; +import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.ZooStore; +import org.apache.accumulo.core.fate.accumulo.AccumuloStore; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.lock.ServiceLock; @@ -192,11 +197,14 @@ public void changeTableStateTest() throws Exception { } private boolean findFate(String aTableName) { + boolean isMeta = aTableName.startsWith(Namespace.ACCUMULO.name()); log.debug("Look for fate {}", aTableName); for (int retry = 0; retry < 5; retry++) { try { - boolean found = lookupFateInZookeeper(aTableName); - log.trace("Try {}: Fate in zk for table {} : {}", retry, aTableName, found); + boolean found = + isMeta ? lookupFateInZookeeper(aTableName) : lookupFateInAccumulo(aTableName); + log.trace("Try {}: Fate in {} for table {} : {}", retry, isMeta ? "zk" : "accumulo", + aTableName, found); if (found) { log.debug("Found fate {}", aTableName); return true; @@ -257,11 +265,14 @@ public void getFateStatus() { ZooStore zs = new ZooStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); + AccumuloStore as = new AccumuloStore<>(context); + Map> fateStores = + Map.of(FateInstanceType.META, zs, FateInstanceType.USER, as); - withLocks = admin.getStatus(zs, zk, lockPath, null, null); + withLocks = admin.getStatus(fateStores, zk, lockPath, null, null); // call method that does not use locks. - noLocks = admin.getTransactionStatus(zs, null, null); + noLocks = admin.getTransactionStatus(fateStores, null, null); // no zk exception, no need to retry break; @@ -366,6 +377,33 @@ private boolean lookupFateInZookeeper(final String tableName) throws KeeperExcep return Boolean.FALSE; } + private boolean lookupFateInAccumulo(final String tableName) throws KeeperException { + AdminUtil admin = new AdminUtil<>(false); + + try { + TableId tableId = context.getTableId(tableName); + + log.trace("tid: {}", tableId); + + AccumuloStore as = new AccumuloStore<>(context); + AdminUtil.FateStatus fateStatus = admin.getStatus(as, null, null); + + log.trace("current fates: {}", fateStatus.getTransactions().size()); + + for (AdminUtil.TransactionStatus tx : fateStatus.getTransactions()) { + if (isCompaction(tx)) { + return true; + } + } + + } catch (TableNotFoundException | InterruptedException ex) { + throw new IllegalStateException(ex); + } + + // did not find appropriate fate transaction for compaction. + return Boolean.FALSE; + } + /** * Test that the transaction top contains "CompactionDriver" and the debug message contains * "CompactRange" diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java index 2d7a71aff34..c2c5b7d0cfb 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java @@ -62,6 +62,7 @@ import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.harness.AccumuloClusterHarness; +import org.apache.accumulo.minicluster.MemoryUnit; import org.apache.accumulo.minicluster.ServerType; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.accumulo.server.util.CheckForMetadataProblems; @@ -92,6 +93,7 @@ protected Duration defaultTimeout() { @Override public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) { cfg.setProperty(Property.TSERV_MAXMEM, "5K"); + cfg.setMemory(ServerType.TABLET_SERVER, 384, MemoryUnit.MEGABYTE); } private String tservMaxMem, tservMajcDelay; diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java index 06c3bec142a..0213a7f48d2 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletManagementIteratorIT.java @@ -61,6 +61,7 @@ import org.apache.accumulo.core.manager.state.TabletManagement; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.manager.thrift.ManagerState; +import org.apache.accumulo.core.metadata.FateTable; import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.StoredTabletFile; import org.apache.accumulo.core.metadata.TServerInstance; @@ -400,10 +401,10 @@ private void copyTable(AccumuloClient client, String source, String copy) while (row.hasNext()) { Entry entry = row.next(); Key k = entry.getKey(); + if (m == null) { m = new Mutation(k.getRow()); } - m.put(k.getColumnFamily(), k.getColumnQualifier(), k.getColumnVisibilityParsed(), k.getTimestamp(), entry.getValue()); } @@ -412,9 +413,10 @@ private void copyTable(AccumuloClient client, String source, String copy) } } - // metadata should be stable with only 6 rows (2 for each table) + // metadata should be stable with only 9 rows (2 for each table) + 1 for the FateTable log.debug("Gathered {} rows to create copy {}", mutations.size(), copy); - assertEquals(8, mutations.size(), "Metadata should have 8 rows (2 for each table)"); + assertEquals(9, mutations.size(), + "Metadata should have 8 rows (2 for each table) + one row for " + FateTable.ID.canonical()); client.tableOperations().create(copy); try (BatchWriter writer = client.createBatchWriter(copy)) { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java index 8114b1b9506..4a93d5acc19 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java @@ -49,6 +49,7 @@ import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.metadata.FateTable; import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection; @@ -122,7 +123,8 @@ public void test() throws Exception { assertEquals(3, countInUse(walsAfterRoll.values()), "all WALs should be in use"); // flush the tables - for (String table : new String[] {tableName, MetadataTable.NAME, RootTable.NAME}) { + for (String table : new String[] {tableName, MetadataTable.NAME, RootTable.NAME, + FateTable.NAME}) { c.tableOperations().flush(table, null, null, true); } Thread.sleep(SECONDS.toMillis(1)); @@ -148,9 +150,14 @@ public void test() throws Exception { Thread.sleep(SECONDS.toMillis(5)); Map> markers = getRecoveryMarkers(c); // log.debug("markers " + markers); - assertEquals(1, markers.size(), "one tablet should have markers"); - assertEquals("1", markers.keySet().iterator().next().tableId().canonical(), + // There should be markers for the created table and also the FateTable + assertEquals(2, markers.size(), "two tablets should have markers"); + assertTrue( + markers.keySet().stream().anyMatch(extent -> extent.tableId().canonical().equals("1")), "tableId of the keyExtent should be 1"); + assertTrue( + markers.keySet().stream().anyMatch(extent -> extent.tableId().equals(FateTable.ID)), + "tableId of the FateTable can't be found"); // put some data in the WAL assertEquals(0, cluster.exec(SetGoalState.class, "NORMAL").getProcess().waitFor());