From aee70f3919016f437ad522ead9db623a61edb1e8 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Sun, 19 Jan 2025 16:32:36 -0500 Subject: [PATCH] wip --- .../core/client/admin/TabletInformation.java | 6 + .../core/clientImpl/TableOperationsImpl.java | 38 +- .../clientImpl/TabletInformationImpl.java | 6 + .../clientImpl/TabletMergeabilityUtil.java | 15 + .../manager/thrift/ManagerClientService.java | 1610 +++++++++++++++++ .../manager/thrift/TTabletMergeability.java | 497 +++++ core/src/main/thrift/manager.thrift | 15 + .../manager/ManagerClientServiceHandler.java | 50 + .../accumulo/test/functional/AddSplitIT.java | 12 + 9 files changed, 2243 insertions(+), 6 deletions(-) create mode 100644 core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TTabletMergeability.java diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TabletInformation.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TabletInformation.java index f19825f9b11..b8580d32b07 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/TabletInformation.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TabletInformation.java @@ -76,4 +76,10 @@ public interface TabletInformation { */ TabletAvailability getTabletAvailability(); + /** + * @return the TabletMergeability object + * + * @since 4.0.0 + */ + TabletMergeability getTabletMergeability(); } 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 9f70d038a9a..7011641e41c 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 @@ -33,6 +33,7 @@ import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LAST; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOGS; +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.MERGEABILITY; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.OPID; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.SUSPEND; @@ -104,6 +105,7 @@ import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.client.admin.SummaryRetriever; import org.apache.accumulo.core.client.admin.TableOperations; +import org.apache.accumulo.core.client.admin.TableOperations.ImportDestinationArguments; import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.client.admin.TabletInformation; import org.apache.accumulo.core.client.admin.TabletMergeability; @@ -510,11 +512,22 @@ public void putSplits(String tableName, SortedMap split tabLocator.invalidateCache(); + var splitsToTablets = mapSplitsToTablets(tableName, tableId, tabLocator, splitsTodo); Map>> tabletSplits = - mapSplitsToTablets(tableName, tableId, tabLocator, splitsTodo); + splitsToTablets.newSplits; + Map existingSplits = + splitsToTablets.existingSplits; List> futures = new ArrayList<>(); + // TODO: Process existingSplits in a new RPC call and updating existing + // before continuing to the fate operations to add new. ie + // if (doRpcToUpdateMergability(existingSplits)) { + // remove from splitsTodo + // } else { + // handle errors, retry? + // } + // begin the fate operation for each tablet without waiting for the operation to complete for (Entry>> splitsForTablet : tabletSplits .entrySet()) { @@ -606,10 +619,11 @@ public void putSplits(String tableName, SortedMap split waitExecutor.shutdown(); } - private Map>> mapSplitsToTablets(String tableName, + private SplitsToTablets mapSplitsToTablets(String tableName, TableId tableId, ClientTabletCache tabLocator, SortedMap splitsTodo) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { - Map>> tabletSplits = new HashMap<>(); + Map>> newSplits = new HashMap<>(); + Map existingSplits = new HashMap<>(); var iterator = splitsTodo.entrySet().iterator(); while (iterator.hasNext()) { @@ -633,12 +647,13 @@ private Map>> mapSplitsToTablets(St } if (split.equals(tablet.getExtent().endRow())) { + existingSplits.put(tablet.getExtent(), splitEntry.getValue()); // split already exists, so remove it iterator.remove(); continue; } - tabletSplits.computeIfAbsent(tablet.getExtent(), k -> new ArrayList<>()) + newSplits.computeIfAbsent(tablet.getExtent(), k -> new ArrayList<>()) .add(Pair.fromEntry(splitEntry)); } catch (InvalidTabletHostingRequestException e) { @@ -646,7 +661,18 @@ private Map>> mapSplitsToTablets(St throw new AccumuloException(e); } } - return tabletSplits; + return new SplitsToTablets(newSplits, existingSplits); + } + + private static class SplitsToTablets { + final Map>> newSplits; + final Map existingSplits; + + private SplitsToTablets(Map>> newSplits, + Map existingSplits) { + this.newSplits = Objects.requireNonNull(newSplits); + this.existingSplits = Objects.requireNonNull(existingSplits); + } } @Override @@ -2244,7 +2270,7 @@ public Stream getTabletInformation(final String tableName, fi TabletsMetadata tabletsMetadata = context.getAmple().readTablets().forTable(tableId).overlapping(scanRangeStart, true, null) - .fetch(AVAILABILITY, LOCATION, DIR, PREV_ROW, FILES, LAST, LOGS, SUSPEND) + .fetch(AVAILABILITY, LOCATION, DIR, PREV_ROW, FILES, LAST, LOGS, SUSPEND, MERGEABILITY) .checkConsistency().build(); Set liveTserverSet = TabletMetadata.getLiveTServers(context); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletInformationImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletInformationImpl.java index 62833477306..b4973bbb705 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletInformationImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletInformationImpl.java @@ -22,6 +22,7 @@ import org.apache.accumulo.core.client.admin.TabletAvailability; import org.apache.accumulo.core.client.admin.TabletInformation; +import org.apache.accumulo.core.client.admin.TabletMergeability; import org.apache.accumulo.core.data.TabletId; import org.apache.accumulo.core.dataImpl.TabletIdImpl; import org.apache.accumulo.core.metadata.schema.DataFileValue; @@ -93,6 +94,11 @@ public TabletAvailability getTabletAvailability() { return tabletMetadata.getTabletAvailability(); } + @Override + public TabletMergeability getTabletMergeability() { + return tabletMetadata.getTabletMergeability().getTabletMergeability(); + } + @Override public String toString() { return "TabletInformationImpl{tabletMetadata=" + tabletMetadata + ", estimatedSize=" diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletMergeabilityUtil.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletMergeabilityUtil.java index c4c8d3a873c..7d882d88cd5 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletMergeabilityUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletMergeabilityUtil.java @@ -30,6 +30,7 @@ import java.util.function.Function; import org.apache.accumulo.core.client.admin.TabletMergeability; +import org.apache.accumulo.core.manager.thrift.TTabletMergeability; import org.apache.accumulo.core.util.ByteBufferUtil; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.TextUtil; @@ -95,6 +96,20 @@ public static Pair decode(String data) { : TabletMergeability.after(Duration.ofNanos(jData.delay))); } + public static TabletMergeability fromThrift(TTabletMergeability thriftTm) { + if (thriftTm.never) { + return TabletMergeability.never(); + } + return TabletMergeability.after(Duration.ofNanos(thriftTm.delay)); + } + + public static TTabletMergeability toThrift(TabletMergeability tabletMergeability) { + if (tabletMergeability.isNever()) { + return new TTabletMergeability(true, -1L); + } + return new TTabletMergeability(false, tabletMergeability.getDelay().orElseThrow().toNanos()); + } + private static class GSonData { byte[] split; boolean never; diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java index 621d35dff55..ea3b1d2ff87 100644 --- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/ManagerClientService.java @@ -69,6 +69,8 @@ public interface Iface { public void requestTabletHosting(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.util.List extents) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException; + public java.util.List updateTabletMergeability(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Map splits) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException; + } public interface AsyncIface { @@ -113,6 +115,8 @@ public interface AsyncIface { public void requestTabletHosting(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, java.util.List extents, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void updateTabletMergeability(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Map splits, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws org.apache.thrift.TException; + } public static class Client extends org.apache.thrift.TServiceClient implements Iface { @@ -773,6 +777,39 @@ public void recv_requestTabletHosting() throws org.apache.accumulo.core.clientIm return; } + @Override + public java.util.List updateTabletMergeability(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Map splits) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException + { + send_updateTabletMergeability(tinfo, credentials, tableName, splits); + return recv_updateTabletMergeability(); + } + + public void send_updateTabletMergeability(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Map splits) throws org.apache.thrift.TException + { + updateTabletMergeability_args args = new updateTabletMergeability_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setTableName(tableName); + args.setSplits(splits); + sendBase("updateTabletMergeability", args); + } + + public java.util.List recv_updateTabletMergeability() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException + { + updateTabletMergeability_result result = new updateTabletMergeability_result(); + receiveBase(result, "updateTabletMergeability"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.sec != null) { + throw result.sec; + } + if (result.toe != null) { + throw result.toe; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "updateTabletMergeability failed: unknown result"); + } + } public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { @@ -1666,6 +1703,50 @@ public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.Thrift } } + @Override + public void updateTabletMergeability(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Map splits, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws org.apache.thrift.TException { + checkReady(); + updateTabletMergeability_call method_call = new updateTabletMergeability_call(tinfo, credentials, tableName, splits, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class updateTabletMergeability_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 java.lang.String tableName; + private java.util.Map splits; + public updateTabletMergeability_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableName, java.util.Map splits, 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.tableName = tableName; + this.splits = splits; + } + + @Override + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("updateTabletMergeability", org.apache.thrift.protocol.TMessageType.CALL, 0)); + updateTabletMergeability_args args = new updateTabletMergeability_args(); + args.setTinfo(tinfo); + args.setCredentials(credentials); + args.setTableName(tableName); + args.setSplits(splits); + args.write(prot); + prot.writeMessageEnd(); + } + + @Override + public java.util.List getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new java.lang.IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_updateTabletMergeability(); + } + } + } public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { @@ -1699,6 +1780,7 @@ protected Processor(I iface, java.util.Map extends org.apache.thrift.ProcessFunction { + public updateTabletMergeability() { + super("updateTabletMergeability"); + } + + @Override + public updateTabletMergeability_args getEmptyArgsInstance() { + return new updateTabletMergeability_args(); + } + + @Override + protected boolean isOneway() { + return false; + } + + @Override + protected boolean rethrowUnhandledExceptions() { + return false; + } + + @Override + public updateTabletMergeability_result getResult(I iface, updateTabletMergeability_args args) throws org.apache.thrift.TException { + updateTabletMergeability_result result = new updateTabletMergeability_result(); + try { + result.success = iface.updateTabletMergeability(args.tinfo, args.credentials, args.tableName, args.splits); + } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { + result.sec = sec; + } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException toe) { + result.toe = toe; + } + return result; + } + } + } public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { @@ -2439,6 +2555,7 @@ protected AsyncProcessor(I iface, java.util.Map extends org.apache.thrift.AsyncProcessFunction> { + public updateTabletMergeability() { + super("updateTabletMergeability"); + } + + @Override + public updateTabletMergeability_args getEmptyArgsInstance() { + return new updateTabletMergeability_args(); + } + + @Override + 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>() { + @Override + public void onComplete(java.util.List o) { + updateTabletMergeability_result result = new updateTabletMergeability_result(); + result.success = o; + try { + fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + } catch (org.apache.thrift.transport.TTransportException e) { + _LOGGER.error("TTransportException writing to internal frame buffer", e); + fb.close(); + } catch (java.lang.Exception e) { + _LOGGER.error("Exception writing to internal frame buffer", e); + onError(e); + } + } + @Override + public void onError(java.lang.Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TSerializable msg; + updateTabletMergeability_result result = new updateTabletMergeability_result(); + if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) { + result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e; + result.setSecIsSet(true); + msg = result; + } else if (e instanceof org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException) { + result.toe = (org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException) e; + result.setToeIsSet(true); + msg = result; + } else if (e instanceof org.apache.thrift.transport.TTransportException) { + _LOGGER.error("TTransportException inside handler", e); + fb.close(); + return; + } else if (e instanceof org.apache.thrift.TApplicationException) { + _LOGGER.error("TApplicationException inside handler", e); + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TApplicationException)e; + } else { + _LOGGER.error("Exception inside handler", e); + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + } catch (java.lang.Exception ex) { + _LOGGER.error("Exception writing to internal frame buffer", ex); + fb.close(); + } + } + }; + } + + @Override + protected boolean isOneway() { + return false; + } + + @Override + public void start(I iface, updateTabletMergeability_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws org.apache.thrift.TException { + iface.updateTabletMergeability(args.tinfo, args.credentials, args.tableName, args.splits,resultHandler); + } + } + } @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) @@ -28744,5 +28936,1423 @@ private static S scheme(org.apache. } } + @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) + public static class updateTabletMergeability_args 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("updateTabletMergeability_args"); + + private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField SPLITS_FIELD_DESC = new org.apache.thrift.protocol.TField("splits", org.apache.thrift.protocol.TType.MAP, (short)4); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new updateTabletMergeability_argsStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new updateTabletMergeability_argsTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required + public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required + public @org.apache.thrift.annotation.Nullable java.lang.String tableName; // required + public @org.apache.thrift.annotation.Nullable java.util.Map splits; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TINFO((short)1, "tinfo"), + CREDENTIALS((short)2, "credentials"), + TABLE_NAME((short)3, "tableName"), + SPLITS((short)4, "splits"); + + 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: // TINFO + return TINFO; + case 2: // CREDENTIALS + return CREDENTIALS; + case 3: // TABLE_NAME + return TABLE_NAME; + case 4: // SPLITS + return SPLITS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + @org.apache.thrift.annotation.Nullable + public static _Fields findByName(java.lang.String name) { + return byName.get(name); + } + + private final short _thriftId; + private final java.lang.String _fieldName; + + _Fields(short thriftId, java.lang.String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + @Override + public short getThriftFieldId() { + return _thriftId; + } + + @Override + public java.lang.String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.TInfo.class))); + tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.SPLITS, new org.apache.thrift.meta_data.FieldMetaData("splits", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTabletMergeability.class)))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateTabletMergeability_args.class, metaDataMap); + } + + public updateTabletMergeability_args() { + } + + public updateTabletMergeability_args( + org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, + org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, + java.lang.String tableName, + java.util.Map splits) + { + this(); + this.tinfo = tinfo; + this.credentials = credentials; + this.tableName = tableName; + this.splits = splits; + } + + /** + * Performs a deep copy on other. + */ + public updateTabletMergeability_args(updateTabletMergeability_args other) { + if (other.isSetTinfo()) { + this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo); + } + if (other.isSetCredentials()) { + this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials); + } + if (other.isSetTableName()) { + this.tableName = other.tableName; + } + if (other.isSetSplits()) { + java.util.Map __this__splits = new java.util.HashMap(other.splits.size()); + for (java.util.Map.Entry other_element : other.splits.entrySet()) { + + org.apache.accumulo.core.dataImpl.thrift.TKeyExtent other_element_key = other_element.getKey(); + TTabletMergeability other_element_value = other_element.getValue(); + + org.apache.accumulo.core.dataImpl.thrift.TKeyExtent __this__splits_copy_key = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other_element_key); + + TTabletMergeability __this__splits_copy_value = new TTabletMergeability(other_element_value); + + __this__splits.put(__this__splits_copy_key, __this__splits_copy_value); + } + this.splits = __this__splits; + } + } + + @Override + public updateTabletMergeability_args deepCopy() { + return new updateTabletMergeability_args(this); + } + + @Override + public void clear() { + this.tinfo = null; + this.credentials = null; + this.tableName = null; + this.splits = null; + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() { + return this.tinfo; + } + + public updateTabletMergeability_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) { + this.tinfo = tinfo; + return this; + } + + public void unsetTinfo() { + this.tinfo = null; + } + + /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */ + public boolean isSetTinfo() { + return this.tinfo != null; + } + + public void setTinfoIsSet(boolean value) { + if (!value) { + this.tinfo = null; + } + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.securityImpl.thrift.TCredentials getCredentials() { + return this.credentials; + } + + public updateTabletMergeability_args setCredentials(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) { + this.credentials = credentials; + return this; + } + + public void unsetCredentials() { + this.credentials = null; + } + + /** Returns true if field credentials is set (has been assigned a value) and false otherwise */ + public boolean isSetCredentials() { + return this.credentials != null; + } + + public void setCredentialsIsSet(boolean value) { + if (!value) { + this.credentials = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getTableName() { + return this.tableName; + } + + public updateTabletMergeability_args setTableName(@org.apache.thrift.annotation.Nullable java.lang.String tableName) { + this.tableName = tableName; + return this; + } + + public void unsetTableName() { + this.tableName = null; + } + + /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ + public boolean isSetTableName() { + return this.tableName != null; + } + + public void setTableNameIsSet(boolean value) { + if (!value) { + this.tableName = null; + } + } + + public int getSplitsSize() { + return (this.splits == null) ? 0 : this.splits.size(); + } + + public void putToSplits(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent key, TTabletMergeability val) { + if (this.splits == null) { + this.splits = new java.util.HashMap(); + } + this.splits.put(key, val); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Map getSplits() { + return this.splits; + } + + public updateTabletMergeability_args setSplits(@org.apache.thrift.annotation.Nullable java.util.Map splits) { + this.splits = splits; + return this; + } + + public void unsetSplits() { + this.splits = null; + } + + /** Returns true if field splits is set (has been assigned a value) and false otherwise */ + public boolean isSetSplits() { + return this.splits != null; + } + + public void setSplitsIsSet(boolean value) { + if (!value) { + this.splits = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case TINFO: + if (value == null) { + unsetTinfo(); + } else { + setTinfo((org.apache.accumulo.core.clientImpl.thrift.TInfo)value); + } + break; + + case CREDENTIALS: + if (value == null) { + unsetCredentials(); + } else { + setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((java.lang.String)value); + } + break; + + case SPLITS: + if (value == null) { + unsetSplits(); + } else { + setSplits((java.util.Map)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case TINFO: + return getTinfo(); + + case CREDENTIALS: + return getCredentials(); + + case TABLE_NAME: + return getTableName(); + + case SPLITS: + return getSplits(); + + } + throw new java.lang.IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + @Override + public boolean isSet(_Fields field) { + if (field == null) { + throw new java.lang.IllegalArgumentException(); + } + + switch (field) { + case TINFO: + return isSetTinfo(); + case CREDENTIALS: + return isSetCredentials(); + case TABLE_NAME: + return isSetTableName(); + case SPLITS: + return isSetSplits(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof updateTabletMergeability_args) + return this.equals((updateTabletMergeability_args)that); + return false; + } + + public boolean equals(updateTabletMergeability_args that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_tinfo = true && this.isSetTinfo(); + boolean that_present_tinfo = true && that.isSetTinfo(); + if (this_present_tinfo || that_present_tinfo) { + if (!(this_present_tinfo && that_present_tinfo)) + return false; + if (!this.tinfo.equals(that.tinfo)) + return false; + } + + boolean this_present_credentials = true && this.isSetCredentials(); + boolean that_present_credentials = true && that.isSetCredentials(); + if (this_present_credentials || that_present_credentials) { + if (!(this_present_credentials && that_present_credentials)) + return false; + if (!this.credentials.equals(that.credentials)) + return false; + } + + boolean this_present_tableName = true && this.isSetTableName(); + boolean that_present_tableName = true && that.isSetTableName(); + if (this_present_tableName || that_present_tableName) { + if (!(this_present_tableName && that_present_tableName)) + return false; + if (!this.tableName.equals(that.tableName)) + return false; + } + + boolean this_present_splits = true && this.isSetSplits(); + boolean that_present_splits = true && that.isSetSplits(); + if (this_present_splits || that_present_splits) { + if (!(this_present_splits && that_present_splits)) + return false; + if (!this.splits.equals(that.splits)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287); + if (isSetTinfo()) + hashCode = hashCode * 8191 + tinfo.hashCode(); + + hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287); + if (isSetCredentials()) + hashCode = hashCode * 8191 + credentials.hashCode(); + + hashCode = hashCode * 8191 + ((isSetTableName()) ? 131071 : 524287); + if (isSetTableName()) + hashCode = hashCode * 8191 + tableName.hashCode(); + + hashCode = hashCode * 8191 + ((isSetSplits()) ? 131071 : 524287); + if (isSetSplits()) + hashCode = hashCode * 8191 + splits.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(updateTabletMergeability_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTinfo()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCredentials()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetTableName(), other.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetSplits(), other.isSetSplits()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSplits()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.splits, other.splits); + 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("updateTabletMergeability_args("); + boolean first = true; + + sb.append("tinfo:"); + if (this.tinfo == null) { + sb.append("null"); + } else { + sb.append(this.tinfo); + } + first = false; + if (!first) sb.append(", "); + sb.append("credentials:"); + if (this.credentials == null) { + sb.append("null"); + } else { + sb.append(this.credentials); + } + first = false; + if (!first) sb.append(", "); + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + sb.append(this.tableName); + } + first = false; + if (!first) sb.append(", "); + sb.append("splits:"); + if (this.splits == null) { + sb.append("null"); + } else { + sb.append(this.splits); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (tinfo != null) { + tinfo.validate(); + } + if (credentials != null) { + credentials.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class updateTabletMergeability_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public updateTabletMergeability_argsStandardScheme getScheme() { + return new updateTabletMergeability_argsStandardScheme(); + } + } + + private static class updateTabletMergeability_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, updateTabletMergeability_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TINFO + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(); + struct.tinfo.read(iprot); + struct.setTinfoIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CREDENTIALS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(); + struct.credentials.read(iprot); + struct.setCredentialsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // SPLITS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map112 = iprot.readMapBegin(); + struct.splits = new java.util.HashMap(2*_map112.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key113; + @org.apache.thrift.annotation.Nullable TTabletMergeability _val114; + for (int _i115 = 0; _i115 < _map112.size; ++_i115) + { + _key113 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _key113.read(iprot); + _val114 = new TTabletMergeability(); + _val114.read(iprot); + struct.splits.put(_key113, _val114); + } + iprot.readMapEnd(); + } + struct.setSplitsIsSet(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, updateTabletMergeability_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.tinfo != null) { + oprot.writeFieldBegin(TINFO_FIELD_DESC); + struct.tinfo.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.credentials != null) { + oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC); + struct.credentials.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.tableName != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); + oprot.writeFieldEnd(); + } + if (struct.splits != null) { + oprot.writeFieldBegin(SPLITS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.splits.size())); + for (java.util.Map.Entry _iter116 : struct.splits.entrySet()) + { + _iter116.getKey().write(oprot); + _iter116.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class updateTabletMergeability_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public updateTabletMergeability_argsTupleScheme getScheme() { + return new updateTabletMergeability_argsTupleScheme(); + } + } + + private static class updateTabletMergeability_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeability_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet optionals = new java.util.BitSet(); + if (struct.isSetTinfo()) { + optionals.set(0); + } + if (struct.isSetCredentials()) { + optionals.set(1); + } + if (struct.isSetTableName()) { + optionals.set(2); + } + if (struct.isSetSplits()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetTinfo()) { + struct.tinfo.write(oprot); + } + if (struct.isSetCredentials()) { + struct.credentials.write(oprot); + } + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); + } + if (struct.isSetSplits()) { + { + oprot.writeI32(struct.splits.size()); + for (java.util.Map.Entry _iter117 : struct.splits.entrySet()) + { + _iter117.getKey().write(oprot); + _iter117.getValue().write(oprot); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeability_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(); + struct.tinfo.read(iprot); + struct.setTinfoIsSet(true); + } + if (incoming.get(1)) { + struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(); + struct.credentials.read(iprot); + struct.setCredentialsIsSet(true); + } + if (incoming.get(2)) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } + if (incoming.get(3)) { + { + org.apache.thrift.protocol.TMap _map118 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT); + struct.splits = new java.util.HashMap(2*_map118.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key119; + @org.apache.thrift.annotation.Nullable TTabletMergeability _val120; + for (int _i121 = 0; _i121 < _map118.size; ++_i121) + { + _key119 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _key119.read(iprot); + _val120 = new TTabletMergeability(); + _val120.read(iprot); + struct.splits.put(_key119, _val120); + } + } + struct.setSplitsIsSet(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(); + } + } + + @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"}) + public static class updateTabletMergeability_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("updateTabletMergeability_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); + private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField TOE_FIELD_DESC = new org.apache.thrift.protocol.TField("toe", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new updateTabletMergeability_resultStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new updateTabletMergeability_resultTupleSchemeFactory(); + + public @org.apache.thrift.annotation.Nullable java.util.List 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.ThriftTableOperationException toe; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + SEC((short)1, "sec"), + TOE((short)2, "toe"); + + 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 0: // SUCCESS + return SUCCESS; + case 1: // SEC + return SEC; + case 2: // TOE + return TOE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + @org.apache.thrift.annotation.Nullable + public static _Fields findByName(java.lang.String name) { + return byName.get(name); + } + + private final short _thriftId; + private final java.lang.String _fieldName; + + _Fields(short thriftId, java.lang.String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + @Override + public short getThriftFieldId() { + return _thriftId; + } + + @Override + public java.lang.String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.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.TOE, new org.apache.thrift.meta_data.FieldMetaData("toe", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException.class))); + metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateTabletMergeability_result.class, metaDataMap); + } + + public updateTabletMergeability_result() { + } + + public updateTabletMergeability_result( + java.util.List success, + org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec, + org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException toe) + { + this(); + this.success = success; + this.sec = sec; + this.toe = toe; + } + + /** + * Performs a deep copy on other. + */ + public updateTabletMergeability_result(updateTabletMergeability_result other) { + if (other.isSetSuccess()) { + java.util.List __this__success = new java.util.ArrayList(other.success.size()); + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent other_element : other.success) { + __this__success.add(new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other_element)); + } + this.success = __this__success; + } + if (other.isSetSec()) { + this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec); + } + if (other.isSetToe()) { + this.toe = new org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException(other.toe); + } + } + + @Override + public updateTabletMergeability_result deepCopy() { + return new updateTabletMergeability_result(this); + } + + @Override + public void clear() { + this.success = null; + this.sec = null; + this.toe = null; + } + + public int getSuccessSize() { + return (this.success == null) ? 0 : this.success.size(); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Iterator getSuccessIterator() { + return (this.success == null) ? null : this.success.iterator(); + } + + public void addToSuccess(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent elem) { + if (this.success == null) { + this.success = new java.util.ArrayList(); + } + this.success.add(elem); + } + + @org.apache.thrift.annotation.Nullable + public java.util.List getSuccess() { + return this.success; + } + + public updateTabletMergeability_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() { + return this.sec; + } + + public updateTabletMergeability_result setSec(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) { + this.sec = sec; + return this; + } + + public void unsetSec() { + this.sec = null; + } + + /** Returns true if field sec is set (has been assigned a value) and false otherwise */ + public boolean isSetSec() { + return this.sec != null; + } + + public void setSecIsSet(boolean value) { + if (!value) { + this.sec = null; + } + } + + @org.apache.thrift.annotation.Nullable + public org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException getToe() { + return this.toe; + } + + public updateTabletMergeability_result setToe(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException toe) { + this.toe = toe; + return this; + } + + public void unsetToe() { + this.toe = null; + } + + /** Returns true if field toe is set (has been assigned a value) and false otherwise */ + public boolean isSetToe() { + return this.toe != null; + } + + public void setToeIsSet(boolean value) { + if (!value) { + this.toe = null; + } + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((java.util.List)value); + } + break; + + case SEC: + if (value == null) { + unsetSec(); + } else { + setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value); + } + break; + + case TOE: + if (value == null) { + unsetToe(); + } else { + setToe((org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case SEC: + return getSec(); + + case TOE: + return getToe(); + + } + throw new java.lang.IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + @Override + public boolean isSet(_Fields field) { + if (field == null) { + throw new java.lang.IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + case SEC: + return isSetSec(); + case TOE: + return isSetToe(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof updateTabletMergeability_result) + return this.equals((updateTabletMergeability_result)that); + return false; + } + + public boolean equals(updateTabletMergeability_result that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_sec = true && this.isSetSec(); + boolean that_present_sec = true && that.isSetSec(); + if (this_present_sec || that_present_sec) { + if (!(this_present_sec && that_present_sec)) + return false; + if (!this.sec.equals(that.sec)) + return false; + } + + boolean this_present_toe = true && this.isSetToe(); + boolean that_present_toe = true && that.isSetToe(); + if (this_present_toe || that_present_toe) { + if (!(this_present_toe && that_present_toe)) + return false; + if (!this.toe.equals(that.toe)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287); + if (isSetSuccess()) + hashCode = hashCode * 8191 + success.hashCode(); + + hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287); + if (isSetSec()) + hashCode = hashCode * 8191 + sec.hashCode(); + + hashCode = hashCode * 8191 + ((isSetToe()) ? 131071 : 524287); + if (isSetToe()) + hashCode = hashCode * 8191 + toe.hashCode(); + + return hashCode; + } + + @Override + public int compareTo(updateTabletMergeability_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetSec(), other.isSetSec()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSec()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, other.sec); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetToe(), other.isSetToe()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetToe()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.toe, other.toe); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + @org.apache.thrift.annotation.Nullable + @Override + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + scheme(iprot).read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + scheme(oprot).write(oprot, this); + } + + @Override + public java.lang.String toString() { + java.lang.StringBuilder sb = new java.lang.StringBuilder("updateTabletMergeability_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("sec:"); + if (this.sec == null) { + sb.append("null"); + } else { + sb.append(this.sec); + } + first = false; + if (!first) sb.append(", "); + sb.append("toe:"); + if (this.toe == null) { + sb.append("null"); + } else { + sb.append(this.toe); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class updateTabletMergeability_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public updateTabletMergeability_resultStandardScheme getScheme() { + return new updateTabletMergeability_resultStandardScheme(); + } + } + + private static class updateTabletMergeability_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, updateTabletMergeability_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list122 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list122.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem123; + for (int _i124 = 0; _i124 < _list122.size; ++_i124) + { + _elem123 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _elem123.read(iprot); + struct.success.add(_elem123); + } + iprot.readListEnd(); + } + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // SEC + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(); + struct.sec.read(iprot); + struct.setSecIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TOE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.toe = new org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException(); + struct.toe.read(iprot); + struct.setToeIsSet(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, updateTabletMergeability_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter125 : struct.success) + { + _iter125.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.sec != null) { + oprot.writeFieldBegin(SEC_FIELD_DESC); + struct.sec.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.toe != null) { + oprot.writeFieldBegin(TOE_FIELD_DESC); + struct.toe.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class updateTabletMergeability_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public updateTabletMergeability_resultTupleScheme getScheme() { + return new updateTabletMergeability_resultTupleScheme(); + } + } + + private static class updateTabletMergeability_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeability_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet optionals = new java.util.BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetSec()) { + optionals.set(1); + } + if (struct.isSetToe()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetSuccess()) { + { + oprot.writeI32(struct.success.size()); + for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter126 : struct.success) + { + _iter126.write(oprot); + } + } + } + if (struct.isSetSec()) { + struct.sec.write(oprot); + } + if (struct.isSetToe()) { + struct.toe.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, updateTabletMergeability_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; + java.util.BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list127 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list127.size); + @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem128; + for (int _i129 = 0; _i129 < _list127.size; ++_i129) + { + _elem128 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(); + _elem128.read(iprot); + struct.success.add(_elem128); + } + } + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(); + struct.sec.read(iprot); + struct.setSecIsSet(true); + } + if (incoming.get(2)) { + struct.toe = new org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException(); + struct.toe.read(iprot); + struct.setToeIsSet(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/TTabletMergeability.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TTabletMergeability.java new file mode 100644 index 00000000000..eb342b77e58 --- /dev/null +++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/manager/thrift/TTabletMergeability.java @@ -0,0 +1,497 @@ +/* + * 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 TTabletMergeability 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("TTabletMergeability"); + + private static final org.apache.thrift.protocol.TField NEVER_FIELD_DESC = new org.apache.thrift.protocol.TField("never", org.apache.thrift.protocol.TType.BOOL, (short)1); + private static final org.apache.thrift.protocol.TField DELAY_FIELD_DESC = new org.apache.thrift.protocol.TField("delay", org.apache.thrift.protocol.TType.I64, (short)2); + + private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new TTabletMergeabilityStandardSchemeFactory(); + private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new TTabletMergeabilityTupleSchemeFactory(); + + public boolean never; // required + public long delay; // 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 { + NEVER((short)1, "never"), + DELAY((short)2, "delay"); + + 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: // NEVER + return NEVER; + case 2: // DELAY + return DELAY; + 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 __NEVER_ISSET_ID = 0; + private static final int __DELAY_ISSET_ID = 1; + private byte __isset_bitfield = 0; + public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NEVER, new org.apache.thrift.meta_data.FieldMetaData("never", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.DELAY, new org.apache.thrift.meta_data.FieldMetaData("delay", 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(TTabletMergeability.class, metaDataMap); + } + + public TTabletMergeability() { + } + + public TTabletMergeability( + boolean never, + long delay) + { + this(); + this.never = never; + setNeverIsSet(true); + this.delay = delay; + setDelayIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TTabletMergeability(TTabletMergeability other) { + __isset_bitfield = other.__isset_bitfield; + this.never = other.never; + this.delay = other.delay; + } + + @Override + public TTabletMergeability deepCopy() { + return new TTabletMergeability(this); + } + + @Override + public void clear() { + setNeverIsSet(false); + this.never = false; + setDelayIsSet(false); + this.delay = 0; + } + + public boolean isNever() { + return this.never; + } + + public TTabletMergeability setNever(boolean never) { + this.never = never; + setNeverIsSet(true); + return this; + } + + public void unsetNever() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __NEVER_ISSET_ID); + } + + /** Returns true if field never is set (has been assigned a value) and false otherwise */ + public boolean isSetNever() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __NEVER_ISSET_ID); + } + + public void setNeverIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __NEVER_ISSET_ID, value); + } + + public long getDelay() { + return this.delay; + } + + public TTabletMergeability setDelay(long delay) { + this.delay = delay; + setDelayIsSet(true); + return this; + } + + public void unsetDelay() { + __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __DELAY_ISSET_ID); + } + + /** Returns true if field delay is set (has been assigned a value) and false otherwise */ + public boolean isSetDelay() { + return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __DELAY_ISSET_ID); + } + + public void setDelayIsSet(boolean value) { + __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __DELAY_ISSET_ID, value); + } + + @Override + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { + switch (field) { + case NEVER: + if (value == null) { + unsetNever(); + } else { + setNever((java.lang.Boolean)value); + } + break; + + case DELAY: + if (value == null) { + unsetDelay(); + } else { + setDelay((java.lang.Long)value); + } + break; + + } + } + + @org.apache.thrift.annotation.Nullable + @Override + public java.lang.Object getFieldValue(_Fields field) { + switch (field) { + case NEVER: + return isNever(); + + case DELAY: + return getDelay(); + + } + 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 NEVER: + return isSetNever(); + case DELAY: + return isSetDelay(); + } + throw new java.lang.IllegalStateException(); + } + + @Override + public boolean equals(java.lang.Object that) { + if (that instanceof TTabletMergeability) + return this.equals((TTabletMergeability)that); + return false; + } + + public boolean equals(TTabletMergeability that) { + if (that == null) + return false; + if (this == that) + return true; + + boolean this_present_never = true; + boolean that_present_never = true; + if (this_present_never || that_present_never) { + if (!(this_present_never && that_present_never)) + return false; + if (this.never != that.never) + return false; + } + + boolean this_present_delay = true; + boolean that_present_delay = true; + if (this_present_delay || that_present_delay) { + if (!(this_present_delay && that_present_delay)) + return false; + if (this.delay != that.delay) + return false; + } + + return true; + } + + @Override + public int hashCode() { + int hashCode = 1; + + hashCode = hashCode * 8191 + ((never) ? 131071 : 524287); + + hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(delay); + + return hashCode; + } + + @Override + public int compareTo(TTabletMergeability other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = java.lang.Boolean.compare(isSetNever(), other.isSetNever()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNever()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.never, other.never); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetDelay(), other.isSetDelay()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDelay()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delay, other.delay); + 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("TTabletMergeability("); + boolean first = true; + + sb.append("never:"); + sb.append(this.never); + first = false; + if (!first) sb.append(", "); + sb.append("delay:"); + sb.append(this.delay); + 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 TTabletMergeabilityStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TTabletMergeabilityStandardScheme getScheme() { + return new TTabletMergeabilityStandardScheme(); + } + } + + private static class TTabletMergeabilityStandardScheme extends org.apache.thrift.scheme.StandardScheme { + + @Override + public void read(org.apache.thrift.protocol.TProtocol iprot, TTabletMergeability 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: // NEVER + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.never = iprot.readBool(); + struct.setNeverIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // DELAY + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.delay = iprot.readI64(); + struct.setDelayIsSet(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, TTabletMergeability struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(NEVER_FIELD_DESC); + oprot.writeBool(struct.never); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(DELAY_FIELD_DESC); + oprot.writeI64(struct.delay); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TTabletMergeabilityTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory { + @Override + public TTabletMergeabilityTupleScheme getScheme() { + return new TTabletMergeabilityTupleScheme(); + } + } + + private static class TTabletMergeabilityTupleScheme extends org.apache.thrift.scheme.TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TTabletMergeability 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.isSetNever()) { + optionals.set(0); + } + if (struct.isSetDelay()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetNever()) { + oprot.writeBool(struct.never); + } + if (struct.isSetDelay()) { + oprot.writeI64(struct.delay); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TTabletMergeability 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.never = iprot.readBool(); + struct.setNeverIsSet(true); + } + if (incoming.get(1)) { + struct.delay = iprot.readI64(); + struct.setDelayIsSet(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/manager.thrift b/core/src/main/thrift/manager.thrift index 08d93dc1852..8947646c4e5 100644 --- a/core/src/main/thrift/manager.thrift +++ b/core/src/main/thrift/manager.thrift @@ -175,6 +175,11 @@ struct TFateId { 2:string txUUIDStr } +struct TTabletMergeability { + 1:bool never + 2:i64 delay +} + service FateService { // register a fate operation by reserving an opid @@ -445,4 +450,14 @@ service ManagerClientService { 1:client.ThriftSecurityException sec 2:client.ThriftTableOperationException toe ) + + list updateTabletMergeability( + 1:client.TInfo tinfo + 2:security.TCredentials credentials + 3:string tableName + 4:map splits + ) throws ( + 1:client.ThriftSecurityException sec + 2:client.ThriftTableOperationException toe + ) } 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 2bf22a28418..a0966bf1765 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 @@ -24,23 +24,30 @@ import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOGS; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW; +import static org.apache.accumulo.core.util.Validators.NOT_ROOT_TABLE_ID; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.ConcurrentModificationException; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import com.google.common.base.Preconditions; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.admin.DelegationTokenConfig; import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.DelegationTokenConfigSerializer; +import org.apache.accumulo.core.clientImpl.TabletMergeabilityUtil; import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode; import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties; @@ -63,11 +70,14 @@ import org.apache.accumulo.core.manager.thrift.ManagerGoalState; import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo; import org.apache.accumulo.core.manager.thrift.ManagerState; +import org.apache.accumulo.core.manager.thrift.TTabletMergeability; import org.apache.accumulo.core.manager.thrift.TabletLoadState; import org.apache.accumulo.core.manager.thrift.ThriftPropertyException; import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.core.metadata.TServerInstance; +import org.apache.accumulo.core.metadata.schema.Ample.ConditionalResult.Status; import org.apache.accumulo.core.metadata.schema.TabletDeletedException; +import org.apache.accumulo.core.metadata.schema.TabletMergeabilityMetadata; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metadata.schema.TabletsMetadata; import org.apache.accumulo.core.securityImpl.thrift.TCredentials; @@ -616,6 +626,46 @@ public void requestTabletHosting(TInfo tinfo, TCredentials credentials, String t manager.hostOndemand(Lists.transform(extents, KeyExtent::fromThrift)); } + @Override + public List updateTabletMergeability(TInfo tinfo, TCredentials credentials, String tableName, + Map splits) + throws ThriftTableOperationException, ThriftSecurityException { + + final TableId tableId = getTableId(manager.getContext(), tableName); + NamespaceId namespaceId = getNamespaceIdFromTableId(null, tableId); + + if (!manager.security.canSplitTablet(credentials, tableId, namespaceId)) { + throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED); + } + + try (var tabletsMutator = manager.getContext().getAmple().conditionallyMutateTablets()) { + for (Entry split : splits.entrySet()) { + var currentTime = manager.getSteadyTime(); + var extent = KeyExtent.fromThrift(split.getKey()); + var tabletMergeability = TabletMergeabilityUtil.fromThrift(split.getValue()); + + var updatedTm = TabletMergeabilityMetadata.toMetadata(tabletMergeability, currentTime); + tabletsMutator.mutateTablet(extent).requireAbsentOperation() + .putTabletMergeability(updatedTm) + .submit(tm -> tm.getTabletMergeability().equals(updatedTm), + () -> "update TabletMergeability for " + extent + " to " + updatedTm); + } + + var results = tabletsMutator.process(); + List unableToUpdate = results.entrySet().stream() + .filter(r -> r.getValue().getStatus() == Status.REJECTED) + .map(r -> r.getKey().toThrift()).collect(Collectors.toList()); + + if (log.isDebugEnabled()) { + for (var extent : unableToUpdate) { + log.debug("Unable to update TableMergeabilty: {}", KeyExtent.fromThrift(extent)); + } + } + + return unableToUpdate; + } + } + protected TableId getTableId(ClientContext context, String tableName) throws ThriftTableOperationException { return ClientServiceHandler.checkTableId(context, tableName, null); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java index 0e6e742d754..fd5baa3eca3 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java @@ -35,6 +35,7 @@ import org.apache.accumulo.core.client.admin.TabletMergeability; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.metadata.schema.TabletMergeabilityMetadata; @@ -137,6 +138,17 @@ public void addSplitWithMergeabilityTest() throws Exception { } }); } + + // Validate getTabletInformation() returns correct results as well + c.tableOperations().getTabletInformation(tableName, new Range()).forEach(ti -> { + // default tablet should be set to never + if (ti.getTabletId().getEndRow() == null) { + assertEquals(TabletMergeability.never(), ti.getTabletMergeability()); + } else { + // New splits should match the original setting in the map + assertEquals(splits.get(ti.getTabletId().getEndRow()), ti.getTabletMergeability()); + } + }); } }