diff --git a/.gitattributes b/.gitattributes index a701943f093..5e0ba737530 100644 --- a/.gitattributes +++ b/.gitattributes @@ -18,3 +18,4 @@ # core/src/main/thrift-gen-java/** linguist-generated=true +rpc/grpc/src/main/protobuf-gen-java/** linguist-generated=true diff --git a/core/pom.xml b/core/pom.xml index f0c1e0683b1..991c6b199f8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -61,6 +61,10 @@ + + com.google.protobuf + protobuf-java + commons-io commons-io @@ -69,6 +73,14 @@ commons-logging commons-logging + + io.grpc + grpc-api + + + io.grpc + grpc-netty-shaded + io.micrometer micrometer-core @@ -85,6 +97,10 @@ org.apache.accumulo accumulo-access + + org.apache.accumulo + accumulo-grpc + org.apache.accumulo accumulo-start @@ -258,12 +274,14 @@ .*[.]impl[.].* .*[.]thrift[.].* + .*[.]protobuf[.].* org[.]apache[.]accumulo[.]core[.]security[.]crypto[.].* - + org[.]apache[.]accumulo[.].*[.]thrift[.].* + org[.]apache[.]accumulo[.].*[.]protobuf[.].* @@ -395,6 +413,13 @@ + + + kr.motd.maven + os-maven-plugin + 1.7.1 + + diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java index 97ecfee7773..917999ffc39 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java @@ -102,6 +102,7 @@ import org.apache.accumulo.core.util.tables.TableZooHelper; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.util.threads.Threads; +import org.apache.accumulo.grpc.compaction.protobuf.PCredentials; import org.apache.hadoop.conf.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -140,7 +141,7 @@ public class ClientContext implements AccumuloClient { private final Supplier saslSupplier; private final Supplier sslSupplier; private final Supplier scanServerSelectorSupplier; - private TCredentials rpcCreds; + private RpcCredentialsHolder rpcCreds; private ThriftTransportPool thriftTransportPool; private volatile boolean closed = false; @@ -457,13 +458,22 @@ public synchronized ConditionalWriterConfig getConditionalWriterConfig() { * Serialize the credentials just before initiating the RPC call */ public synchronized TCredentials rpcCreds() { + return getRpcCreds().thrift(); + } + + public synchronized PCredentials gRpcCreds() { + return getRpcCreds().protobuf(); + } + + private synchronized RpcCredentialsHolder getRpcCreds() { ensureOpen(); if (getCredentials().getToken().isDestroyed()) { rpcCreds = null; } if (rpcCreds == null) { - rpcCreds = getCredentials().toThrift(getInstanceID()); + rpcCreds = new RpcCredentialsHolder(() -> getCredentials().toThrift(getInstanceID()), + () -> getCredentials().toProtobuf(getInstanceID())); } return rpcCreds; @@ -1093,4 +1103,23 @@ && getConfiguration().getBoolean(Property.GENERAL_MICROMETER_CACHE_METRICS_ENABL return caches; } + private static class RpcCredentialsHolder { + + private final com.google.common.base.Supplier tCreds; + private final com.google.common.base.Supplier pCreds; + + RpcCredentialsHolder(com.google.common.base.Supplier tCreds, + com.google.common.base.Supplier pCreds) { + this.tCreds = Suppliers.memoize(tCreds); + this.pCreds = Suppliers.memoize(pCreds); + } + + TCredentials thrift() { + return tCreds.get(); + } + + PCredentials protobuf() { + return pCreds.get(); + } + } } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java index eb7dad6b417..7a1c17232a1 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java @@ -30,6 +30,9 @@ import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode; import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.securityImpl.thrift.TCredentials; +import org.apache.accumulo.grpc.compaction.protobuf.PCredentials; + +import com.google.protobuf.ByteString; /** * A wrapper for internal use. This class carries the instance, principal, and authentication token @@ -99,6 +102,18 @@ public TCredentials toThrift(InstanceId instanceID) { return tCreds; } + public PCredentials toProtobuf(InstanceId instanceID) { + PCredentials pCreds = PCredentials.newBuilder().setPrincipal(getPrincipal()) + .setTokenClassName(getToken().getClass().getName()) + .setToken(ByteString.copyFrom(AuthenticationTokenSerializer.serialize(getToken()))) + .setInstanceId(instanceID.canonical()).build(); + if (getToken().isDestroyed()) { + throw new IllegalStateException("Token has been destroyed", + new AccumuloSecurityException(getPrincipal(), SecurityErrorCode.TOKEN_EXPIRED)); + } + return pCreds; + } + /** * Converts a given thrift object to our internal Credentials representation. * diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java index 0786f353615..05bb4b1332a 100644 --- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java +++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java @@ -362,6 +362,8 @@ public enum Property { "Properties in this category affect the behavior of the manager server.", "2.1.0"), MANAGER_CLIENTPORT("manager.port.client", "9999", PropertyType.PORT, "The port used for handling client connections on the manager.", "1.3.5"), + MANAGER_GRPC_CLIENTPORT("manager.port.grpc.client", "8999", PropertyType.PORT, + "The port used for handling gRPC client connections on the manager.", "1.3.5"), MANAGER_TABLET_BALANCER("manager.tablet.balancer", "org.apache.accumulo.core.spi.balancer.TableLoadBalancer", PropertyType.CLASSNAME, "The balancer class that accumulo will use to make tablet assignment and " @@ -1432,8 +1434,8 @@ public static boolean isValidTablePropertyKey(String key) { // take effect; these are always system-level properties, and not namespace or table properties public static final EnumSet fixedProperties = EnumSet.of( // port options - GC_PORT, MANAGER_CLIENTPORT, TSERV_CLIENTPORT, SSERV_CLIENTPORT, SSERV_PORTSEARCH, - COMPACTOR_PORTSEARCH, TSERV_PORTSEARCH, + GC_PORT, MANAGER_CLIENTPORT, MANAGER_GRPC_CLIENTPORT, TSERV_CLIENTPORT, SSERV_CLIENTPORT, + SSERV_PORTSEARCH, COMPACTOR_PORTSEARCH, TSERV_PORTSEARCH, // max message options RPC_MAX_MESSAGE_SIZE, diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java index a11b7a24714..43cb4f0fcbd 100644 --- a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java +++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java @@ -54,9 +54,12 @@ import org.apache.accumulo.core.util.ByteBufferUtil; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.TextUtil; +import org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent; import org.apache.hadoop.io.BinaryComparable; import org.apache.hadoop.io.Text; +import com.google.protobuf.ByteString; + /** * keeps track of information needed to identify a tablet */ @@ -128,6 +131,32 @@ public TKeyExtent toThrift() { prevEndRow() == null ? null : TextUtil.getByteBuffer(prevEndRow())); } + /** + * Convert to Protobuf form. + */ + public PKeyExtent toProtobuf() { + PKeyExtent.Builder builder = PKeyExtent.newBuilder().setTable(tableId().canonical()); + if (endRow() != null) { + builder.setEndRow(ByteString.copyFrom(endRow().getBytes())); + } + if (prevEndRow() != null) { + builder.setPrevEndRow(ByteString.copyFrom(prevEndRow().getBytes())); + } + return builder.build(); + } + + /** + * Create a KeyExtent from its Protobuf form. + * + * @param pke the KeyExtent in its Protobuf object form + */ + public static KeyExtent fromProtobuf(PKeyExtent pke) { + TableId tableId = TableId.of(pke.getTable()); + Text endRow = !pke.hasEndRow() ? null : new Text(pke.getEndRow().toByteArray()); + Text prevEndRow = !pke.hasPrevEndRow() ? null : new Text(pke.getPrevEndRow().toByteArray()); + return new KeyExtent(tableId, endRow, prevEndRow); + } + /** * Create a KeyExtent from a metadata previous end row entry. * diff --git a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java index cfad4ee1c96..eb93998878e 100644 --- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/SystemIteratorUtil.java @@ -30,32 +30,32 @@ import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.tabletserver.thrift.IteratorConfig; -import org.apache.accumulo.core.tabletserver.thrift.TIteratorSetting; +import org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig; +import org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting; /** * System utility class. Not for client use. */ public class SystemIteratorUtil { - public static TIteratorSetting toTIteratorSetting(IteratorSetting is) { - return new TIteratorSetting(is.getPriority(), is.getName(), is.getIteratorClass(), - is.getOptions()); + public static PIteratorSetting toPIteratorSetting(IteratorSetting is) { + return PIteratorSetting.newBuilder().setPriority(is.getPriority()).setName(is.getName()) + .setIteratorClass(is.getIteratorClass()).putAllProperties(is.getOptions()).build(); } - public static IteratorSetting toIteratorSetting(TIteratorSetting tis) { - return new IteratorSetting(tis.getPriority(), tis.getName(), tis.getIteratorClass(), - tis.getProperties()); + public static IteratorSetting toIteratorSetting(PIteratorSetting pis) { + return new IteratorSetting(pis.getPriority(), pis.getName(), pis.getIteratorClass(), + pis.getPropertiesMap()); } - public static IteratorConfig toIteratorConfig(List iterators) { - ArrayList tisList = new ArrayList<>(); + public static PIteratorConfig toIteratorConfig(List iterators) { + ArrayList pisList = new ArrayList<>(); for (IteratorSetting iteratorSetting : iterators) { - tisList.add(toTIteratorSetting(iteratorSetting)); + pisList.add(toPIteratorSetting(iteratorSetting)); } - return new IteratorConfig(tisList); + return PIteratorConfig.newBuilder().addAllIterators(pisList).build(); } public static SortedKeyValueIterator setupSystemScanIterators( diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/grpc/GrpcUtil.java b/core/src/main/java/org/apache/accumulo/core/rpc/grpc/GrpcUtil.java new file mode 100644 index 00000000000..527b5fa871e --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/rpc/grpc/GrpcUtil.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.core.rpc.grpc; + +import java.io.FileInputStream; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.TrustManagerFactory; + +import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.rpc.SslConnectionParams; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.RemovalListener; +import com.google.common.base.Preconditions; +import com.google.common.net.HostAndPort; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.grpc.ChannelCredentials; +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import io.grpc.netty.shaded.io.grpc.netty.NettySslContextChannelCredentials; +import io.grpc.netty.shaded.io.netty.handler.ssl.ClientAuth; +import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext; +import io.grpc.netty.shaded.io.netty.handler.ssl.SslContextBuilder; + +public class GrpcUtil { + + // TODO: We are just sharing a single ManagedChannel for all RPC requests to the same server + // We need to look into pooling to see if that is necessary with gRPC or if ManagedChannel can + // handle multiple connections using ManagedChannelBuilder or NettyChannelBuilder + private static final Cache grpcChannels = Caffeine.newBuilder() + // TODO: When to expire? + .expireAfterAccess(60_000, TimeUnit.MILLISECONDS) + .removalListener((RemovalListener) (key, value, cause) -> { + if (value != null) { + value.shutdownNow(); + } + }).build(); + + public static ManagedChannel getChannel(HostAndPort hostAndPort, ClientContext context) { + ChannelCredentials credentials = + Optional.ofNullable(SslConnectionParams.forClient(context.getConfiguration())) + .map(sslParams -> NettySslContextChannelCredentials + .create(GrpcUtil.buildSslContext(sslParams, false))) + .orElse(InsecureChannelCredentials.create()); + + // TODO: we are just using the config for now for the Port as we need to update ZK to + // store the new gRPC port for the service as currently this is just the Thrift port + return grpcChannels.get(hostAndPort, + hp -> Grpc + .newChannelBuilderForAddress(hp.getHost(), + context.getConfiguration().getPortStream(Property.MANAGER_GRPC_CLIENTPORT) + .findFirst().orElseThrow(), + credentials) + .idleTimeout(context.getClientTimeoutInMillis(), TimeUnit.MILLISECONDS).build()); + } + + // TODO: This builds the SSL context but still needs to be tested that everything works + // for client and server + @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", + justification = "code runs in same security context as user who providing the keystore file") + public static SslContext buildSslContext(SslConnectionParams sslParams, boolean server) { + Preconditions.checkArgument(!server || sslParams.isKeyStoreSet(), + "KeyStore must be set on the server."); + + try { + KeyManagerFactory kmf = null; + if (sslParams.isKeyStoreSet()) { + KeyStore keyStore = KeyStore.getInstance(sslParams.getKeyStoreType()); + try (FileInputStream fis = new FileInputStream(sslParams.getKeyStorePath())) { + keyStore.load(fis, sslParams.getKeyStorePass().toCharArray()); + } + kmf = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm()); + kmf.init(keyStore, sslParams.getKeyStorePass().toCharArray()); + } + + final var sslContextBuilder = + server ? SslContextBuilder.forServer(kmf) : SslContextBuilder.forClient(); + sslContextBuilder.protocols(sslParams.getServerProtocols()); + + if (sslParams.isTrustStoreSet()) { + KeyStore trustStore = KeyStore.getInstance(sslParams.getTrustStoreType()); + try (FileInputStream fis = new FileInputStream(sslParams.getTrustStorePath())) { + trustStore.load(fis, sslParams.getTrustStorePass().toCharArray()); + } + var tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()); + tmf.init(trustStore); + sslContextBuilder.trustManager(tmf); + } + + if (sslParams.isClientAuth()) { + sslContextBuilder.clientAuth(ClientAuth.REQUIRE); + } + + return sslContextBuilder.build(); + } catch (GeneralSecurityException | IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/grpc/ThriftProtobufUtil.java b/core/src/main/java/org/apache/accumulo/core/rpc/grpc/ThriftProtobufUtil.java new file mode 100644 index 00000000000..a359cccbfbf --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/rpc/grpc/ThriftProtobufUtil.java @@ -0,0 +1,331 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.accumulo.core.rpc.grpc; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.stream.Collectors.toList; + +import java.util.Optional; + +import org.apache.accumulo.core.clientImpl.thrift.TInfo; +import org.apache.accumulo.core.compaction.thrift.TCompactionState; +import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; +import org.apache.accumulo.core.compaction.thrift.TNextCompactionJob; +import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent; +import org.apache.accumulo.core.manager.thrift.TFateId; +import org.apache.accumulo.core.manager.thrift.TFateInstanceType; +import org.apache.accumulo.core.securityImpl.thrift.TCredentials; +import org.apache.accumulo.core.tabletserver.thrift.InputFile; +import org.apache.accumulo.core.tabletserver.thrift.IteratorConfig; +import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind; +import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.core.tabletserver.thrift.TIteratorSetting; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate; +import org.apache.accumulo.grpc.compaction.protobuf.PCredentials; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.PFateId; +import org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType; +import org.apache.accumulo.grpc.compaction.protobuf.PInputFile; +import org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig; +import org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting; +import org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent; +import org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.protobuf.ByteString; + +/** + * This is a utility class for converting between the equivalent Thrift and Protobuf versions of the + * objects needed using the gRPC service. + * + * TODO: This code may eventually be able to be removed + * + * This class is designed mostly just for this prototype/proof of concept and would probably not be + * needed if we plan to keep using protocol buffers and gRPC going forward. The Thrift API is used + * by the rest of the compaction code in the Compactor and Compaction Coordinator, so doing a + * conversion made it easy to isolate the gRPC changes to the one getCompactionJob() RPC call + * without having to impact the rest of the code for this prototype. + * + * Ideally, if we decide to go with gRPC and keep protocol buffers, then we would want to remove the + * Thrift conversion entirely for performance reasons and just go between the native objects and + * protocol buffer versions. + * + * Thrift and proto3 are pretty similar with their defintions and usage but there are a couple key + * differences to note: + * + * 1) Thrift uses constructors/setters for creation objects and protocol buffers uses the builder + * pattern + * + * 2) There are no nulls allowed in proto3. All values have a default value (empty string, etc) + * depending on the type. There are also no truly optional fields (everything is initialized), + * however, the latest versions of proto3 adds back support for the "optional" keyword. This allows + * us to replicate null because adding the keyword generates hasX() methods. + * + * Instead of checking if a field is null, we can check if the field was set by calling hasX(). For + * example, below with TCredentials, we would check if the instanceId was set by checking for null + * but for the proto version of PCredentials we can call credentials.hasInstanceId() and check that + * boolean. For more info see https://protobuf.dev/programming-guides/proto3/ + * + */ +public class ThriftProtobufUtil { + + private static final Logger LOG = LoggerFactory.getLogger(ThriftProtobufUtil.class); + + public static ProtoTInfo convert(TInfo tinfo) { + var builder = ProtoTInfo.newBuilder(); + Optional.ofNullable(tinfo.getHeaders()).ifPresent(builder::putAllHeaders); + return builder.build(); + } + + public static TInfo convert(ProtoTInfo ptinfo) { + return new TInfo(ptinfo.getHeadersMap()); + } + + public static PCredentials convert(TCredentials credentials) { + var builder = PCredentials.newBuilder(); + Optional.ofNullable(credentials.getPrincipal()).ifPresent(builder::setPrincipal); + Optional.ofNullable(credentials.getTokenClassName()).ifPresent(builder::setTokenClassName); + Optional.ofNullable(credentials.getToken()) + .ifPresent(token -> builder.setToken(ByteString.copyFrom(token))); + Optional.ofNullable(credentials.getInstanceId()).ifPresent(builder::setInstanceId); + return builder.build(); + } + + public static TCredentials convert(PCredentials credentials) { + var principal = credentials.hasPrincipal() ? credentials.getPrincipal() : null; + var tokenClassName = credentials.hasTokenClassName() ? credentials.getTokenClassName() : null; + var token = credentials.hasToken() ? credentials.getToken().asReadOnlyByteBuffer() : null; + var instanceId = credentials.hasInstanceId() ? credentials.getInstanceId() : null; + return new TCredentials(principal, tokenClassName, token, instanceId); + } + + public static TKeyExtent convert(PKeyExtent extent) { + TKeyExtent tExtent = new TKeyExtent(); + tExtent.setTable(extent.getTable().getBytes(UTF_8)); + if (extent.hasPrevEndRow()) { + tExtent.setPrevEndRow(extent.getPrevEndRow().asReadOnlyByteBuffer()); + } + if (extent.hasEndRow()) { + tExtent.setEndRow(extent.getEndRow().asReadOnlyByteBuffer()); + } + return tExtent; + } + + public static PKeyExtent convert(TKeyExtent extent) { + var builder = PKeyExtent.newBuilder().setTable(new String(extent.getTable(), UTF_8)); + Optional.ofNullable(extent.getPrevEndRow()) + .ifPresent(prevEndRow -> builder.setPrevEndRow(ByteString.copyFrom(prevEndRow))); + Optional.ofNullable(extent.getEndRow()) + .ifPresent(endRow -> builder.setEndRow(ByteString.copyFrom(endRow))); + return builder.build(); + } + + public static InputFile convert(PInputFile inputFile) { + InputFile tInputFile = new InputFile(); + tInputFile.setMetadataFileEntry(inputFile.getMetadataFileEntry()); + tInputFile.setSize(inputFile.getSize()); + tInputFile.setEntries(inputFile.getEntries()); + tInputFile.setTimestamp(inputFile.getTimestamp()); + return tInputFile; + } + + public static PInputFile convert(InputFile inputFile) { + return PInputFile.newBuilder().setMetadataFileEntry(inputFile.getMetadataFileEntry()) + .setSize(inputFile.getSize()).setEntries(inputFile.getEntries()) + .setTimestamp(inputFile.getTimestamp()).build(); + } + + public static TIteratorSetting convert(PIteratorSetting iteratorSetting) { + TIteratorSetting tIterSetting = new TIteratorSetting(); + tIterSetting.setPriority(iteratorSetting.getPriority()); + tIterSetting.setName(iteratorSetting.getName()); + tIterSetting.setIteratorClass(iteratorSetting.getIteratorClass()); + tIterSetting.setProperties(iteratorSetting.getPropertiesMap()); + return tIterSetting; + } + + public static PIteratorSetting convert(TIteratorSetting iteratorSetting) { + return PIteratorSetting.newBuilder().setPriority(iteratorSetting.getPriority()) + .setName(iteratorSetting.getName()).setIteratorClass(iteratorSetting.getIteratorClass()) + .putAllProperties(iteratorSetting.getProperties()).build(); + } + + public static IteratorConfig convert(PIteratorConfig setting) { + IteratorConfig tIterConfig = new IteratorConfig(); + tIterConfig.setIterators( + setting.getIteratorsList().stream().map(ThriftProtobufUtil::convert).collect(toList())); + return tIterConfig; + } + + public static PIteratorConfig convert(IteratorConfig setting) { + return PIteratorConfig.newBuilder() + .addAllIterators( + setting.getIterators().stream().map(ThriftProtobufUtil::convert).collect(toList())) + .build(); + } + + public static TCompactionKind convert(PCompactionKind kind) { + switch (kind) { + case SYSTEM: + return TCompactionKind.SYSTEM; + case USER: + return TCompactionKind.USER; + default: + throw new IllegalArgumentException("Unexpected PCompactionKind: " + kind); + } + } + + public static PCompactionKind convert(TCompactionKind kind) { + if (kind == null) { + return PCompactionKind.UNKNOWN; + } + switch (kind) { + case SYSTEM: + return PCompactionKind.SYSTEM; + case USER: + return PCompactionKind.USER; + default: + throw new IllegalArgumentException("Unexpected TCompactionKind: " + kind); + } + } + + public static TFateId convert(PFateId fateId) { + TFateInstanceType type; + + switch (fateId.getType()) { + case META: + type = TFateInstanceType.META; + break; + case USER: + type = TFateInstanceType.USER; + break; + default: + throw new IllegalArgumentException("Unexpected TFateInstanceType: " + fateId.getType()); + } + + return new TFateId(type, fateId.getTxUUIDStr()); + } + + public static PFateId convert(TFateId fateId) { + PFateInstanceType type; + + switch (fateId.getType()) { + case META: + type = PFateInstanceType.META; + break; + case USER: + type = PFateInstanceType.USER; + break; + default: + throw new IllegalArgumentException("Unexpected TFateInstanceType: " + fateId.getType()); + } + + return PFateId.newBuilder().setTypeValue(type.getNumber()).setTxUUIDStr(fateId.getTxUUIDStr()) + .build(); + } + + public static PExternalCompactionJob convert(TExternalCompactionJob extJob) { + var builder = PExternalCompactionJob.newBuilder(); + + Optional.ofNullable(extJob.getExternalCompactionId()) + .ifPresent(builder::setExternalCompactionId); + Optional.ofNullable(extJob.getExtent()).map(ThriftProtobufUtil::convert) + .ifPresent(builder::setExtent); + Optional.ofNullable(extJob.getFiles()) + .map(files -> files.stream().map(ThriftProtobufUtil::convert).collect(toList())) + .ifPresent(builder::addAllFiles); + Optional.ofNullable(extJob.getIteratorSettings()).map(ThriftProtobufUtil::convert) + .ifPresent(builder::setIteratorSettings); + Optional.ofNullable(extJob.getOutputFile()).ifPresent(builder::setOutputFile); + builder.setPropagateDeletes(extJob.isPropagateDeletes()); + Optional.ofNullable(extJob.getKind()).map(ThriftProtobufUtil::convert) + .ifPresent(builder::setKind); + Optional.ofNullable(extJob.getFateId()).map(ThriftProtobufUtil::convert) + .ifPresent(builder::setFateId); + Optional.ofNullable(extJob.getOverrides()).ifPresent(builder::putAllOverrides); + + PExternalCompactionJob job = builder.build(); + + if (extJob.getExternalCompactionId() != null) { + LOG.debug("TExternalCompactionJob: {}", extJob); + LOG.debug("PExternalCompactionJob: {}", job); + } + + return job; + } + + public static TExternalCompactionJob convert(PExternalCompactionJob extJob) { + TExternalCompactionJob tExtJob = new TExternalCompactionJob(); + + var optExtJob = Optional.of(extJob); + optExtJob.filter(PExternalCompactionJob::hasExternalCompactionId) + .map(PExternalCompactionJob::getExternalCompactionId) + .ifPresent(tExtJob::setExternalCompactionId); + optExtJob.filter(PExternalCompactionJob::hasExtent).map(job -> convert(job.getExtent())) + .ifPresent(tExtJob::setExtent); + if (!extJob.getFilesList().isEmpty()) { + tExtJob.setFiles( + extJob.getFilesList().stream().map(ThriftProtobufUtil::convert).collect(toList())); + } + optExtJob.filter(PExternalCompactionJob::hasIteratorSettings) + .map(job -> convert(job.getIteratorSettings())).ifPresent(tExtJob::setIteratorSettings); + optExtJob.filter(PExternalCompactionJob::hasOutputFile) + .map(PExternalCompactionJob::getOutputFile).ifPresent(tExtJob::setOutputFile); + tExtJob.setPropagateDeletes(extJob.getPropagateDeletes()); + optExtJob.filter(PExternalCompactionJob::hasKind).map(job -> convert(job.getKind())) + .ifPresent(tExtJob::setKind); + optExtJob.filter(PExternalCompactionJob::hasFateId).map(job -> convert(job.getFateId())) + .ifPresent(tExtJob::setFateId); + tExtJob.setOverrides(extJob.getOverridesMap()); + + if (!extJob.getExternalCompactionId().isEmpty()) { + LOG.trace("PExternalCompactionJob: {}", extJob); + LOG.trace("TExternalCompactionJob: {}", tExtJob); + } + + return tExtJob; + } + + public static TNextCompactionJob convert(PNextCompactionJob nextJob) { + var job = new TNextCompactionJob(); + job.setJob(convert(nextJob.getJob())); + job.setCompactorCount(nextJob.getCompactorCount()); + return job; + } + + public static PNextCompactionJob convert(TNextCompactionJob nextJob) { + return PNextCompactionJob.newBuilder().setCompactorCount(nextJob.getCompactorCount()) + .setJob(convert(nextJob.getJob())).build(); + } + + public static TCompactionStatusUpdate convert(PCompactionStatusUpdate pstatus) { + var update = new TCompactionStatusUpdate(); + update.setState(TCompactionState.valueOf(pstatus.getState().name())); + update.setMessage(pstatus.getMessage()); + update.setEntriesToBeCompacted(pstatus.getEntriesToBeCompacted()); + update.setEntriesRead(pstatus.getEntriesRead()); + update.setEntriesWritten(pstatus.getEntriesWritten()); + update.setCompactionAgeNanos(pstatus.getCompactionAgeNanos()); + return update; + } + +} diff --git a/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java b/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java index 78d658a38e1..6b9a5a4ac76 100644 --- a/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java @@ -28,6 +28,8 @@ import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo; +import org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -140,6 +142,13 @@ public static TInfo traceInfo() { return tinfo; } + public static ProtoTInfo protoTraceInfo() { + ProtoTInfo.Builder ptinfo = ProtoTInfo.newBuilder(); + W3CTraceContextPropagator.getInstance().inject(Context.current(), ptinfo, + ProtoTInfo.Builder::putHeaders); + return ptinfo.build(); + } + /** * Returns a newly created Context from the TInfo object sent by a remote process. The Context can * then be used in this process to continue the tracing. The Context is used like: diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java index 057818b0ac9..e945cf62f30 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java @@ -19,6 +19,7 @@ package org.apache.accumulo.core.util.compaction; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.accumulo.core.rpc.grpc.ThriftProtobufUtil.convert; import static org.apache.accumulo.core.util.threads.ThreadPoolNames.COMPACTOR_RUNNING_COMPACTIONS_POOL; import static org.apache.accumulo.core.util.threads.ThreadPoolNames.COMPACTOR_RUNNING_COMPACTION_IDS_POOL; @@ -50,6 +51,7 @@ import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.Timer; import org.apache.accumulo.core.util.threads.ThreadPools; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob; import org.apache.thrift.TException; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; @@ -63,10 +65,10 @@ public class ExternalCompactionUtil { private static class RunningCompactionFuture { private final String group; private final HostAndPort compactor; - private final Future future; + private final Future future; public RunningCompactionFuture(String group, HostAndPort compactor, - Future future) { + Future future) { this.group = group; this.compactor = compactor; this.future = future; @@ -80,7 +82,7 @@ public HostAndPort getCompactor() { return compactor; } - public Future getFuture() { + public Future getFuture() { return future; } } @@ -177,7 +179,7 @@ public static List getActiveCompaction(HostAndPort compactor, * @param context context * @return external compaction job or null if none running */ - public static TExternalCompactionJob getRunningCompaction(HostAndPort compactorAddr, + public static PExternalCompactionJob getRunningCompaction(HostAndPort compactorAddr, ClientContext context) { CompactorService.Client client = null; @@ -187,7 +189,7 @@ public static TExternalCompactionJob getRunningCompaction(HostAndPort compactorA client.getRunningCompaction(TraceUtil.traceInfo(), context.rpcCreds()); if (job.getExternalCompactionId() != null) { LOG.debug("Compactor {} is running {}", compactorAddr, job.getExternalCompactionId()); - return job; + return convert(job); } } catch (TException e) { LOG.debug("Failed to contact compactor {}", compactorAddr, e); @@ -238,8 +240,8 @@ public static List getCompactionsRunningOnCompactors(ClientCo final List results = new ArrayList<>(); rcFutures.forEach(rcf -> { try { - TExternalCompactionJob job = rcf.getFuture().get(); - if (null != job && null != job.getExternalCompactionId()) { + PExternalCompactionJob job = rcf.getFuture().get(); + if (null != job && job.hasExternalCompactionId()) { var compactorAddress = getHostPortString(rcf.getCompactor()); results.add(new RunningCompaction(job, compactorAddress, rcf.getGroup())); } diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompaction.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompaction.java index f2aab6b04a1..bc6e01a5a5b 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompaction.java +++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompaction.java @@ -21,40 +21,40 @@ import java.util.Map; import java.util.TreeMap; -import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; -import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; -import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob; public class RunningCompaction { - private final TExternalCompactionJob job; + private final PExternalCompactionJob job; private final String compactorAddress; private final String groupName; - private final Map updates = new TreeMap<>(); + private final Map updates = new TreeMap<>(); - public RunningCompaction(TExternalCompactionJob job, String compactorAddress, String groupName) { + public RunningCompaction(PExternalCompactionJob job, String compactorAddress, String groupName) { this.job = job; this.compactorAddress = compactorAddress; this.groupName = groupName; } - public RunningCompaction(TExternalCompaction tEC) { + public RunningCompaction(PExternalCompaction tEC) { this(tEC.getJob(), tEC.getCompactor(), tEC.getGroupName()); } - public Map getUpdates() { + public Map getUpdates() { synchronized (updates) { return new TreeMap<>(updates); } } - public void addUpdate(Long timestamp, TCompactionStatusUpdate update) { + public void addUpdate(Long timestamp, PCompactionStatusUpdate update) { synchronized (updates) { this.updates.put(timestamp, update); } } - public TExternalCompactionJob getJob() { + public PExternalCompactionJob getJob() { return job; } diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompactionInfo.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompactionInfo.java index 6b7bed87c03..5ffbf1f5084 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompactionInfo.java +++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/RunningCompactionInfo.java @@ -24,9 +24,9 @@ import java.util.TreeMap; -import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; -import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,26 +49,26 @@ public class RunningCompactionInfo { * Info parsed about the external running compaction. Calculate the progress, which is defined as * the percentage of bytesRead / bytesToBeCompacted of the last update. */ - public RunningCompactionInfo(TExternalCompaction ec) { - requireNonNull(ec, "Thrift external compaction is null."); - var updates = requireNonNull(ec.getUpdates(), "Missing Thrift external compaction updates"); + public RunningCompactionInfo(PExternalCompaction ec) { + requireNonNull(ec, "Protobuf external compaction is null."); + var updates = requireNonNull(ec.getUpdatesMap(), "Missing Thrift external compaction updates"); var job = requireNonNull(ec.getJob(), "Thrift external compaction job is null"); server = ec.getCompactor(); queueName = ec.getGroupName(); ecid = job.getExternalCompactionId(); kind = job.getKind().name(); - tableId = KeyExtent.fromThrift(job.getExtent()).tableId().canonical(); - numFiles = job.getFiles().size(); + tableId = KeyExtent.fromProtobuf(job.getExtent()).tableId().canonical(); + numFiles = job.getFilesList().size(); // parse the updates map long nowMillis = System.currentTimeMillis(); float percent = 0f; long updateMillis; - TCompactionStatusUpdate last; + PCompactionStatusUpdate last; // sort updates by key, which is a timestamp - TreeMap sorted = new TreeMap<>(updates); + TreeMap sorted = new TreeMap<>(updates); var lastEntry = sorted.lastEntry(); // last entry is all we care about so bail if null @@ -103,7 +103,7 @@ public RunningCompactionInfo(TExternalCompaction ec) { if (updates.isEmpty()) { status = "na"; } else { - status = last.state.name(); + status = last.getState().name(); } log.debug("Parsed running compaction {} for {} with progress = {}%", status, ecid, progress); if (sinceLastUpdateSeconds > 30) { diff --git a/hadoop-mapreduce/pom.xml b/hadoop-mapreduce/pom.xml index 90e8dccdc56..f6321a67b7c 100644 --- a/hadoop-mapreduce/pom.xml +++ b/hadoop-mapreduce/pom.xml @@ -101,7 +101,7 @@ .*[.]impl[.].* - org[.]apache[.]accumulo[.]core[.](?:client|data|security)[.](?!.*(impl|thrift|crypto).*).* + org[.]apache[.]accumulo[.]core[.](?:client|data|security)[.](?!.*(impl|thrift|protobuf|crypto).*).* org[.]apache[.]accumulo[.]core[.]util[.]Pair biz.aQute.bnd:biz.aQute.bnd.annotation:jar:* org.junit.jupiter:junit-jupiter-engine:jar:* + + io.grpc:grpc-netty-shaded:jar:* @@ -1103,7 +1125,7 @@ warning true - **/thrift/*.java + **/thrift/*.java,**/protobuf/*.java @@ -1531,6 +1553,36 @@ + + add-protobuf-java-source + + + src/main/protobuf-gen-java + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-protobuf-java-source + + add-source + + generate-sources + + + src/main/protobuf-gen-java + + + + + + + + add-spotbugs-excludes @@ -1605,7 +1657,7 @@ -XDcompilePolicy=simple -Xplugin:ErrorProne \ - -XepExcludedPaths:.*/(thrift|generated-sources|src/test)/.* \ + -XepExcludedPaths:.*/(thrift|protobuf|generated-sources|src/test)/.* \ -XepDisableWarningsInGeneratedCode \ -XepDisableAllWarnings \ diff --git a/rpc/grpc/.gitignore b/rpc/grpc/.gitignore new file mode 100644 index 00000000000..55d7f58f9d5 --- /dev/null +++ b/rpc/grpc/.gitignore @@ -0,0 +1,36 @@ +# +# 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. +# + +# Maven ignores +/target/ + +# IDE ignores +/.settings/ +/.project +/.classpath +/.pydevproject +/.idea +/*.iml +/*.ipr +/*.iws +/nbproject/ +/nbactions.xml +/nb-configuration.xml +/.vscode/ +/.factorypath diff --git a/rpc/grpc/pom.xml b/rpc/grpc/pom.xml new file mode 100644 index 00000000000..2ac8dd46163 --- /dev/null +++ b/rpc/grpc/pom.xml @@ -0,0 +1,130 @@ + + + + 4.0.0 + + org.apache.accumulo + accumulo-project + 4.0.0-SNAPSHOT + ../../pom.xml + + accumulo-grpc + Apache Accumulo gRPC + Apache Accumulo gRPC module. + + + com.google.guava + guava + + + com.google.protobuf + protobuf-java + + + io.grpc + grpc-api + + + io.grpc + grpc-protobuf + + + io.grpc + grpc-stub + + + io.grpc + grpc-netty-shaded + runtime + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + 1 + false + + + + org.apache.maven.plugins + maven-failsafe-plugin + + true + + + + + + + kr.motd.maven + os-maven-plugin + 1.7.1 + + + + + + protobuf + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.6.1 + + com.google.protobuf:protoc:3.25.3:exe:${os.detected.classifier} + grpc-java + io.grpc:protoc-gen-grpc-java:1.64.0:exe:${os.detected.classifier} + src/main/protobuf-gen-java/ + + @generated=omit + + + + generate-protobuf + + compile + + generate-sources + + + generate-grpc + + compile-custom + + generate-sources + + false + + + + + + + + + diff --git a/rpc/grpc/src/main/proto/client.proto b/rpc/grpc/src/main/proto/client.proto new file mode 100644 index 00000000000..13cbb05a8cd --- /dev/null +++ b/rpc/grpc/src/main/proto/client.proto @@ -0,0 +1,30 @@ +// +// 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. +// + +syntax = "proto3"; + +option java_multiple_files = true; +option java_package = "org.apache.accumulo.grpc.compaction.protobuf"; +option java_outer_classname = "ClientProto"; + +package client; + +message ProtoTInfo { + map headers = 1; +} diff --git a/rpc/grpc/src/main/proto/compaction-coordinator.proto b/rpc/grpc/src/main/proto/compaction-coordinator.proto new file mode 100644 index 00000000000..8d3f4e46b8a --- /dev/null +++ b/rpc/grpc/src/main/proto/compaction-coordinator.proto @@ -0,0 +1,159 @@ +// +// 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. +// + +syntax = "proto3"; + +option java_multiple_files = true; +option java_package = "org.apache.accumulo.grpc.compaction.protobuf"; +option java_outer_classname = "CompactionCoordinatorServiceProto"; + +import "google/protobuf/empty.proto"; +import "security.proto"; +import "client.proto"; +import "data.proto"; +import "tabletserver.proto"; +import "manager.proto"; + +package compaction_coordinator; + +// Interface exported by the server. +service CompactionCoordinatorService { + /* + * Called by Compactor to get the next compaction job + */ + rpc GetCompactionJob(CompactionJobRequest) returns (PNextCompactionJob); + + /* + * Called by Compactor on successful completion of compaction job + */ + rpc CompactionCompleted(CompactionCompletedRequest) returns (google.protobuf.Empty); + + /* + * Called by Compactor to update the Coordinator with the state of the compaction + */ + rpc UpdateCompactionStatus(UpdateCompactionStatusRequest) returns (google.protobuf.Empty); + + /* + * Called by Compactor on unsuccessful completion of compaction job + */ + rpc CompactionFailed(CompactionFailedRequest) returns (google.protobuf.Empty); + + /* + * Called by the Monitor to get progress information + */ + rpc GetRunningCompactions(GetRunningCompactionsRequest) returns (PExternalCompactionList); + + /* + * Called by the Monitor to get progress information + */ + rpc GetCompletedCompactions(GetCompletedCompactionsRequest) returns (PExternalCompactionList); + + /* + * Called by Compactor on unsuccessful completion of compaction job + */ + rpc Cancel(CancelRequest) returns (google.protobuf.Empty); +} + +message CompactionJobRequest { + client.ProtoTInfo ptinfo = 1; + security.PCredentials credentials = 2; + string groupName = 3; + string compactor = 4; + string externalCompactionId = 5; +} + +message CompactionCompletedRequest { + client.ProtoTInfo ptinfo = 1; + security.PCredentials credentials = 2; + string externalCompactionId = 3; + data.PKeyExtent extent = 4; + tabletserver.PCompactionStats stats = 5; +} + +message UpdateCompactionStatusRequest { + client.ProtoTInfo ptinfo = 1; + security.PCredentials credentials = 2; + string externalCompactionId = 3; + PCompactionStatusUpdate status = 4; + int64 timestamp = 5; +} + +message CompactionFailedRequest { + client.ProtoTInfo ptinfo = 1; + security.PCredentials credentials = 2; + string externalCompactionId = 3; + data.PKeyExtent extent = 4; +} + +message GetRunningCompactionsRequest { + client.ProtoTInfo ptinfo = 1; + security.PCredentials credentials = 2; +} + +message GetCompletedCompactionsRequest { + client.ProtoTInfo ptinfo = 1; + security.PCredentials credentials = 2; +} + +message CancelRequest { + client.ProtoTInfo ptinfo = 1; + security.PCredentials credentials = 2; + string externalCompactionId = 3; +} + +message PExternalCompaction { + string groupName = 1; + string compactor = 2; + map updates = 3; + tabletserver.PExternalCompactionJob job = 4; + +} + +message PExternalCompactionList { + map compactions = 1; +} + +message PNextCompactionJob { + tabletserver.PExternalCompactionJob job = 1; + int32 compactorCount = 2; +} + +enum PCompactionState { + // Coordinator should set state to ASSIGNED when getCompactionJob is called by Compactor + ASSIGNED = 0; + // Compactor should set state to STARTED when compaction has successfully begun + STARTED = 1; + // Compactor can call repeatedly with an updated message to reflect percentage complete + IN_PROGRESS = 2; + // Compactor should set state to SUCCEEDED when compaction job has successfully finished + SUCCEEDED = 3; + // Compactor should set state to FAILED when compaction job fails, message should be mandatory + FAILED = 4; + // Compactor should set state to CANCELLED to acknowledge that it has stopped compacting + CANCELLED = 5; +} + +message PCompactionStatusUpdate { + PCompactionState state = 1; + string message = 2; + int64 entriesToBeCompacted = 3; + int64 entriesRead = 4; + int64 entriesWritten = 5; + int64 compactionAgeNanos = 6; +} diff --git a/rpc/grpc/src/main/proto/data.proto b/rpc/grpc/src/main/proto/data.proto new file mode 100644 index 00000000000..9ca7efb6625 --- /dev/null +++ b/rpc/grpc/src/main/proto/data.proto @@ -0,0 +1,36 @@ +// +// 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. +// + +syntax = "proto3"; + +option java_multiple_files = true; +option java_package = "org.apache.accumulo.grpc.compaction.protobuf"; +option java_outer_classname = "DataProto"; + +package data; + + +message PKeyExtent { + // Use a string to simplify things (thrift uses a byte array) + string table = 1; + // optional to generate hasEndRow() and hasPrevEndRow() because + // of no nulls in protobuf + optional bytes endRow = 2; + optional bytes prevEndRow = 3; +} diff --git a/rpc/grpc/src/main/proto/manager.proto b/rpc/grpc/src/main/proto/manager.proto new file mode 100644 index 00000000000..1140b9461b6 --- /dev/null +++ b/rpc/grpc/src/main/proto/manager.proto @@ -0,0 +1,38 @@ +// +// 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. +// + +syntax = "proto3"; + +option java_multiple_files = true; +option java_package = "org.apache.accumulo.grpc.compaction.protobuf"; +option java_outer_classname = "ManagerProto"; + +package manager; + + +enum PFateInstanceType { + UNKNOWN = 0; + META = 1; + USER = 2; +} + +message PFateId { + PFateInstanceType type = 1; + string txUUIDStr = 2; +} diff --git a/rpc/grpc/src/main/proto/security.proto b/rpc/grpc/src/main/proto/security.proto new file mode 100644 index 00000000000..9a3c645aa74 --- /dev/null +++ b/rpc/grpc/src/main/proto/security.proto @@ -0,0 +1,37 @@ +// +// 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. +// + +syntax = "proto3"; + +option java_multiple_files = true; +option java_package = "org.apache.accumulo.grpc.compaction.protobuf"; +option java_outer_classname = "SecurityProto"; + +package security; + +// There are no nulls with Protobuf3, so the default will be an empty string +// Our TCredentials version with thrift currently is using/checking for null so +// using the optional field will generate "has" methods which we can use to +// check if the value was set +message PCredentials { + optional string principal = 1; + optional string tokenClassName = 2; + optional bytes token = 3; + optional string instanceId = 4; +} diff --git a/rpc/grpc/src/main/proto/tabletserver.proto b/rpc/grpc/src/main/proto/tabletserver.proto new file mode 100644 index 00000000000..28e4157b80b --- /dev/null +++ b/rpc/grpc/src/main/proto/tabletserver.proto @@ -0,0 +1,76 @@ +// +// 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. +// + +syntax = "proto3"; + +option java_multiple_files = true; +option java_package = "org.apache.accumulo.grpc.compaction.protobuf"; +option java_outer_classname = "TabletServerProto"; + +import "data.proto"; +import "manager.proto"; + +package tabletserver; + + +message PExternalCompactionJob { + optional string externalCompactionId = 1; + optional data.PKeyExtent extent = 2; + repeated tabletserver.PInputFile files = 3; + optional tabletserver.PIteratorConfig iteratorSettings = 4; + optional string outputFile = 5; + optional bool propagateDeletes = 6; + optional tabletserver.PCompactionKind kind = 7; + optional manager.PFateId fateId = 8; + map overrides = 9; +} + +message PInputFile { + // This is nullable in Thrift but I am not marking as optional as + // I think it should always be set + string metadataFileEntry = 1; + int64 size = 2; + int64 entries = 3; + int64 timestamp = 4; +} + +message PIteratorSetting { + int32 priority = 1; + string name = 2; + string iteratorClass = 3; + map properties = 4; +} + +message PIteratorConfig { + repeated PIteratorSetting iterators = 1; +} + +enum PCompactionKind { + // SELECTOR was removed and used to be zero + // However protobuf3 requires starting with 0 which is the default + UNKNOWN = 0; + SYSTEM = 1; + USER = 2; +} + +message PCompactionStats { + int64 entriesRead = 1; + int64 entriesWritten = 2; + int64 fileSize = 3; +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CancelRequest.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CancelRequest.java new file mode 100644 index 00000000000..2e52f260e6e --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CancelRequest.java @@ -0,0 +1,938 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.CancelRequest} + */ +public final class CancelRequest extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.CancelRequest) + CancelRequestOrBuilder { +private static final long serialVersionUID = 0L; + // Use CancelRequest.newBuilder() to construct. + private CancelRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private CancelRequest() { + externalCompactionId_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new CancelRequest(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CancelRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CancelRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.CancelRequest.class, org.apache.accumulo.grpc.compaction.protobuf.CancelRequest.Builder.class); + } + + private int bitField0_; + public static final int PTINFO_FIELD_NUMBER = 1; + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + @java.lang.Override + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + + public static final int CREDENTIALS_FIELD_NUMBER = 2; + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + @java.lang.Override + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + /** + * .security.PCredentials credentials = 2; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + + public static final int EXTERNALCOMPACTIONID_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object externalCompactionId_ = ""; + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + @java.lang.Override + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } + } + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getCredentials()); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(externalCompactionId_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 3, externalCompactionId_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, getCredentials()); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(externalCompactionId_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, externalCompactionId_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.CancelRequest)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.CancelRequest other = (org.apache.accumulo.grpc.compaction.protobuf.CancelRequest) obj; + + if (hasPtinfo() != other.hasPtinfo()) return false; + if (hasPtinfo()) { + if (!getPtinfo() + .equals(other.getPtinfo())) return false; + } + if (hasCredentials() != other.hasCredentials()) return false; + if (hasCredentials()) { + if (!getCredentials() + .equals(other.getCredentials())) return false; + } + if (!getExternalCompactionId() + .equals(other.getExternalCompactionId())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasPtinfo()) { + hash = (37 * hash) + PTINFO_FIELD_NUMBER; + hash = (53 * hash) + getPtinfo().hashCode(); + } + if (hasCredentials()) { + hash = (37 * hash) + CREDENTIALS_FIELD_NUMBER; + hash = (53 * hash) + getCredentials().hashCode(); + } + hash = (37 * hash) + EXTERNALCOMPACTIONID_FIELD_NUMBER; + hash = (53 * hash) + getExternalCompactionId().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.CancelRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.CancelRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.CancelRequest) + org.apache.accumulo.grpc.compaction.protobuf.CancelRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CancelRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CancelRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.CancelRequest.class, org.apache.accumulo.grpc.compaction.protobuf.CancelRequest.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.CancelRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getPtinfoFieldBuilder(); + getCredentialsFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + externalCompactionId_ = ""; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CancelRequest_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CancelRequest getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CancelRequest.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CancelRequest build() { + org.apache.accumulo.grpc.compaction.protobuf.CancelRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CancelRequest buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.CancelRequest result = new org.apache.accumulo.grpc.compaction.protobuf.CancelRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.CancelRequest result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.ptinfo_ = ptinfoBuilder_ == null + ? ptinfo_ + : ptinfoBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.credentials_ = credentialsBuilder_ == null + ? credentials_ + : credentialsBuilder_.build(); + to_bitField0_ |= 0x00000002; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.externalCompactionId_ = externalCompactionId_; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.CancelRequest) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.CancelRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.CancelRequest other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.CancelRequest.getDefaultInstance()) return this; + if (other.hasPtinfo()) { + mergePtinfo(other.getPtinfo()); + } + if (other.hasCredentials()) { + mergeCredentials(other.getCredentials()); + } + if (!other.getExternalCompactionId().isEmpty()) { + externalCompactionId_ = other.externalCompactionId_; + bitField0_ |= 0x00000004; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getPtinfoFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getCredentialsFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + externalCompactionId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> ptinfoBuilder_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + if (ptinfoBuilder_ == null) { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } else { + return ptinfoBuilder_.getMessage(); + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ptinfo_ = value; + } else { + ptinfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo( + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder builderForValue) { + if (ptinfoBuilder_ == null) { + ptinfo_ = builderForValue.build(); + } else { + ptinfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder mergePtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) && + ptinfo_ != null && + ptinfo_ != org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance()) { + getPtinfoBuilder().mergeFrom(value); + } else { + ptinfo_ = value; + } + } else { + ptinfoBuilder_.mergeFrom(value); + } + if (ptinfo_ != null) { + bitField0_ |= 0x00000001; + onChanged(); + } + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder clearPtinfo() { + bitField0_ = (bitField0_ & ~0x00000001); + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder getPtinfoBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getPtinfoFieldBuilder().getBuilder(); + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + if (ptinfoBuilder_ != null) { + return ptinfoBuilder_.getMessageOrBuilder(); + } else { + return ptinfo_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> + getPtinfoFieldBuilder() { + if (ptinfoBuilder_ == null) { + ptinfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder>( + getPtinfo(), + getParentForChildren(), + isClean()); + ptinfo_ = null; + } + return ptinfoBuilder_; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> credentialsBuilder_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + if (credentialsBuilder_ == null) { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } else { + return credentialsBuilder_.getMessage(); + } + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + credentials_ = value; + } else { + credentialsBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials( + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder builderForValue) { + if (credentialsBuilder_ == null) { + credentials_ = builderForValue.build(); + } else { + credentialsBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder mergeCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) && + credentials_ != null && + credentials_ != org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance()) { + getCredentialsBuilder().mergeFrom(value); + } else { + credentials_ = value; + } + } else { + credentialsBuilder_.mergeFrom(value); + } + if (credentials_ != null) { + bitField0_ |= 0x00000002; + onChanged(); + } + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder clearCredentials() { + bitField0_ = (bitField0_ & ~0x00000002); + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder getCredentialsBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getCredentialsFieldBuilder().getBuilder(); + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + if (credentialsBuilder_ != null) { + return credentialsBuilder_.getMessageOrBuilder(); + } else { + return credentials_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + } + /** + * .security.PCredentials credentials = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> + getCredentialsFieldBuilder() { + if (credentialsBuilder_ == null) { + credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder>( + getCredentials(), + getParentForChildren(), + isClean()); + credentials_ = null; + } + return credentialsBuilder_; + } + + private java.lang.Object externalCompactionId_ = ""; + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string externalCompactionId = 3; + * @param value The externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + externalCompactionId_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string externalCompactionId = 3; + * @return This builder for chaining. + */ + public Builder clearExternalCompactionId() { + externalCompactionId_ = getDefaultInstance().getExternalCompactionId(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string externalCompactionId = 3; + * @param value The bytes for externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + externalCompactionId_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.CancelRequest) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.CancelRequest) + private static final org.apache.accumulo.grpc.compaction.protobuf.CancelRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.CancelRequest(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CancelRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public CancelRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CancelRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CancelRequestOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CancelRequestOrBuilder.java new file mode 100644 index 00000000000..17cdc5c5a30 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CancelRequestOrBuilder.java @@ -0,0 +1,70 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface CancelRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.CancelRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + boolean hasPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder(); + + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + boolean hasCredentials(); + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials(); + /** + * .security.PCredentials credentials = 2; + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder(); + + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + java.lang.String getExternalCompactionId(); + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + com.google.protobuf.ByteString + getExternalCompactionIdBytes(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ClientProto.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ClientProto.java new file mode 100644 index 00000000000..b13d8b160d8 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ClientProto.java @@ -0,0 +1,81 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: client.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public final class ClientProto { + private ClientProto() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_client_ProtoTInfo_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_client_ProtoTInfo_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_client_ProtoTInfo_HeadersEntry_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_client_ProtoTInfo_HeadersEntry_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\014client.proto\022\006client\"n\n\nProtoTInfo\0220\n\007" + + "headers\030\001 \003(\0132\037.client.ProtoTInfo.Header" + + "sEntry\032.\n\014HeadersEntry\022\013\n\003key\030\001 \001(\t\022\r\n\005v" + + "alue\030\002 \001(\t:\0028\001B=\n,org.apache.accumulo.gr" + + "pc.compaction.protobufB\013ClientProtoP\001b\006p" + + "roto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_client_ProtoTInfo_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_client_ProtoTInfo_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_client_ProtoTInfo_descriptor, + new java.lang.String[] { "Headers", }); + internal_static_client_ProtoTInfo_HeadersEntry_descriptor = + internal_static_client_ProtoTInfo_descriptor.getNestedTypes().get(0); + internal_static_client_ProtoTInfo_HeadersEntry_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_client_ProtoTInfo_HeadersEntry_descriptor, + new java.lang.String[] { "Key", "Value", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCompletedRequest.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCompletedRequest.java new file mode 100644 index 00000000000..83c155ed027 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCompletedRequest.java @@ -0,0 +1,1308 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.CompactionCompletedRequest} + */ +public final class CompactionCompletedRequest extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.CompactionCompletedRequest) + CompactionCompletedRequestOrBuilder { +private static final long serialVersionUID = 0L; + // Use CompactionCompletedRequest.newBuilder() to construct. + private CompactionCompletedRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private CompactionCompletedRequest() { + externalCompactionId_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new CompactionCompletedRequest(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionCompletedRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionCompletedRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest.class, org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest.Builder.class); + } + + private int bitField0_; + public static final int PTINFO_FIELD_NUMBER = 1; + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + @java.lang.Override + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + + public static final int CREDENTIALS_FIELD_NUMBER = 2; + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + @java.lang.Override + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + /** + * .security.PCredentials credentials = 2; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + + public static final int EXTERNALCOMPACTIONID_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object externalCompactionId_ = ""; + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + @java.lang.Override + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } + } + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int EXTENT_FIELD_NUMBER = 4; + private org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent extent_; + /** + * .data.PKeyExtent extent = 4; + * @return Whether the extent field is set. + */ + @java.lang.Override + public boolean hasExtent() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * .data.PKeyExtent extent = 4; + * @return The extent. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getExtent() { + return extent_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } + /** + * .data.PKeyExtent extent = 4; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder getExtentOrBuilder() { + return extent_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } + + public static final int STATS_FIELD_NUMBER = 5; + private org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats stats_; + /** + * .tabletserver.PCompactionStats stats = 5; + * @return Whether the stats field is set. + */ + @java.lang.Override + public boolean hasStats() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * .tabletserver.PCompactionStats stats = 5; + * @return The stats. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats getStats() { + return stats_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.getDefaultInstance() : stats_; + } + /** + * .tabletserver.PCompactionStats stats = 5; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatsOrBuilder getStatsOrBuilder() { + return stats_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.getDefaultInstance() : stats_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getCredentials()); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(externalCompactionId_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 3, externalCompactionId_); + } + if (((bitField0_ & 0x00000004) != 0)) { + output.writeMessage(4, getExtent()); + } + if (((bitField0_ & 0x00000008) != 0)) { + output.writeMessage(5, getStats()); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, getCredentials()); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(externalCompactionId_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, externalCompactionId_); + } + if (((bitField0_ & 0x00000004) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(4, getExtent()); + } + if (((bitField0_ & 0x00000008) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(5, getStats()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest other = (org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest) obj; + + if (hasPtinfo() != other.hasPtinfo()) return false; + if (hasPtinfo()) { + if (!getPtinfo() + .equals(other.getPtinfo())) return false; + } + if (hasCredentials() != other.hasCredentials()) return false; + if (hasCredentials()) { + if (!getCredentials() + .equals(other.getCredentials())) return false; + } + if (!getExternalCompactionId() + .equals(other.getExternalCompactionId())) return false; + if (hasExtent() != other.hasExtent()) return false; + if (hasExtent()) { + if (!getExtent() + .equals(other.getExtent())) return false; + } + if (hasStats() != other.hasStats()) return false; + if (hasStats()) { + if (!getStats() + .equals(other.getStats())) return false; + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasPtinfo()) { + hash = (37 * hash) + PTINFO_FIELD_NUMBER; + hash = (53 * hash) + getPtinfo().hashCode(); + } + if (hasCredentials()) { + hash = (37 * hash) + CREDENTIALS_FIELD_NUMBER; + hash = (53 * hash) + getCredentials().hashCode(); + } + hash = (37 * hash) + EXTERNALCOMPACTIONID_FIELD_NUMBER; + hash = (53 * hash) + getExternalCompactionId().hashCode(); + if (hasExtent()) { + hash = (37 * hash) + EXTENT_FIELD_NUMBER; + hash = (53 * hash) + getExtent().hashCode(); + } + if (hasStats()) { + hash = (37 * hash) + STATS_FIELD_NUMBER; + hash = (53 * hash) + getStats().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.CompactionCompletedRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.CompactionCompletedRequest) + org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionCompletedRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionCompletedRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest.class, org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getPtinfoFieldBuilder(); + getCredentialsFieldBuilder(); + getExtentFieldBuilder(); + getStatsFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + externalCompactionId_ = ""; + extent_ = null; + if (extentBuilder_ != null) { + extentBuilder_.dispose(); + extentBuilder_ = null; + } + stats_ = null; + if (statsBuilder_ != null) { + statsBuilder_.dispose(); + statsBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionCompletedRequest_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest build() { + org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest result = new org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.ptinfo_ = ptinfoBuilder_ == null + ? ptinfo_ + : ptinfoBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.credentials_ = credentialsBuilder_ == null + ? credentials_ + : credentialsBuilder_.build(); + to_bitField0_ |= 0x00000002; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.externalCompactionId_ = externalCompactionId_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.extent_ = extentBuilder_ == null + ? extent_ + : extentBuilder_.build(); + to_bitField0_ |= 0x00000004; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.stats_ = statsBuilder_ == null + ? stats_ + : statsBuilder_.build(); + to_bitField0_ |= 0x00000008; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest.getDefaultInstance()) return this; + if (other.hasPtinfo()) { + mergePtinfo(other.getPtinfo()); + } + if (other.hasCredentials()) { + mergeCredentials(other.getCredentials()); + } + if (!other.getExternalCompactionId().isEmpty()) { + externalCompactionId_ = other.externalCompactionId_; + bitField0_ |= 0x00000004; + onChanged(); + } + if (other.hasExtent()) { + mergeExtent(other.getExtent()); + } + if (other.hasStats()) { + mergeStats(other.getStats()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getPtinfoFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getCredentialsFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + externalCompactionId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + input.readMessage( + getExtentFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 42: { + input.readMessage( + getStatsFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000010; + break; + } // case 42 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> ptinfoBuilder_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + if (ptinfoBuilder_ == null) { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } else { + return ptinfoBuilder_.getMessage(); + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ptinfo_ = value; + } else { + ptinfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo( + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder builderForValue) { + if (ptinfoBuilder_ == null) { + ptinfo_ = builderForValue.build(); + } else { + ptinfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder mergePtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) && + ptinfo_ != null && + ptinfo_ != org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance()) { + getPtinfoBuilder().mergeFrom(value); + } else { + ptinfo_ = value; + } + } else { + ptinfoBuilder_.mergeFrom(value); + } + if (ptinfo_ != null) { + bitField0_ |= 0x00000001; + onChanged(); + } + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder clearPtinfo() { + bitField0_ = (bitField0_ & ~0x00000001); + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder getPtinfoBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getPtinfoFieldBuilder().getBuilder(); + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + if (ptinfoBuilder_ != null) { + return ptinfoBuilder_.getMessageOrBuilder(); + } else { + return ptinfo_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> + getPtinfoFieldBuilder() { + if (ptinfoBuilder_ == null) { + ptinfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder>( + getPtinfo(), + getParentForChildren(), + isClean()); + ptinfo_ = null; + } + return ptinfoBuilder_; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> credentialsBuilder_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + if (credentialsBuilder_ == null) { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } else { + return credentialsBuilder_.getMessage(); + } + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + credentials_ = value; + } else { + credentialsBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials( + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder builderForValue) { + if (credentialsBuilder_ == null) { + credentials_ = builderForValue.build(); + } else { + credentialsBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder mergeCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) && + credentials_ != null && + credentials_ != org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance()) { + getCredentialsBuilder().mergeFrom(value); + } else { + credentials_ = value; + } + } else { + credentialsBuilder_.mergeFrom(value); + } + if (credentials_ != null) { + bitField0_ |= 0x00000002; + onChanged(); + } + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder clearCredentials() { + bitField0_ = (bitField0_ & ~0x00000002); + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder getCredentialsBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getCredentialsFieldBuilder().getBuilder(); + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + if (credentialsBuilder_ != null) { + return credentialsBuilder_.getMessageOrBuilder(); + } else { + return credentials_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + } + /** + * .security.PCredentials credentials = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> + getCredentialsFieldBuilder() { + if (credentialsBuilder_ == null) { + credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder>( + getCredentials(), + getParentForChildren(), + isClean()); + credentials_ = null; + } + return credentialsBuilder_; + } + + private java.lang.Object externalCompactionId_ = ""; + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string externalCompactionId = 3; + * @param value The externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + externalCompactionId_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string externalCompactionId = 3; + * @return This builder for chaining. + */ + public Builder clearExternalCompactionId() { + externalCompactionId_ = getDefaultInstance().getExternalCompactionId(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string externalCompactionId = 3; + * @param value The bytes for externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + externalCompactionId_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent extent_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder> extentBuilder_; + /** + * .data.PKeyExtent extent = 4; + * @return Whether the extent field is set. + */ + public boolean hasExtent() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * .data.PKeyExtent extent = 4; + * @return The extent. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getExtent() { + if (extentBuilder_ == null) { + return extent_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } else { + return extentBuilder_.getMessage(); + } + } + /** + * .data.PKeyExtent extent = 4; + */ + public Builder setExtent(org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent value) { + if (extentBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + extent_ = value; + } else { + extentBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * .data.PKeyExtent extent = 4; + */ + public Builder setExtent( + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder builderForValue) { + if (extentBuilder_ == null) { + extent_ = builderForValue.build(); + } else { + extentBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * .data.PKeyExtent extent = 4; + */ + public Builder mergeExtent(org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent value) { + if (extentBuilder_ == null) { + if (((bitField0_ & 0x00000008) != 0) && + extent_ != null && + extent_ != org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance()) { + getExtentBuilder().mergeFrom(value); + } else { + extent_ = value; + } + } else { + extentBuilder_.mergeFrom(value); + } + if (extent_ != null) { + bitField0_ |= 0x00000008; + onChanged(); + } + return this; + } + /** + * .data.PKeyExtent extent = 4; + */ + public Builder clearExtent() { + bitField0_ = (bitField0_ & ~0x00000008); + extent_ = null; + if (extentBuilder_ != null) { + extentBuilder_.dispose(); + extentBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .data.PKeyExtent extent = 4; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder getExtentBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getExtentFieldBuilder().getBuilder(); + } + /** + * .data.PKeyExtent extent = 4; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder getExtentOrBuilder() { + if (extentBuilder_ != null) { + return extentBuilder_.getMessageOrBuilder(); + } else { + return extent_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } + } + /** + * .data.PKeyExtent extent = 4; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder> + getExtentFieldBuilder() { + if (extentBuilder_ == null) { + extentBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder>( + getExtent(), + getParentForChildren(), + isClean()); + extent_ = null; + } + return extentBuilder_; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats stats_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatsOrBuilder> statsBuilder_; + /** + * .tabletserver.PCompactionStats stats = 5; + * @return Whether the stats field is set. + */ + public boolean hasStats() { + return ((bitField0_ & 0x00000010) != 0); + } + /** + * .tabletserver.PCompactionStats stats = 5; + * @return The stats. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats getStats() { + if (statsBuilder_ == null) { + return stats_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.getDefaultInstance() : stats_; + } else { + return statsBuilder_.getMessage(); + } + } + /** + * .tabletserver.PCompactionStats stats = 5; + */ + public Builder setStats(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats value) { + if (statsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + stats_ = value; + } else { + statsBuilder_.setMessage(value); + } + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * .tabletserver.PCompactionStats stats = 5; + */ + public Builder setStats( + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.Builder builderForValue) { + if (statsBuilder_ == null) { + stats_ = builderForValue.build(); + } else { + statsBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * .tabletserver.PCompactionStats stats = 5; + */ + public Builder mergeStats(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats value) { + if (statsBuilder_ == null) { + if (((bitField0_ & 0x00000010) != 0) && + stats_ != null && + stats_ != org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.getDefaultInstance()) { + getStatsBuilder().mergeFrom(value); + } else { + stats_ = value; + } + } else { + statsBuilder_.mergeFrom(value); + } + if (stats_ != null) { + bitField0_ |= 0x00000010; + onChanged(); + } + return this; + } + /** + * .tabletserver.PCompactionStats stats = 5; + */ + public Builder clearStats() { + bitField0_ = (bitField0_ & ~0x00000010); + stats_ = null; + if (statsBuilder_ != null) { + statsBuilder_.dispose(); + statsBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .tabletserver.PCompactionStats stats = 5; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.Builder getStatsBuilder() { + bitField0_ |= 0x00000010; + onChanged(); + return getStatsFieldBuilder().getBuilder(); + } + /** + * .tabletserver.PCompactionStats stats = 5; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatsOrBuilder getStatsOrBuilder() { + if (statsBuilder_ != null) { + return statsBuilder_.getMessageOrBuilder(); + } else { + return stats_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.getDefaultInstance() : stats_; + } + } + /** + * .tabletserver.PCompactionStats stats = 5; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatsOrBuilder> + getStatsFieldBuilder() { + if (statsBuilder_ == null) { + statsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatsOrBuilder>( + getStats(), + getParentForChildren(), + isClean()); + stats_ = null; + } + return statsBuilder_; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.CompactionCompletedRequest) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.CompactionCompletedRequest) + private static final org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public CompactionCompletedRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCompletedRequestOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCompletedRequestOrBuilder.java new file mode 100644 index 00000000000..e8e9341375e --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCompletedRequestOrBuilder.java @@ -0,0 +1,100 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface CompactionCompletedRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.CompactionCompletedRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + boolean hasPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder(); + + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + boolean hasCredentials(); + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials(); + /** + * .security.PCredentials credentials = 2; + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder(); + + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + java.lang.String getExternalCompactionId(); + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + com.google.protobuf.ByteString + getExternalCompactionIdBytes(); + + /** + * .data.PKeyExtent extent = 4; + * @return Whether the extent field is set. + */ + boolean hasExtent(); + /** + * .data.PKeyExtent extent = 4; + * @return The extent. + */ + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getExtent(); + /** + * .data.PKeyExtent extent = 4; + */ + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder getExtentOrBuilder(); + + /** + * .tabletserver.PCompactionStats stats = 5; + * @return Whether the stats field is set. + */ + boolean hasStats(); + /** + * .tabletserver.PCompactionStats stats = 5; + * @return The stats. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats getStats(); + /** + * .tabletserver.PCompactionStats stats = 5; + */ + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatsOrBuilder getStatsOrBuilder(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCoordinatorServiceGrpc.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCoordinatorServiceGrpc.java new file mode 100644 index 00000000000..6eb79ad4678 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCoordinatorServiceGrpc.java @@ -0,0 +1,854 @@ +/* + * 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.grpc.compaction.protobuf; + +import static io.grpc.MethodDescriptor.generateFullMethodName; + +/** + *
+ * Interface exported by the server.
+ * 
+ */ +@io.grpc.stub.annotations.GrpcGenerated +public final class CompactionCoordinatorServiceGrpc { + + private CompactionCoordinatorServiceGrpc() {} + + public static final java.lang.String SERVICE_NAME = "compaction_coordinator.CompactionCoordinatorService"; + + // Static method descriptors that strictly reflect the proto. + private static volatile io.grpc.MethodDescriptor getGetCompactionJobMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "GetCompactionJob", + requestType = org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest.class, + responseType = org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getGetCompactionJobMethod() { + io.grpc.MethodDescriptor getGetCompactionJobMethod; + if ((getGetCompactionJobMethod = CompactionCoordinatorServiceGrpc.getGetCompactionJobMethod) == null) { + synchronized (CompactionCoordinatorServiceGrpc.class) { + if ((getGetCompactionJobMethod = CompactionCoordinatorServiceGrpc.getGetCompactionJobMethod) == null) { + CompactionCoordinatorServiceGrpc.getGetCompactionJobMethod = getGetCompactionJobMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "GetCompactionJob")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob.getDefaultInstance())) + .setSchemaDescriptor(new CompactionCoordinatorServiceMethodDescriptorSupplier("GetCompactionJob")) + .build(); + } + } + } + return getGetCompactionJobMethod; + } + + private static volatile io.grpc.MethodDescriptor getCompactionCompletedMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "CompactionCompleted", + requestType = org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest.class, + responseType = com.google.protobuf.Empty.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getCompactionCompletedMethod() { + io.grpc.MethodDescriptor getCompactionCompletedMethod; + if ((getCompactionCompletedMethod = CompactionCoordinatorServiceGrpc.getCompactionCompletedMethod) == null) { + synchronized (CompactionCoordinatorServiceGrpc.class) { + if ((getCompactionCompletedMethod = CompactionCoordinatorServiceGrpc.getCompactionCompletedMethod) == null) { + CompactionCoordinatorServiceGrpc.getCompactionCompletedMethod = getCompactionCompletedMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "CompactionCompleted")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.google.protobuf.Empty.getDefaultInstance())) + .setSchemaDescriptor(new CompactionCoordinatorServiceMethodDescriptorSupplier("CompactionCompleted")) + .build(); + } + } + } + return getCompactionCompletedMethod; + } + + private static volatile io.grpc.MethodDescriptor getUpdateCompactionStatusMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "UpdateCompactionStatus", + requestType = org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest.class, + responseType = com.google.protobuf.Empty.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getUpdateCompactionStatusMethod() { + io.grpc.MethodDescriptor getUpdateCompactionStatusMethod; + if ((getUpdateCompactionStatusMethod = CompactionCoordinatorServiceGrpc.getUpdateCompactionStatusMethod) == null) { + synchronized (CompactionCoordinatorServiceGrpc.class) { + if ((getUpdateCompactionStatusMethod = CompactionCoordinatorServiceGrpc.getUpdateCompactionStatusMethod) == null) { + CompactionCoordinatorServiceGrpc.getUpdateCompactionStatusMethod = getUpdateCompactionStatusMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "UpdateCompactionStatus")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.google.protobuf.Empty.getDefaultInstance())) + .setSchemaDescriptor(new CompactionCoordinatorServiceMethodDescriptorSupplier("UpdateCompactionStatus")) + .build(); + } + } + } + return getUpdateCompactionStatusMethod; + } + + private static volatile io.grpc.MethodDescriptor getCompactionFailedMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "CompactionFailed", + requestType = org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest.class, + responseType = com.google.protobuf.Empty.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getCompactionFailedMethod() { + io.grpc.MethodDescriptor getCompactionFailedMethod; + if ((getCompactionFailedMethod = CompactionCoordinatorServiceGrpc.getCompactionFailedMethod) == null) { + synchronized (CompactionCoordinatorServiceGrpc.class) { + if ((getCompactionFailedMethod = CompactionCoordinatorServiceGrpc.getCompactionFailedMethod) == null) { + CompactionCoordinatorServiceGrpc.getCompactionFailedMethod = getCompactionFailedMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "CompactionFailed")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.google.protobuf.Empty.getDefaultInstance())) + .setSchemaDescriptor(new CompactionCoordinatorServiceMethodDescriptorSupplier("CompactionFailed")) + .build(); + } + } + } + return getCompactionFailedMethod; + } + + private static volatile io.grpc.MethodDescriptor getGetRunningCompactionsMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "GetRunningCompactions", + requestType = org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest.class, + responseType = org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getGetRunningCompactionsMethod() { + io.grpc.MethodDescriptor getGetRunningCompactionsMethod; + if ((getGetRunningCompactionsMethod = CompactionCoordinatorServiceGrpc.getGetRunningCompactionsMethod) == null) { + synchronized (CompactionCoordinatorServiceGrpc.class) { + if ((getGetRunningCompactionsMethod = CompactionCoordinatorServiceGrpc.getGetRunningCompactionsMethod) == null) { + CompactionCoordinatorServiceGrpc.getGetRunningCompactionsMethod = getGetRunningCompactionsMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "GetRunningCompactions")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.getDefaultInstance())) + .setSchemaDescriptor(new CompactionCoordinatorServiceMethodDescriptorSupplier("GetRunningCompactions")) + .build(); + } + } + } + return getGetRunningCompactionsMethod; + } + + private static volatile io.grpc.MethodDescriptor getGetCompletedCompactionsMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "GetCompletedCompactions", + requestType = org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest.class, + responseType = org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getGetCompletedCompactionsMethod() { + io.grpc.MethodDescriptor getGetCompletedCompactionsMethod; + if ((getGetCompletedCompactionsMethod = CompactionCoordinatorServiceGrpc.getGetCompletedCompactionsMethod) == null) { + synchronized (CompactionCoordinatorServiceGrpc.class) { + if ((getGetCompletedCompactionsMethod = CompactionCoordinatorServiceGrpc.getGetCompletedCompactionsMethod) == null) { + CompactionCoordinatorServiceGrpc.getGetCompletedCompactionsMethod = getGetCompletedCompactionsMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "GetCompletedCompactions")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.getDefaultInstance())) + .setSchemaDescriptor(new CompactionCoordinatorServiceMethodDescriptorSupplier("GetCompletedCompactions")) + .build(); + } + } + } + return getGetCompletedCompactionsMethod; + } + + private static volatile io.grpc.MethodDescriptor getCancelMethod; + + @io.grpc.stub.annotations.RpcMethod( + fullMethodName = SERVICE_NAME + '/' + "Cancel", + requestType = org.apache.accumulo.grpc.compaction.protobuf.CancelRequest.class, + responseType = com.google.protobuf.Empty.class, + methodType = io.grpc.MethodDescriptor.MethodType.UNARY) + public static io.grpc.MethodDescriptor getCancelMethod() { + io.grpc.MethodDescriptor getCancelMethod; + if ((getCancelMethod = CompactionCoordinatorServiceGrpc.getCancelMethod) == null) { + synchronized (CompactionCoordinatorServiceGrpc.class) { + if ((getCancelMethod = CompactionCoordinatorServiceGrpc.getCancelMethod) == null) { + CompactionCoordinatorServiceGrpc.getCancelMethod = getCancelMethod = + io.grpc.MethodDescriptor.newBuilder() + .setType(io.grpc.MethodDescriptor.MethodType.UNARY) + .setFullMethodName(generateFullMethodName(SERVICE_NAME, "Cancel")) + .setSampledToLocalTracing(true) + .setRequestMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + org.apache.accumulo.grpc.compaction.protobuf.CancelRequest.getDefaultInstance())) + .setResponseMarshaller(io.grpc.protobuf.ProtoUtils.marshaller( + com.google.protobuf.Empty.getDefaultInstance())) + .setSchemaDescriptor(new CompactionCoordinatorServiceMethodDescriptorSupplier("Cancel")) + .build(); + } + } + } + return getCancelMethod; + } + + /** + * Creates a new async stub that supports all call types for the service + */ + public static CompactionCoordinatorServiceStub newStub(io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public CompactionCoordinatorServiceStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new CompactionCoordinatorServiceStub(channel, callOptions); + } + }; + return CompactionCoordinatorServiceStub.newStub(factory, channel); + } + + /** + * Creates a new blocking-style stub that supports unary and streaming output calls on the service + */ + public static CompactionCoordinatorServiceBlockingStub newBlockingStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public CompactionCoordinatorServiceBlockingStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new CompactionCoordinatorServiceBlockingStub(channel, callOptions); + } + }; + return CompactionCoordinatorServiceBlockingStub.newStub(factory, channel); + } + + /** + * Creates a new ListenableFuture-style stub that supports unary calls on the service + */ + public static CompactionCoordinatorServiceFutureStub newFutureStub( + io.grpc.Channel channel) { + io.grpc.stub.AbstractStub.StubFactory factory = + new io.grpc.stub.AbstractStub.StubFactory() { + @java.lang.Override + public CompactionCoordinatorServiceFutureStub newStub(io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new CompactionCoordinatorServiceFutureStub(channel, callOptions); + } + }; + return CompactionCoordinatorServiceFutureStub.newStub(factory, channel); + } + + /** + *
+   * Interface exported by the server.
+   * 
+ */ + public interface AsyncService { + + /** + *
+     * Called by Compactor to get the next compaction job
+     * 
+ */ + default void getCompactionJob(org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getGetCompactionJobMethod(), responseObserver); + } + + /** + *
+     * Called by Compactor on successful completion of compaction job
+     * 
+ */ + default void compactionCompleted(org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getCompactionCompletedMethod(), responseObserver); + } + + /** + *
+     * Called by Compactor to update the Coordinator with the state of the compaction
+     * 
+ */ + default void updateCompactionStatus(org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getUpdateCompactionStatusMethod(), responseObserver); + } + + /** + *
+     * Called by Compactor on unsuccessful completion of compaction job
+     * 
+ */ + default void compactionFailed(org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getCompactionFailedMethod(), responseObserver); + } + + /** + *
+     * Called by the Monitor to get progress information
+     * 
+ */ + default void getRunningCompactions(org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getGetRunningCompactionsMethod(), responseObserver); + } + + /** + *
+     * Called by the Monitor to get progress information
+     * 
+ */ + default void getCompletedCompactions(org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getGetCompletedCompactionsMethod(), responseObserver); + } + + /** + *
+     * Called by Compactor on unsuccessful completion of compaction job
+     * 
+ */ + default void cancel(org.apache.accumulo.grpc.compaction.protobuf.CancelRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ServerCalls.asyncUnimplementedUnaryCall(getCancelMethod(), responseObserver); + } + } + + /** + * Base class for the server implementation of the service CompactionCoordinatorService. + *
+   * Interface exported by the server.
+   * 
+ */ + public static abstract class CompactionCoordinatorServiceImplBase + implements io.grpc.BindableService, AsyncService { + + @java.lang.Override public final io.grpc.ServerServiceDefinition bindService() { + return CompactionCoordinatorServiceGrpc.bindService(this); + } + } + + /** + * A stub to allow clients to do asynchronous rpc calls to service CompactionCoordinatorService. + *
+   * Interface exported by the server.
+   * 
+ */ + public static final class CompactionCoordinatorServiceStub + extends io.grpc.stub.AbstractAsyncStub { + private CompactionCoordinatorServiceStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected CompactionCoordinatorServiceStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new CompactionCoordinatorServiceStub(channel, callOptions); + } + + /** + *
+     * Called by Compactor to get the next compaction job
+     * 
+ */ + public void getCompactionJob(org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getGetCompactionJobMethod(), getCallOptions()), request, responseObserver); + } + + /** + *
+     * Called by Compactor on successful completion of compaction job
+     * 
+ */ + public void compactionCompleted(org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getCompactionCompletedMethod(), getCallOptions()), request, responseObserver); + } + + /** + *
+     * Called by Compactor to update the Coordinator with the state of the compaction
+     * 
+ */ + public void updateCompactionStatus(org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getUpdateCompactionStatusMethod(), getCallOptions()), request, responseObserver); + } + + /** + *
+     * Called by Compactor on unsuccessful completion of compaction job
+     * 
+ */ + public void compactionFailed(org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getCompactionFailedMethod(), getCallOptions()), request, responseObserver); + } + + /** + *
+     * Called by the Monitor to get progress information
+     * 
+ */ + public void getRunningCompactions(org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getGetRunningCompactionsMethod(), getCallOptions()), request, responseObserver); + } + + /** + *
+     * Called by the Monitor to get progress information
+     * 
+ */ + public void getCompletedCompactions(org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getGetCompletedCompactionsMethod(), getCallOptions()), request, responseObserver); + } + + /** + *
+     * Called by Compactor on unsuccessful completion of compaction job
+     * 
+ */ + public void cancel(org.apache.accumulo.grpc.compaction.protobuf.CancelRequest request, + io.grpc.stub.StreamObserver responseObserver) { + io.grpc.stub.ClientCalls.asyncUnaryCall( + getChannel().newCall(getCancelMethod(), getCallOptions()), request, responseObserver); + } + } + + /** + * A stub to allow clients to do synchronous rpc calls to service CompactionCoordinatorService. + *
+   * Interface exported by the server.
+   * 
+ */ + public static final class CompactionCoordinatorServiceBlockingStub + extends io.grpc.stub.AbstractBlockingStub { + private CompactionCoordinatorServiceBlockingStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected CompactionCoordinatorServiceBlockingStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new CompactionCoordinatorServiceBlockingStub(channel, callOptions); + } + + /** + *
+     * Called by Compactor to get the next compaction job
+     * 
+ */ + public org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob getCompactionJob(org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getGetCompactionJobMethod(), getCallOptions(), request); + } + + /** + *
+     * Called by Compactor on successful completion of compaction job
+     * 
+ */ + public com.google.protobuf.Empty compactionCompleted(org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getCompactionCompletedMethod(), getCallOptions(), request); + } + + /** + *
+     * Called by Compactor to update the Coordinator with the state of the compaction
+     * 
+ */ + public com.google.protobuf.Empty updateCompactionStatus(org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getUpdateCompactionStatusMethod(), getCallOptions(), request); + } + + /** + *
+     * Called by Compactor on unsuccessful completion of compaction job
+     * 
+ */ + public com.google.protobuf.Empty compactionFailed(org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getCompactionFailedMethod(), getCallOptions(), request); + } + + /** + *
+     * Called by the Monitor to get progress information
+     * 
+ */ + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList getRunningCompactions(org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getGetRunningCompactionsMethod(), getCallOptions(), request); + } + + /** + *
+     * Called by the Monitor to get progress information
+     * 
+ */ + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList getCompletedCompactions(org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getGetCompletedCompactionsMethod(), getCallOptions(), request); + } + + /** + *
+     * Called by Compactor on unsuccessful completion of compaction job
+     * 
+ */ + public com.google.protobuf.Empty cancel(org.apache.accumulo.grpc.compaction.protobuf.CancelRequest request) { + return io.grpc.stub.ClientCalls.blockingUnaryCall( + getChannel(), getCancelMethod(), getCallOptions(), request); + } + } + + /** + * A stub to allow clients to do ListenableFuture-style rpc calls to service CompactionCoordinatorService. + *
+   * Interface exported by the server.
+   * 
+ */ + public static final class CompactionCoordinatorServiceFutureStub + extends io.grpc.stub.AbstractFutureStub { + private CompactionCoordinatorServiceFutureStub( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + super(channel, callOptions); + } + + @java.lang.Override + protected CompactionCoordinatorServiceFutureStub build( + io.grpc.Channel channel, io.grpc.CallOptions callOptions) { + return new CompactionCoordinatorServiceFutureStub(channel, callOptions); + } + + /** + *
+     * Called by Compactor to get the next compaction job
+     * 
+ */ + public com.google.common.util.concurrent.ListenableFuture getCompactionJob( + org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getGetCompactionJobMethod(), getCallOptions()), request); + } + + /** + *
+     * Called by Compactor on successful completion of compaction job
+     * 
+ */ + public com.google.common.util.concurrent.ListenableFuture compactionCompleted( + org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getCompactionCompletedMethod(), getCallOptions()), request); + } + + /** + *
+     * Called by Compactor to update the Coordinator with the state of the compaction
+     * 
+ */ + public com.google.common.util.concurrent.ListenableFuture updateCompactionStatus( + org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getUpdateCompactionStatusMethod(), getCallOptions()), request); + } + + /** + *
+     * Called by Compactor on unsuccessful completion of compaction job
+     * 
+ */ + public com.google.common.util.concurrent.ListenableFuture compactionFailed( + org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getCompactionFailedMethod(), getCallOptions()), request); + } + + /** + *
+     * Called by the Monitor to get progress information
+     * 
+ */ + public com.google.common.util.concurrent.ListenableFuture getRunningCompactions( + org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getGetRunningCompactionsMethod(), getCallOptions()), request); + } + + /** + *
+     * Called by the Monitor to get progress information
+     * 
+ */ + public com.google.common.util.concurrent.ListenableFuture getCompletedCompactions( + org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getGetCompletedCompactionsMethod(), getCallOptions()), request); + } + + /** + *
+     * Called by Compactor on unsuccessful completion of compaction job
+     * 
+ */ + public com.google.common.util.concurrent.ListenableFuture cancel( + org.apache.accumulo.grpc.compaction.protobuf.CancelRequest request) { + return io.grpc.stub.ClientCalls.futureUnaryCall( + getChannel().newCall(getCancelMethod(), getCallOptions()), request); + } + } + + private static final int METHODID_GET_COMPACTION_JOB = 0; + private static final int METHODID_COMPACTION_COMPLETED = 1; + private static final int METHODID_UPDATE_COMPACTION_STATUS = 2; + private static final int METHODID_COMPACTION_FAILED = 3; + private static final int METHODID_GET_RUNNING_COMPACTIONS = 4; + private static final int METHODID_GET_COMPLETED_COMPACTIONS = 5; + private static final int METHODID_CANCEL = 6; + + private static final class MethodHandlers implements + io.grpc.stub.ServerCalls.UnaryMethod, + io.grpc.stub.ServerCalls.ServerStreamingMethod, + io.grpc.stub.ServerCalls.ClientStreamingMethod, + io.grpc.stub.ServerCalls.BidiStreamingMethod { + private final AsyncService serviceImpl; + private final int methodId; + + MethodHandlers(AsyncService serviceImpl, int methodId) { + this.serviceImpl = serviceImpl; + this.methodId = methodId; + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public void invoke(Req request, io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + case METHODID_GET_COMPACTION_JOB: + serviceImpl.getCompactionJob((org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + case METHODID_COMPACTION_COMPLETED: + serviceImpl.compactionCompleted((org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + case METHODID_UPDATE_COMPACTION_STATUS: + serviceImpl.updateCompactionStatus((org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + case METHODID_COMPACTION_FAILED: + serviceImpl.compactionFailed((org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + case METHODID_GET_RUNNING_COMPACTIONS: + serviceImpl.getRunningCompactions((org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + case METHODID_GET_COMPLETED_COMPACTIONS: + serviceImpl.getCompletedCompactions((org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + case METHODID_CANCEL: + serviceImpl.cancel((org.apache.accumulo.grpc.compaction.protobuf.CancelRequest) request, + (io.grpc.stub.StreamObserver) responseObserver); + break; + default: + throw new AssertionError(); + } + } + + @java.lang.Override + @java.lang.SuppressWarnings("unchecked") + public io.grpc.stub.StreamObserver invoke( + io.grpc.stub.StreamObserver responseObserver) { + switch (methodId) { + default: + throw new AssertionError(); + } + } + } + + public static final io.grpc.ServerServiceDefinition bindService(AsyncService service) { + return io.grpc.ServerServiceDefinition.builder(getServiceDescriptor()) + .addMethod( + getGetCompactionJobMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest, + org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob>( + service, METHODID_GET_COMPACTION_JOB))) + .addMethod( + getCompactionCompletedMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest, + com.google.protobuf.Empty>( + service, METHODID_COMPACTION_COMPLETED))) + .addMethod( + getUpdateCompactionStatusMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest, + com.google.protobuf.Empty>( + service, METHODID_UPDATE_COMPACTION_STATUS))) + .addMethod( + getCompactionFailedMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest, + com.google.protobuf.Empty>( + service, METHODID_COMPACTION_FAILED))) + .addMethod( + getGetRunningCompactionsMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest, + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList>( + service, METHODID_GET_RUNNING_COMPACTIONS))) + .addMethod( + getGetCompletedCompactionsMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest, + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList>( + service, METHODID_GET_COMPLETED_COMPACTIONS))) + .addMethod( + getCancelMethod(), + io.grpc.stub.ServerCalls.asyncUnaryCall( + new MethodHandlers< + org.apache.accumulo.grpc.compaction.protobuf.CancelRequest, + com.google.protobuf.Empty>( + service, METHODID_CANCEL))) + .build(); + } + + private static abstract class CompactionCoordinatorServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoFileDescriptorSupplier, io.grpc.protobuf.ProtoServiceDescriptorSupplier { + CompactionCoordinatorServiceBaseDescriptorSupplier() {} + + @java.lang.Override + public com.google.protobuf.Descriptors.FileDescriptor getFileDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.getDescriptor(); + } + + @java.lang.Override + public com.google.protobuf.Descriptors.ServiceDescriptor getServiceDescriptor() { + return getFileDescriptor().findServiceByName("CompactionCoordinatorService"); + } + } + + private static final class CompactionCoordinatorServiceFileDescriptorSupplier + extends CompactionCoordinatorServiceBaseDescriptorSupplier { + CompactionCoordinatorServiceFileDescriptorSupplier() {} + } + + private static final class CompactionCoordinatorServiceMethodDescriptorSupplier + extends CompactionCoordinatorServiceBaseDescriptorSupplier + implements io.grpc.protobuf.ProtoMethodDescriptorSupplier { + private final java.lang.String methodName; + + CompactionCoordinatorServiceMethodDescriptorSupplier(java.lang.String methodName) { + this.methodName = methodName; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.MethodDescriptor getMethodDescriptor() { + return getServiceDescriptor().findMethodByName(methodName); + } + } + + private static volatile io.grpc.ServiceDescriptor serviceDescriptor; + + public static io.grpc.ServiceDescriptor getServiceDescriptor() { + io.grpc.ServiceDescriptor result = serviceDescriptor; + if (result == null) { + synchronized (CompactionCoordinatorServiceGrpc.class) { + result = serviceDescriptor; + if (result == null) { + serviceDescriptor = result = io.grpc.ServiceDescriptor.newBuilder(SERVICE_NAME) + .setSchemaDescriptor(new CompactionCoordinatorServiceFileDescriptorSupplier()) + .addMethod(getGetCompactionJobMethod()) + .addMethod(getCompactionCompletedMethod()) + .addMethod(getUpdateCompactionStatusMethod()) + .addMethod(getCompactionFailedMethod()) + .addMethod(getGetRunningCompactionsMethod()) + .addMethod(getGetCompletedCompactionsMethod()) + .addMethod(getCancelMethod()) + .build(); + } + } + } + return result; + } +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCoordinatorServiceProto.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCoordinatorServiceProto.java new file mode 100644 index 00000000000..20a5bd39fb6 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionCoordinatorServiceProto.java @@ -0,0 +1,286 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public final class CompactionCoordinatorServiceProto { + private CompactionCoordinatorServiceProto() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_CompactionJobRequest_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_CompactionJobRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_CompactionCompletedRequest_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_CompactionCompletedRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_UpdateCompactionStatusRequest_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_UpdateCompactionStatusRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_CompactionFailedRequest_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_CompactionFailedRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_GetRunningCompactionsRequest_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_GetRunningCompactionsRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_GetCompletedCompactionsRequest_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_GetCompletedCompactionsRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_CancelRequest_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_CancelRequest_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_PExternalCompaction_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_PExternalCompaction_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_PExternalCompaction_UpdatesEntry_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_PExternalCompaction_UpdatesEntry_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_PExternalCompactionList_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_PExternalCompactionList_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_PExternalCompactionList_CompactionsEntry_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_PExternalCompactionList_CompactionsEntry_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_PNextCompactionJob_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_PNextCompactionJob_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_compaction_coordinator_PCompactionStatusUpdate_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_compaction_coordinator_PCompactionStatusUpdate_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\034compaction-coordinator.proto\022\026compacti" + + "on_coordinator\032\033google/protobuf/empty.pr" + + "oto\032\016security.proto\032\014client.proto\032\ndata." + + "proto\032\022tabletserver.proto\032\rmanager.proto" + + "\"\253\001\n\024CompactionJobRequest\022\"\n\006ptinfo\030\001 \001(" + + "\0132\022.client.ProtoTInfo\022+\n\013credentials\030\002 \001" + + "(\0132\026.security.PCredentials\022\021\n\tgroupName\030" + + "\003 \001(\t\022\021\n\tcompactor\030\004 \001(\t\022\034\n\024externalComp" + + "actionId\030\005 \001(\t\"\334\001\n\032CompactionCompletedRe" + + "quest\022\"\n\006ptinfo\030\001 \001(\0132\022.client.ProtoTInf" + + "o\022+\n\013credentials\030\002 \001(\0132\026.security.PCrede" + + "ntials\022\034\n\024externalCompactionId\030\003 \001(\t\022 \n\006" + + "extent\030\004 \001(\0132\020.data.PKeyExtent\022-\n\005stats\030" + + "\005 \001(\0132\036.tabletserver.PCompactionStats\"\342\001" + + "\n\035UpdateCompactionStatusRequest\022\"\n\006ptinf" + + "o\030\001 \001(\0132\022.client.ProtoTInfo\022+\n\013credentia" + + "ls\030\002 \001(\0132\026.security.PCredentials\022\034\n\024exte" + + "rnalCompactionId\030\003 \001(\t\022?\n\006status\030\004 \001(\0132/" + + ".compaction_coordinator.PCompactionStatu" + + "sUpdate\022\021\n\ttimestamp\030\005 \001(\003\"\252\001\n\027Compactio" + + "nFailedRequest\022\"\n\006ptinfo\030\001 \001(\0132\022.client." + + "ProtoTInfo\022+\n\013credentials\030\002 \001(\0132\026.securi" + + "ty.PCredentials\022\034\n\024externalCompactionId\030" + + "\003 \001(\t\022 \n\006extent\030\004 \001(\0132\020.data.PKeyExtent\"" + + "o\n\034GetRunningCompactionsRequest\022\"\n\006ptinf" + + "o\030\001 \001(\0132\022.client.ProtoTInfo\022+\n\013credentia" + + "ls\030\002 \001(\0132\026.security.PCredentials\"q\n\036GetC" + + "ompletedCompactionsRequest\022\"\n\006ptinfo\030\001 \001" + + "(\0132\022.client.ProtoTInfo\022+\n\013credentials\030\002 " + + "\001(\0132\026.security.PCredentials\"~\n\rCancelReq" + + "uest\022\"\n\006ptinfo\030\001 \001(\0132\022.client.ProtoTInfo" + + "\022+\n\013credentials\030\002 \001(\0132\026.security.PCreden" + + "tials\022\034\n\024externalCompactionId\030\003 \001(\t\"\232\002\n\023" + + "PExternalCompaction\022\021\n\tgroupName\030\001 \001(\t\022\021" + + "\n\tcompactor\030\002 \001(\t\022I\n\007updates\030\003 \003(\01328.com" + + "paction_coordinator.PExternalCompaction." + + "UpdatesEntry\0221\n\003job\030\004 \001(\0132$.tabletserver" + + ".PExternalCompactionJob\032_\n\014UpdatesEntry\022" + + "\013\n\003key\030\001 \001(\003\022>\n\005value\030\002 \001(\0132/.compaction" + + "_coordinator.PCompactionStatusUpdate:\0028\001" + + "\"\321\001\n\027PExternalCompactionList\022U\n\013compacti" + + "ons\030\001 \003(\0132@.compaction_coordinator.PExte" + + "rnalCompactionList.CompactionsEntry\032_\n\020C" + + "ompactionsEntry\022\013\n\003key\030\001 \001(\t\022:\n\005value\030\002 " + + "\001(\0132+.compaction_coordinator.PExternalCo" + + "mpaction:\0028\001\"_\n\022PNextCompactionJob\0221\n\003jo" + + "b\030\001 \001(\0132$.tabletserver.PExternalCompacti" + + "onJob\022\026\n\016compactorCount\030\002 \001(\005\"\312\001\n\027PCompa" + + "ctionStatusUpdate\0227\n\005state\030\001 \001(\0162(.compa" + + "ction_coordinator.PCompactionState\022\017\n\007me" + + "ssage\030\002 \001(\t\022\034\n\024entriesToBeCompacted\030\003 \001(" + + "\003\022\023\n\013entriesRead\030\004 \001(\003\022\026\n\016entriesWritten" + + "\030\005 \001(\003\022\032\n\022compactionAgeNanos\030\006 \001(\003*h\n\020PC" + + "ompactionState\022\014\n\010ASSIGNED\020\000\022\013\n\007STARTED\020" + + "\001\022\017\n\013IN_PROGRESS\020\002\022\r\n\tSUCCEEDED\020\003\022\n\n\006FAI" + + "LED\020\004\022\r\n\tCANCELLED\020\0052\203\006\n\034CompactionCoord" + + "inatorService\022l\n\020GetCompactionJob\022,.comp" + + "action_coordinator.CompactionJobRequest\032" + + "*.compaction_coordinator.PNextCompaction" + + "Job\022a\n\023CompactionCompleted\0222.compaction_" + + "coordinator.CompactionCompletedRequest\032\026" + + ".google.protobuf.Empty\022g\n\026UpdateCompacti" + + "onStatus\0225.compaction_coordinator.Update" + + "CompactionStatusRequest\032\026.google.protobu" + + "f.Empty\022[\n\020CompactionFailed\022/.compaction" + + "_coordinator.CompactionFailedRequest\032\026.g" + + "oogle.protobuf.Empty\022~\n\025GetRunningCompac" + + "tions\0224.compaction_coordinator.GetRunnin" + + "gCompactionsRequest\032/.compaction_coordin" + + "ator.PExternalCompactionList\022\202\001\n\027GetComp" + + "letedCompactions\0226.compaction_coordinato" + + "r.GetCompletedCompactionsRequest\032/.compa" + + "ction_coordinator.PExternalCompactionLis" + + "t\022G\n\006Cancel\022%.compaction_coordinator.Can" + + "celRequest\032\026.google.protobuf.EmptyBS\n,or" + + "g.apache.accumulo.grpc.compaction.protob" + + "ufB!CompactionCoordinatorServiceProtoP\001b" + + "\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + com.google.protobuf.EmptyProto.getDescriptor(), + org.apache.accumulo.grpc.compaction.protobuf.SecurityProto.getDescriptor(), + org.apache.accumulo.grpc.compaction.protobuf.ClientProto.getDescriptor(), + org.apache.accumulo.grpc.compaction.protobuf.DataProto.getDescriptor(), + org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.getDescriptor(), + org.apache.accumulo.grpc.compaction.protobuf.ManagerProto.getDescriptor(), + }); + internal_static_compaction_coordinator_CompactionJobRequest_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_compaction_coordinator_CompactionJobRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_CompactionJobRequest_descriptor, + new java.lang.String[] { "Ptinfo", "Credentials", "GroupName", "Compactor", "ExternalCompactionId", }); + internal_static_compaction_coordinator_CompactionCompletedRequest_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_compaction_coordinator_CompactionCompletedRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_CompactionCompletedRequest_descriptor, + new java.lang.String[] { "Ptinfo", "Credentials", "ExternalCompactionId", "Extent", "Stats", }); + internal_static_compaction_coordinator_UpdateCompactionStatusRequest_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_compaction_coordinator_UpdateCompactionStatusRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_UpdateCompactionStatusRequest_descriptor, + new java.lang.String[] { "Ptinfo", "Credentials", "ExternalCompactionId", "Status", "Timestamp", }); + internal_static_compaction_coordinator_CompactionFailedRequest_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_compaction_coordinator_CompactionFailedRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_CompactionFailedRequest_descriptor, + new java.lang.String[] { "Ptinfo", "Credentials", "ExternalCompactionId", "Extent", }); + internal_static_compaction_coordinator_GetRunningCompactionsRequest_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_compaction_coordinator_GetRunningCompactionsRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_GetRunningCompactionsRequest_descriptor, + new java.lang.String[] { "Ptinfo", "Credentials", }); + internal_static_compaction_coordinator_GetCompletedCompactionsRequest_descriptor = + getDescriptor().getMessageTypes().get(5); + internal_static_compaction_coordinator_GetCompletedCompactionsRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_GetCompletedCompactionsRequest_descriptor, + new java.lang.String[] { "Ptinfo", "Credentials", }); + internal_static_compaction_coordinator_CancelRequest_descriptor = + getDescriptor().getMessageTypes().get(6); + internal_static_compaction_coordinator_CancelRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_CancelRequest_descriptor, + new java.lang.String[] { "Ptinfo", "Credentials", "ExternalCompactionId", }); + internal_static_compaction_coordinator_PExternalCompaction_descriptor = + getDescriptor().getMessageTypes().get(7); + internal_static_compaction_coordinator_PExternalCompaction_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_PExternalCompaction_descriptor, + new java.lang.String[] { "GroupName", "Compactor", "Updates", "Job", }); + internal_static_compaction_coordinator_PExternalCompaction_UpdatesEntry_descriptor = + internal_static_compaction_coordinator_PExternalCompaction_descriptor.getNestedTypes().get(0); + internal_static_compaction_coordinator_PExternalCompaction_UpdatesEntry_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_PExternalCompaction_UpdatesEntry_descriptor, + new java.lang.String[] { "Key", "Value", }); + internal_static_compaction_coordinator_PExternalCompactionList_descriptor = + getDescriptor().getMessageTypes().get(8); + internal_static_compaction_coordinator_PExternalCompactionList_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_PExternalCompactionList_descriptor, + new java.lang.String[] { "Compactions", }); + internal_static_compaction_coordinator_PExternalCompactionList_CompactionsEntry_descriptor = + internal_static_compaction_coordinator_PExternalCompactionList_descriptor.getNestedTypes().get(0); + internal_static_compaction_coordinator_PExternalCompactionList_CompactionsEntry_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_PExternalCompactionList_CompactionsEntry_descriptor, + new java.lang.String[] { "Key", "Value", }); + internal_static_compaction_coordinator_PNextCompactionJob_descriptor = + getDescriptor().getMessageTypes().get(9); + internal_static_compaction_coordinator_PNextCompactionJob_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_PNextCompactionJob_descriptor, + new java.lang.String[] { "Job", "CompactorCount", }); + internal_static_compaction_coordinator_PCompactionStatusUpdate_descriptor = + getDescriptor().getMessageTypes().get(10); + internal_static_compaction_coordinator_PCompactionStatusUpdate_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_compaction_coordinator_PCompactionStatusUpdate_descriptor, + new java.lang.String[] { "State", "Message", "EntriesToBeCompacted", "EntriesRead", "EntriesWritten", "CompactionAgeNanos", }); + com.google.protobuf.EmptyProto.getDescriptor(); + org.apache.accumulo.grpc.compaction.protobuf.SecurityProto.getDescriptor(); + org.apache.accumulo.grpc.compaction.protobuf.ClientProto.getDescriptor(); + org.apache.accumulo.grpc.compaction.protobuf.DataProto.getDescriptor(); + org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.getDescriptor(); + org.apache.accumulo.grpc.compaction.protobuf.ManagerProto.getDescriptor(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionFailedRequest.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionFailedRequest.java new file mode 100644 index 00000000000..4a239ee7728 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionFailedRequest.java @@ -0,0 +1,1123 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.CompactionFailedRequest} + */ +public final class CompactionFailedRequest extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.CompactionFailedRequest) + CompactionFailedRequestOrBuilder { +private static final long serialVersionUID = 0L; + // Use CompactionFailedRequest.newBuilder() to construct. + private CompactionFailedRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private CompactionFailedRequest() { + externalCompactionId_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new CompactionFailedRequest(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionFailedRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionFailedRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest.class, org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest.Builder.class); + } + + private int bitField0_; + public static final int PTINFO_FIELD_NUMBER = 1; + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + @java.lang.Override + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + + public static final int CREDENTIALS_FIELD_NUMBER = 2; + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + @java.lang.Override + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + /** + * .security.PCredentials credentials = 2; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + + public static final int EXTERNALCOMPACTIONID_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object externalCompactionId_ = ""; + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + @java.lang.Override + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } + } + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int EXTENT_FIELD_NUMBER = 4; + private org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent extent_; + /** + * .data.PKeyExtent extent = 4; + * @return Whether the extent field is set. + */ + @java.lang.Override + public boolean hasExtent() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * .data.PKeyExtent extent = 4; + * @return The extent. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getExtent() { + return extent_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } + /** + * .data.PKeyExtent extent = 4; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder getExtentOrBuilder() { + return extent_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getCredentials()); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(externalCompactionId_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 3, externalCompactionId_); + } + if (((bitField0_ & 0x00000004) != 0)) { + output.writeMessage(4, getExtent()); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, getCredentials()); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(externalCompactionId_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, externalCompactionId_); + } + if (((bitField0_ & 0x00000004) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(4, getExtent()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest other = (org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest) obj; + + if (hasPtinfo() != other.hasPtinfo()) return false; + if (hasPtinfo()) { + if (!getPtinfo() + .equals(other.getPtinfo())) return false; + } + if (hasCredentials() != other.hasCredentials()) return false; + if (hasCredentials()) { + if (!getCredentials() + .equals(other.getCredentials())) return false; + } + if (!getExternalCompactionId() + .equals(other.getExternalCompactionId())) return false; + if (hasExtent() != other.hasExtent()) return false; + if (hasExtent()) { + if (!getExtent() + .equals(other.getExtent())) return false; + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasPtinfo()) { + hash = (37 * hash) + PTINFO_FIELD_NUMBER; + hash = (53 * hash) + getPtinfo().hashCode(); + } + if (hasCredentials()) { + hash = (37 * hash) + CREDENTIALS_FIELD_NUMBER; + hash = (53 * hash) + getCredentials().hashCode(); + } + hash = (37 * hash) + EXTERNALCOMPACTIONID_FIELD_NUMBER; + hash = (53 * hash) + getExternalCompactionId().hashCode(); + if (hasExtent()) { + hash = (37 * hash) + EXTENT_FIELD_NUMBER; + hash = (53 * hash) + getExtent().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.CompactionFailedRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.CompactionFailedRequest) + org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionFailedRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionFailedRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest.class, org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getPtinfoFieldBuilder(); + getCredentialsFieldBuilder(); + getExtentFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + externalCompactionId_ = ""; + extent_ = null; + if (extentBuilder_ != null) { + extentBuilder_.dispose(); + extentBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionFailedRequest_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest build() { + org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest result = new org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.ptinfo_ = ptinfoBuilder_ == null + ? ptinfo_ + : ptinfoBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.credentials_ = credentialsBuilder_ == null + ? credentials_ + : credentialsBuilder_.build(); + to_bitField0_ |= 0x00000002; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.externalCompactionId_ = externalCompactionId_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.extent_ = extentBuilder_ == null + ? extent_ + : extentBuilder_.build(); + to_bitField0_ |= 0x00000004; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest.getDefaultInstance()) return this; + if (other.hasPtinfo()) { + mergePtinfo(other.getPtinfo()); + } + if (other.hasCredentials()) { + mergeCredentials(other.getCredentials()); + } + if (!other.getExternalCompactionId().isEmpty()) { + externalCompactionId_ = other.externalCompactionId_; + bitField0_ |= 0x00000004; + onChanged(); + } + if (other.hasExtent()) { + mergeExtent(other.getExtent()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getPtinfoFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getCredentialsFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + externalCompactionId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + input.readMessage( + getExtentFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000008; + break; + } // case 34 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> ptinfoBuilder_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + if (ptinfoBuilder_ == null) { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } else { + return ptinfoBuilder_.getMessage(); + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ptinfo_ = value; + } else { + ptinfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo( + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder builderForValue) { + if (ptinfoBuilder_ == null) { + ptinfo_ = builderForValue.build(); + } else { + ptinfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder mergePtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) && + ptinfo_ != null && + ptinfo_ != org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance()) { + getPtinfoBuilder().mergeFrom(value); + } else { + ptinfo_ = value; + } + } else { + ptinfoBuilder_.mergeFrom(value); + } + if (ptinfo_ != null) { + bitField0_ |= 0x00000001; + onChanged(); + } + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder clearPtinfo() { + bitField0_ = (bitField0_ & ~0x00000001); + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder getPtinfoBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getPtinfoFieldBuilder().getBuilder(); + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + if (ptinfoBuilder_ != null) { + return ptinfoBuilder_.getMessageOrBuilder(); + } else { + return ptinfo_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> + getPtinfoFieldBuilder() { + if (ptinfoBuilder_ == null) { + ptinfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder>( + getPtinfo(), + getParentForChildren(), + isClean()); + ptinfo_ = null; + } + return ptinfoBuilder_; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> credentialsBuilder_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + if (credentialsBuilder_ == null) { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } else { + return credentialsBuilder_.getMessage(); + } + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + credentials_ = value; + } else { + credentialsBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials( + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder builderForValue) { + if (credentialsBuilder_ == null) { + credentials_ = builderForValue.build(); + } else { + credentialsBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder mergeCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) && + credentials_ != null && + credentials_ != org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance()) { + getCredentialsBuilder().mergeFrom(value); + } else { + credentials_ = value; + } + } else { + credentialsBuilder_.mergeFrom(value); + } + if (credentials_ != null) { + bitField0_ |= 0x00000002; + onChanged(); + } + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder clearCredentials() { + bitField0_ = (bitField0_ & ~0x00000002); + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder getCredentialsBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getCredentialsFieldBuilder().getBuilder(); + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + if (credentialsBuilder_ != null) { + return credentialsBuilder_.getMessageOrBuilder(); + } else { + return credentials_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + } + /** + * .security.PCredentials credentials = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> + getCredentialsFieldBuilder() { + if (credentialsBuilder_ == null) { + credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder>( + getCredentials(), + getParentForChildren(), + isClean()); + credentials_ = null; + } + return credentialsBuilder_; + } + + private java.lang.Object externalCompactionId_ = ""; + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string externalCompactionId = 3; + * @param value The externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + externalCompactionId_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string externalCompactionId = 3; + * @return This builder for chaining. + */ + public Builder clearExternalCompactionId() { + externalCompactionId_ = getDefaultInstance().getExternalCompactionId(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string externalCompactionId = 3; + * @param value The bytes for externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + externalCompactionId_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent extent_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder> extentBuilder_; + /** + * .data.PKeyExtent extent = 4; + * @return Whether the extent field is set. + */ + public boolean hasExtent() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * .data.PKeyExtent extent = 4; + * @return The extent. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getExtent() { + if (extentBuilder_ == null) { + return extent_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } else { + return extentBuilder_.getMessage(); + } + } + /** + * .data.PKeyExtent extent = 4; + */ + public Builder setExtent(org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent value) { + if (extentBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + extent_ = value; + } else { + extentBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * .data.PKeyExtent extent = 4; + */ + public Builder setExtent( + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder builderForValue) { + if (extentBuilder_ == null) { + extent_ = builderForValue.build(); + } else { + extentBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * .data.PKeyExtent extent = 4; + */ + public Builder mergeExtent(org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent value) { + if (extentBuilder_ == null) { + if (((bitField0_ & 0x00000008) != 0) && + extent_ != null && + extent_ != org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance()) { + getExtentBuilder().mergeFrom(value); + } else { + extent_ = value; + } + } else { + extentBuilder_.mergeFrom(value); + } + if (extent_ != null) { + bitField0_ |= 0x00000008; + onChanged(); + } + return this; + } + /** + * .data.PKeyExtent extent = 4; + */ + public Builder clearExtent() { + bitField0_ = (bitField0_ & ~0x00000008); + extent_ = null; + if (extentBuilder_ != null) { + extentBuilder_.dispose(); + extentBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .data.PKeyExtent extent = 4; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder getExtentBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getExtentFieldBuilder().getBuilder(); + } + /** + * .data.PKeyExtent extent = 4; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder getExtentOrBuilder() { + if (extentBuilder_ != null) { + return extentBuilder_.getMessageOrBuilder(); + } else { + return extent_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } + } + /** + * .data.PKeyExtent extent = 4; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder> + getExtentFieldBuilder() { + if (extentBuilder_ == null) { + extentBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder>( + getExtent(), + getParentForChildren(), + isClean()); + extent_ = null; + } + return extentBuilder_; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.CompactionFailedRequest) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.CompactionFailedRequest) + private static final org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public CompactionFailedRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionFailedRequestOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionFailedRequestOrBuilder.java new file mode 100644 index 00000000000..f8ae2df87d3 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionFailedRequestOrBuilder.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface CompactionFailedRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.CompactionFailedRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + boolean hasPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder(); + + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + boolean hasCredentials(); + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials(); + /** + * .security.PCredentials credentials = 2; + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder(); + + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + java.lang.String getExternalCompactionId(); + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + com.google.protobuf.ByteString + getExternalCompactionIdBytes(); + + /** + * .data.PKeyExtent extent = 4; + * @return Whether the extent field is set. + */ + boolean hasExtent(); + /** + * .data.PKeyExtent extent = 4; + * @return The extent. + */ + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getExtent(); + /** + * .data.PKeyExtent extent = 4; + */ + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder getExtentOrBuilder(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionJobRequest.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionJobRequest.java new file mode 100644 index 00000000000..78d2b933a8d --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionJobRequest.java @@ -0,0 +1,1210 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.CompactionJobRequest} + */ +public final class CompactionJobRequest extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.CompactionJobRequest) + CompactionJobRequestOrBuilder { +private static final long serialVersionUID = 0L; + // Use CompactionJobRequest.newBuilder() to construct. + private CompactionJobRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private CompactionJobRequest() { + groupName_ = ""; + compactor_ = ""; + externalCompactionId_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new CompactionJobRequest(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionJobRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionJobRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest.class, org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest.Builder.class); + } + + private int bitField0_; + public static final int PTINFO_FIELD_NUMBER = 1; + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + @java.lang.Override + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + + public static final int CREDENTIALS_FIELD_NUMBER = 2; + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + @java.lang.Override + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + /** + * .security.PCredentials credentials = 2; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + + public static final int GROUPNAME_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object groupName_ = ""; + /** + * string groupName = 3; + * @return The groupName. + */ + @java.lang.Override + public java.lang.String getGroupName() { + java.lang.Object ref = groupName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + groupName_ = s; + return s; + } + } + /** + * string groupName = 3; + * @return The bytes for groupName. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getGroupNameBytes() { + java.lang.Object ref = groupName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + groupName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int COMPACTOR_FIELD_NUMBER = 4; + @SuppressWarnings("serial") + private volatile java.lang.Object compactor_ = ""; + /** + * string compactor = 4; + * @return The compactor. + */ + @java.lang.Override + public java.lang.String getCompactor() { + java.lang.Object ref = compactor_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + compactor_ = s; + return s; + } + } + /** + * string compactor = 4; + * @return The bytes for compactor. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getCompactorBytes() { + java.lang.Object ref = compactor_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + compactor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int EXTERNALCOMPACTIONID_FIELD_NUMBER = 5; + @SuppressWarnings("serial") + private volatile java.lang.Object externalCompactionId_ = ""; + /** + * string externalCompactionId = 5; + * @return The externalCompactionId. + */ + @java.lang.Override + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } + } + /** + * string externalCompactionId = 5; + * @return The bytes for externalCompactionId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getCredentials()); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(groupName_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 3, groupName_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(compactor_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 4, compactor_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(externalCompactionId_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 5, externalCompactionId_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, getCredentials()); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(groupName_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, groupName_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(compactor_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, compactor_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(externalCompactionId_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, externalCompactionId_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest other = (org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest) obj; + + if (hasPtinfo() != other.hasPtinfo()) return false; + if (hasPtinfo()) { + if (!getPtinfo() + .equals(other.getPtinfo())) return false; + } + if (hasCredentials() != other.hasCredentials()) return false; + if (hasCredentials()) { + if (!getCredentials() + .equals(other.getCredentials())) return false; + } + if (!getGroupName() + .equals(other.getGroupName())) return false; + if (!getCompactor() + .equals(other.getCompactor())) return false; + if (!getExternalCompactionId() + .equals(other.getExternalCompactionId())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasPtinfo()) { + hash = (37 * hash) + PTINFO_FIELD_NUMBER; + hash = (53 * hash) + getPtinfo().hashCode(); + } + if (hasCredentials()) { + hash = (37 * hash) + CREDENTIALS_FIELD_NUMBER; + hash = (53 * hash) + getCredentials().hashCode(); + } + hash = (37 * hash) + GROUPNAME_FIELD_NUMBER; + hash = (53 * hash) + getGroupName().hashCode(); + hash = (37 * hash) + COMPACTOR_FIELD_NUMBER; + hash = (53 * hash) + getCompactor().hashCode(); + hash = (37 * hash) + EXTERNALCOMPACTIONID_FIELD_NUMBER; + hash = (53 * hash) + getExternalCompactionId().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.CompactionJobRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.CompactionJobRequest) + org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionJobRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionJobRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest.class, org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getPtinfoFieldBuilder(); + getCredentialsFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + groupName_ = ""; + compactor_ = ""; + externalCompactionId_ = ""; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_CompactionJobRequest_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest build() { + org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest result = new org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.ptinfo_ = ptinfoBuilder_ == null + ? ptinfo_ + : ptinfoBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.credentials_ = credentialsBuilder_ == null + ? credentials_ + : credentialsBuilder_.build(); + to_bitField0_ |= 0x00000002; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.groupName_ = groupName_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.compactor_ = compactor_; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.externalCompactionId_ = externalCompactionId_; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest.getDefaultInstance()) return this; + if (other.hasPtinfo()) { + mergePtinfo(other.getPtinfo()); + } + if (other.hasCredentials()) { + mergeCredentials(other.getCredentials()); + } + if (!other.getGroupName().isEmpty()) { + groupName_ = other.groupName_; + bitField0_ |= 0x00000004; + onChanged(); + } + if (!other.getCompactor().isEmpty()) { + compactor_ = other.compactor_; + bitField0_ |= 0x00000008; + onChanged(); + } + if (!other.getExternalCompactionId().isEmpty()) { + externalCompactionId_ = other.externalCompactionId_; + bitField0_ |= 0x00000010; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getPtinfoFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getCredentialsFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + groupName_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + compactor_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 42: { + externalCompactionId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000010; + break; + } // case 42 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> ptinfoBuilder_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + if (ptinfoBuilder_ == null) { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } else { + return ptinfoBuilder_.getMessage(); + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ptinfo_ = value; + } else { + ptinfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo( + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder builderForValue) { + if (ptinfoBuilder_ == null) { + ptinfo_ = builderForValue.build(); + } else { + ptinfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder mergePtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) && + ptinfo_ != null && + ptinfo_ != org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance()) { + getPtinfoBuilder().mergeFrom(value); + } else { + ptinfo_ = value; + } + } else { + ptinfoBuilder_.mergeFrom(value); + } + if (ptinfo_ != null) { + bitField0_ |= 0x00000001; + onChanged(); + } + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder clearPtinfo() { + bitField0_ = (bitField0_ & ~0x00000001); + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder getPtinfoBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getPtinfoFieldBuilder().getBuilder(); + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + if (ptinfoBuilder_ != null) { + return ptinfoBuilder_.getMessageOrBuilder(); + } else { + return ptinfo_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> + getPtinfoFieldBuilder() { + if (ptinfoBuilder_ == null) { + ptinfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder>( + getPtinfo(), + getParentForChildren(), + isClean()); + ptinfo_ = null; + } + return ptinfoBuilder_; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> credentialsBuilder_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + if (credentialsBuilder_ == null) { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } else { + return credentialsBuilder_.getMessage(); + } + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + credentials_ = value; + } else { + credentialsBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials( + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder builderForValue) { + if (credentialsBuilder_ == null) { + credentials_ = builderForValue.build(); + } else { + credentialsBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder mergeCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) && + credentials_ != null && + credentials_ != org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance()) { + getCredentialsBuilder().mergeFrom(value); + } else { + credentials_ = value; + } + } else { + credentialsBuilder_.mergeFrom(value); + } + if (credentials_ != null) { + bitField0_ |= 0x00000002; + onChanged(); + } + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder clearCredentials() { + bitField0_ = (bitField0_ & ~0x00000002); + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder getCredentialsBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getCredentialsFieldBuilder().getBuilder(); + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + if (credentialsBuilder_ != null) { + return credentialsBuilder_.getMessageOrBuilder(); + } else { + return credentials_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + } + /** + * .security.PCredentials credentials = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> + getCredentialsFieldBuilder() { + if (credentialsBuilder_ == null) { + credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder>( + getCredentials(), + getParentForChildren(), + isClean()); + credentials_ = null; + } + return credentialsBuilder_; + } + + private java.lang.Object groupName_ = ""; + /** + * string groupName = 3; + * @return The groupName. + */ + public java.lang.String getGroupName() { + java.lang.Object ref = groupName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + groupName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string groupName = 3; + * @return The bytes for groupName. + */ + public com.google.protobuf.ByteString + getGroupNameBytes() { + java.lang.Object ref = groupName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + groupName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string groupName = 3; + * @param value The groupName to set. + * @return This builder for chaining. + */ + public Builder setGroupName( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + groupName_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string groupName = 3; + * @return This builder for chaining. + */ + public Builder clearGroupName() { + groupName_ = getDefaultInstance().getGroupName(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string groupName = 3; + * @param value The bytes for groupName to set. + * @return This builder for chaining. + */ + public Builder setGroupNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + groupName_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private java.lang.Object compactor_ = ""; + /** + * string compactor = 4; + * @return The compactor. + */ + public java.lang.String getCompactor() { + java.lang.Object ref = compactor_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + compactor_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string compactor = 4; + * @return The bytes for compactor. + */ + public com.google.protobuf.ByteString + getCompactorBytes() { + java.lang.Object ref = compactor_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + compactor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string compactor = 4; + * @param value The compactor to set. + * @return This builder for chaining. + */ + public Builder setCompactor( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + compactor_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * string compactor = 4; + * @return This builder for chaining. + */ + public Builder clearCompactor() { + compactor_ = getDefaultInstance().getCompactor(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * string compactor = 4; + * @param value The bytes for compactor to set. + * @return This builder for chaining. + */ + public Builder setCompactorBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + compactor_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + + private java.lang.Object externalCompactionId_ = ""; + /** + * string externalCompactionId = 5; + * @return The externalCompactionId. + */ + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string externalCompactionId = 5; + * @return The bytes for externalCompactionId. + */ + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string externalCompactionId = 5; + * @param value The externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + externalCompactionId_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * string externalCompactionId = 5; + * @return This builder for chaining. + */ + public Builder clearExternalCompactionId() { + externalCompactionId_ = getDefaultInstance().getExternalCompactionId(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + return this; + } + /** + * string externalCompactionId = 5; + * @param value The bytes for externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + externalCompactionId_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.CompactionJobRequest) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.CompactionJobRequest) + private static final org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public CompactionJobRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionJobRequestOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionJobRequestOrBuilder.java new file mode 100644 index 00000000000..29e2366bf11 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/CompactionJobRequestOrBuilder.java @@ -0,0 +1,94 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface CompactionJobRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.CompactionJobRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + boolean hasPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder(); + + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + boolean hasCredentials(); + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials(); + /** + * .security.PCredentials credentials = 2; + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder(); + + /** + * string groupName = 3; + * @return The groupName. + */ + java.lang.String getGroupName(); + /** + * string groupName = 3; + * @return The bytes for groupName. + */ + com.google.protobuf.ByteString + getGroupNameBytes(); + + /** + * string compactor = 4; + * @return The compactor. + */ + java.lang.String getCompactor(); + /** + * string compactor = 4; + * @return The bytes for compactor. + */ + com.google.protobuf.ByteString + getCompactorBytes(); + + /** + * string externalCompactionId = 5; + * @return The externalCompactionId. + */ + java.lang.String getExternalCompactionId(); + /** + * string externalCompactionId = 5; + * @return The bytes for externalCompactionId. + */ + com.google.protobuf.ByteString + getExternalCompactionIdBytes(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/DataProto.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/DataProto.java new file mode 100644 index 00000000000..cc867bce374 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/DataProto.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: data.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public final class DataProto { + private DataProto() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_data_PKeyExtent_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_data_PKeyExtent_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\ndata.proto\022\004data\"c\n\nPKeyExtent\022\r\n\005tabl" + + "e\030\001 \001(\t\022\023\n\006endRow\030\002 \001(\014H\000\210\001\001\022\027\n\nprevEndR" + + "ow\030\003 \001(\014H\001\210\001\001B\t\n\007_endRowB\r\n\013_prevEndRowB" + + ";\n,org.apache.accumulo.grpc.compaction.p" + + "rotobufB\tDataProtoP\001b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_data_PKeyExtent_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_data_PKeyExtent_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_data_PKeyExtent_descriptor, + new java.lang.String[] { "Table", "EndRow", "PrevEndRow", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetCompletedCompactionsRequest.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetCompletedCompactionsRequest.java new file mode 100644 index 00000000000..99b0ec22a96 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetCompletedCompactionsRequest.java @@ -0,0 +1,802 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.GetCompletedCompactionsRequest} + */ +public final class GetCompletedCompactionsRequest extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.GetCompletedCompactionsRequest) + GetCompletedCompactionsRequestOrBuilder { +private static final long serialVersionUID = 0L; + // Use GetCompletedCompactionsRequest.newBuilder() to construct. + private GetCompletedCompactionsRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private GetCompletedCompactionsRequest() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new GetCompletedCompactionsRequest(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_GetCompletedCompactionsRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_GetCompletedCompactionsRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest.class, org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest.Builder.class); + } + + private int bitField0_; + public static final int PTINFO_FIELD_NUMBER = 1; + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + @java.lang.Override + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + + public static final int CREDENTIALS_FIELD_NUMBER = 2; + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + @java.lang.Override + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + /** + * .security.PCredentials credentials = 2; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getCredentials()); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, getCredentials()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest other = (org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest) obj; + + if (hasPtinfo() != other.hasPtinfo()) return false; + if (hasPtinfo()) { + if (!getPtinfo() + .equals(other.getPtinfo())) return false; + } + if (hasCredentials() != other.hasCredentials()) return false; + if (hasCredentials()) { + if (!getCredentials() + .equals(other.getCredentials())) return false; + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasPtinfo()) { + hash = (37 * hash) + PTINFO_FIELD_NUMBER; + hash = (53 * hash) + getPtinfo().hashCode(); + } + if (hasCredentials()) { + hash = (37 * hash) + CREDENTIALS_FIELD_NUMBER; + hash = (53 * hash) + getCredentials().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.GetCompletedCompactionsRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.GetCompletedCompactionsRequest) + org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_GetCompletedCompactionsRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_GetCompletedCompactionsRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest.class, org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getPtinfoFieldBuilder(); + getCredentialsFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_GetCompletedCompactionsRequest_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest build() { + org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest result = new org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.ptinfo_ = ptinfoBuilder_ == null + ? ptinfo_ + : ptinfoBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.credentials_ = credentialsBuilder_ == null + ? credentials_ + : credentialsBuilder_.build(); + to_bitField0_ |= 0x00000002; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest.getDefaultInstance()) return this; + if (other.hasPtinfo()) { + mergePtinfo(other.getPtinfo()); + } + if (other.hasCredentials()) { + mergeCredentials(other.getCredentials()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getPtinfoFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getCredentialsFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> ptinfoBuilder_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + if (ptinfoBuilder_ == null) { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } else { + return ptinfoBuilder_.getMessage(); + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ptinfo_ = value; + } else { + ptinfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo( + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder builderForValue) { + if (ptinfoBuilder_ == null) { + ptinfo_ = builderForValue.build(); + } else { + ptinfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder mergePtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) && + ptinfo_ != null && + ptinfo_ != org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance()) { + getPtinfoBuilder().mergeFrom(value); + } else { + ptinfo_ = value; + } + } else { + ptinfoBuilder_.mergeFrom(value); + } + if (ptinfo_ != null) { + bitField0_ |= 0x00000001; + onChanged(); + } + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder clearPtinfo() { + bitField0_ = (bitField0_ & ~0x00000001); + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder getPtinfoBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getPtinfoFieldBuilder().getBuilder(); + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + if (ptinfoBuilder_ != null) { + return ptinfoBuilder_.getMessageOrBuilder(); + } else { + return ptinfo_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> + getPtinfoFieldBuilder() { + if (ptinfoBuilder_ == null) { + ptinfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder>( + getPtinfo(), + getParentForChildren(), + isClean()); + ptinfo_ = null; + } + return ptinfoBuilder_; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> credentialsBuilder_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + if (credentialsBuilder_ == null) { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } else { + return credentialsBuilder_.getMessage(); + } + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + credentials_ = value; + } else { + credentialsBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials( + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder builderForValue) { + if (credentialsBuilder_ == null) { + credentials_ = builderForValue.build(); + } else { + credentialsBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder mergeCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) && + credentials_ != null && + credentials_ != org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance()) { + getCredentialsBuilder().mergeFrom(value); + } else { + credentials_ = value; + } + } else { + credentialsBuilder_.mergeFrom(value); + } + if (credentials_ != null) { + bitField0_ |= 0x00000002; + onChanged(); + } + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder clearCredentials() { + bitField0_ = (bitField0_ & ~0x00000002); + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder getCredentialsBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getCredentialsFieldBuilder().getBuilder(); + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + if (credentialsBuilder_ != null) { + return credentialsBuilder_.getMessageOrBuilder(); + } else { + return credentials_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + } + /** + * .security.PCredentials credentials = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> + getCredentialsFieldBuilder() { + if (credentialsBuilder_ == null) { + credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder>( + getCredentials(), + getParentForChildren(), + isClean()); + credentials_ = null; + } + return credentialsBuilder_; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.GetCompletedCompactionsRequest) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.GetCompletedCompactionsRequest) + private static final org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public GetCompletedCompactionsRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetCompletedCompactionsRequestOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetCompletedCompactionsRequestOrBuilder.java new file mode 100644 index 00000000000..6cde359d4a2 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetCompletedCompactionsRequestOrBuilder.java @@ -0,0 +1,58 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface GetCompletedCompactionsRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.GetCompletedCompactionsRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + boolean hasPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder(); + + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + boolean hasCredentials(); + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials(); + /** + * .security.PCredentials credentials = 2; + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetRunningCompactionsRequest.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetRunningCompactionsRequest.java new file mode 100644 index 00000000000..55f825bd2fa --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetRunningCompactionsRequest.java @@ -0,0 +1,802 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.GetRunningCompactionsRequest} + */ +public final class GetRunningCompactionsRequest extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.GetRunningCompactionsRequest) + GetRunningCompactionsRequestOrBuilder { +private static final long serialVersionUID = 0L; + // Use GetRunningCompactionsRequest.newBuilder() to construct. + private GetRunningCompactionsRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private GetRunningCompactionsRequest() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new GetRunningCompactionsRequest(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_GetRunningCompactionsRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_GetRunningCompactionsRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest.class, org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest.Builder.class); + } + + private int bitField0_; + public static final int PTINFO_FIELD_NUMBER = 1; + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + @java.lang.Override + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + + public static final int CREDENTIALS_FIELD_NUMBER = 2; + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + @java.lang.Override + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + /** + * .security.PCredentials credentials = 2; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getCredentials()); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, getCredentials()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest other = (org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest) obj; + + if (hasPtinfo() != other.hasPtinfo()) return false; + if (hasPtinfo()) { + if (!getPtinfo() + .equals(other.getPtinfo())) return false; + } + if (hasCredentials() != other.hasCredentials()) return false; + if (hasCredentials()) { + if (!getCredentials() + .equals(other.getCredentials())) return false; + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasPtinfo()) { + hash = (37 * hash) + PTINFO_FIELD_NUMBER; + hash = (53 * hash) + getPtinfo().hashCode(); + } + if (hasCredentials()) { + hash = (37 * hash) + CREDENTIALS_FIELD_NUMBER; + hash = (53 * hash) + getCredentials().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.GetRunningCompactionsRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.GetRunningCompactionsRequest) + org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_GetRunningCompactionsRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_GetRunningCompactionsRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest.class, org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getPtinfoFieldBuilder(); + getCredentialsFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_GetRunningCompactionsRequest_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest build() { + org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest result = new org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.ptinfo_ = ptinfoBuilder_ == null + ? ptinfo_ + : ptinfoBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.credentials_ = credentialsBuilder_ == null + ? credentials_ + : credentialsBuilder_.build(); + to_bitField0_ |= 0x00000002; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest.getDefaultInstance()) return this; + if (other.hasPtinfo()) { + mergePtinfo(other.getPtinfo()); + } + if (other.hasCredentials()) { + mergeCredentials(other.getCredentials()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getPtinfoFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getCredentialsFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> ptinfoBuilder_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + if (ptinfoBuilder_ == null) { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } else { + return ptinfoBuilder_.getMessage(); + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ptinfo_ = value; + } else { + ptinfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo( + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder builderForValue) { + if (ptinfoBuilder_ == null) { + ptinfo_ = builderForValue.build(); + } else { + ptinfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder mergePtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) && + ptinfo_ != null && + ptinfo_ != org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance()) { + getPtinfoBuilder().mergeFrom(value); + } else { + ptinfo_ = value; + } + } else { + ptinfoBuilder_.mergeFrom(value); + } + if (ptinfo_ != null) { + bitField0_ |= 0x00000001; + onChanged(); + } + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder clearPtinfo() { + bitField0_ = (bitField0_ & ~0x00000001); + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder getPtinfoBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getPtinfoFieldBuilder().getBuilder(); + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + if (ptinfoBuilder_ != null) { + return ptinfoBuilder_.getMessageOrBuilder(); + } else { + return ptinfo_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> + getPtinfoFieldBuilder() { + if (ptinfoBuilder_ == null) { + ptinfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder>( + getPtinfo(), + getParentForChildren(), + isClean()); + ptinfo_ = null; + } + return ptinfoBuilder_; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> credentialsBuilder_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + if (credentialsBuilder_ == null) { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } else { + return credentialsBuilder_.getMessage(); + } + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + credentials_ = value; + } else { + credentialsBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials( + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder builderForValue) { + if (credentialsBuilder_ == null) { + credentials_ = builderForValue.build(); + } else { + credentialsBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder mergeCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) && + credentials_ != null && + credentials_ != org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance()) { + getCredentialsBuilder().mergeFrom(value); + } else { + credentials_ = value; + } + } else { + credentialsBuilder_.mergeFrom(value); + } + if (credentials_ != null) { + bitField0_ |= 0x00000002; + onChanged(); + } + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder clearCredentials() { + bitField0_ = (bitField0_ & ~0x00000002); + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder getCredentialsBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getCredentialsFieldBuilder().getBuilder(); + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + if (credentialsBuilder_ != null) { + return credentialsBuilder_.getMessageOrBuilder(); + } else { + return credentials_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + } + /** + * .security.PCredentials credentials = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> + getCredentialsFieldBuilder() { + if (credentialsBuilder_ == null) { + credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder>( + getCredentials(), + getParentForChildren(), + isClean()); + credentials_ = null; + } + return credentialsBuilder_; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.GetRunningCompactionsRequest) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.GetRunningCompactionsRequest) + private static final org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public GetRunningCompactionsRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetRunningCompactionsRequestOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetRunningCompactionsRequestOrBuilder.java new file mode 100644 index 00000000000..d18486105e2 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/GetRunningCompactionsRequestOrBuilder.java @@ -0,0 +1,58 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface GetRunningCompactionsRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.GetRunningCompactionsRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + boolean hasPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder(); + + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + boolean hasCredentials(); + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials(); + /** + * .security.PCredentials credentials = 2; + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ManagerProto.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ManagerProto.java new file mode 100644 index 00000000000..89c3c782eb5 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ManagerProto.java @@ -0,0 +1,70 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: manager.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public final class ManagerProto { + private ManagerProto() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_manager_PFateId_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_manager_PFateId_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\rmanager.proto\022\007manager\"F\n\007PFateId\022(\n\004t" + + "ype\030\001 \001(\0162\032.manager.PFateInstanceType\022\021\n" + + "\ttxUUIDStr\030\002 \001(\t*4\n\021PFateInstanceType\022\013\n" + + "\007UNKNOWN\020\000\022\010\n\004META\020\001\022\010\n\004USER\020\002B>\n,org.ap" + + "ache.accumulo.grpc.compaction.protobufB\014" + + "ManagerProtoP\001b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_manager_PFateId_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_manager_PFateId_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_manager_PFateId_descriptor, + new java.lang.String[] { "Type", "TxUUIDStr", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionKind.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionKind.java new file mode 100644 index 00000000000..b3c06d6d6ac --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionKind.java @@ -0,0 +1,151 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf enum {@code tabletserver.PCompactionKind} + */ +public enum PCompactionKind + implements com.google.protobuf.ProtocolMessageEnum { + /** + *
+   * SELECTOR was removed and used to be zero
+   * However protobuf3 requires starting with 0 which is the default
+   * 
+ * + * UNKNOWN = 0; + */ + UNKNOWN(0), + /** + * SYSTEM = 1; + */ + SYSTEM(1), + /** + * USER = 2; + */ + USER(2), + UNRECOGNIZED(-1), + ; + + /** + *
+   * SELECTOR was removed and used to be zero
+   * However protobuf3 requires starting with 0 which is the default
+   * 
+ * + * UNKNOWN = 0; + */ + public static final int UNKNOWN_VALUE = 0; + /** + * SYSTEM = 1; + */ + public static final int SYSTEM_VALUE = 1; + /** + * USER = 2; + */ + public static final int USER_VALUE = 2; + + + public final int getNumber() { + if (this == UNRECOGNIZED) { + throw new java.lang.IllegalArgumentException( + "Can't get the number of an unknown enum value."); + } + return value; + } + + /** + * @param value The numeric wire value of the corresponding enum entry. + * @return The enum associated with the given numeric wire value. + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static PCompactionKind valueOf(int value) { + return forNumber(value); + } + + /** + * @param value The numeric wire value of the corresponding enum entry. + * @return The enum associated with the given numeric wire value. + */ + public static PCompactionKind forNumber(int value) { + switch (value) { + case 0: return UNKNOWN; + case 1: return SYSTEM; + case 2: return USER; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static final com.google.protobuf.Internal.EnumLiteMap< + PCompactionKind> internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public PCompactionKind findValueByNumber(int number) { + return PCompactionKind.forNumber(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + if (this == UNRECOGNIZED) { + throw new java.lang.IllegalStateException( + "Can't get the descriptor of an unrecognized enum value."); + } + return getDescriptor().getValues().get(ordinal()); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.getDescriptor().getEnumTypes().get(0); + } + + private static final PCompactionKind[] VALUES = values(); + + public static PCompactionKind valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + if (desc.getIndex() == -1) { + return UNRECOGNIZED; + } + return VALUES[desc.getIndex()]; + } + + private final int value; + + private PCompactionKind(int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:tabletserver.PCompactionKind) +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionState.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionState.java new file mode 100644 index 00000000000..e4aab748a6f --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionState.java @@ -0,0 +1,216 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf enum {@code compaction_coordinator.PCompactionState} + */ +public enum PCompactionState + implements com.google.protobuf.ProtocolMessageEnum { + /** + *
+   * Coordinator should set state to ASSIGNED when getCompactionJob is called by Compactor
+   * 
+ * + * ASSIGNED = 0; + */ + ASSIGNED(0), + /** + *
+   * Compactor should set state to STARTED when compaction has successfully begun
+   * 
+ * + * STARTED = 1; + */ + STARTED(1), + /** + *
+   * Compactor can call repeatedly with an updated message to reflect percentage complete
+   * 
+ * + * IN_PROGRESS = 2; + */ + IN_PROGRESS(2), + /** + *
+   * Compactor should set state to SUCCEEDED when compaction job has successfully finished
+   * 
+ * + * SUCCEEDED = 3; + */ + SUCCEEDED(3), + /** + *
+   * Compactor should set state to FAILED when compaction job fails, message should be mandatory
+   * 
+ * + * FAILED = 4; + */ + FAILED(4), + /** + *
+   * Compactor should set state to CANCELLED to acknowledge that it has stopped compacting
+   * 
+ * + * CANCELLED = 5; + */ + CANCELLED(5), + UNRECOGNIZED(-1), + ; + + /** + *
+   * Coordinator should set state to ASSIGNED when getCompactionJob is called by Compactor
+   * 
+ * + * ASSIGNED = 0; + */ + public static final int ASSIGNED_VALUE = 0; + /** + *
+   * Compactor should set state to STARTED when compaction has successfully begun
+   * 
+ * + * STARTED = 1; + */ + public static final int STARTED_VALUE = 1; + /** + *
+   * Compactor can call repeatedly with an updated message to reflect percentage complete
+   * 
+ * + * IN_PROGRESS = 2; + */ + public static final int IN_PROGRESS_VALUE = 2; + /** + *
+   * Compactor should set state to SUCCEEDED when compaction job has successfully finished
+   * 
+ * + * SUCCEEDED = 3; + */ + public static final int SUCCEEDED_VALUE = 3; + /** + *
+   * Compactor should set state to FAILED when compaction job fails, message should be mandatory
+   * 
+ * + * FAILED = 4; + */ + public static final int FAILED_VALUE = 4; + /** + *
+   * Compactor should set state to CANCELLED to acknowledge that it has stopped compacting
+   * 
+ * + * CANCELLED = 5; + */ + public static final int CANCELLED_VALUE = 5; + + + public final int getNumber() { + if (this == UNRECOGNIZED) { + throw new java.lang.IllegalArgumentException( + "Can't get the number of an unknown enum value."); + } + return value; + } + + /** + * @param value The numeric wire value of the corresponding enum entry. + * @return The enum associated with the given numeric wire value. + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static PCompactionState valueOf(int value) { + return forNumber(value); + } + + /** + * @param value The numeric wire value of the corresponding enum entry. + * @return The enum associated with the given numeric wire value. + */ + public static PCompactionState forNumber(int value) { + switch (value) { + case 0: return ASSIGNED; + case 1: return STARTED; + case 2: return IN_PROGRESS; + case 3: return SUCCEEDED; + case 4: return FAILED; + case 5: return CANCELLED; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static final com.google.protobuf.Internal.EnumLiteMap< + PCompactionState> internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public PCompactionState findValueByNumber(int number) { + return PCompactionState.forNumber(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + if (this == UNRECOGNIZED) { + throw new java.lang.IllegalStateException( + "Can't get the descriptor of an unrecognized enum value."); + } + return getDescriptor().getValues().get(ordinal()); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.getDescriptor().getEnumTypes().get(0); + } + + private static final PCompactionState[] VALUES = values(); + + public static PCompactionState valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + if (desc.getIndex() == -1) { + return UNRECOGNIZED; + } + return VALUES[desc.getIndex()]; + } + + private final int value; + + private PCompactionState(int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:compaction_coordinator.PCompactionState) +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStats.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStats.java new file mode 100644 index 00000000000..9098338c995 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStats.java @@ -0,0 +1,625 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code tabletserver.PCompactionStats} + */ +public final class PCompactionStats extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:tabletserver.PCompactionStats) + PCompactionStatsOrBuilder { +private static final long serialVersionUID = 0L; + // Use PCompactionStats.newBuilder() to construct. + private PCompactionStats(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PCompactionStats() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PCompactionStats(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PCompactionStats_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PCompactionStats_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.class, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.Builder.class); + } + + public static final int ENTRIESREAD_FIELD_NUMBER = 1; + private long entriesRead_ = 0L; + /** + * int64 entriesRead = 1; + * @return The entriesRead. + */ + @java.lang.Override + public long getEntriesRead() { + return entriesRead_; + } + + public static final int ENTRIESWRITTEN_FIELD_NUMBER = 2; + private long entriesWritten_ = 0L; + /** + * int64 entriesWritten = 2; + * @return The entriesWritten. + */ + @java.lang.Override + public long getEntriesWritten() { + return entriesWritten_; + } + + public static final int FILESIZE_FIELD_NUMBER = 3; + private long fileSize_ = 0L; + /** + * int64 fileSize = 3; + * @return The fileSize. + */ + @java.lang.Override + public long getFileSize() { + return fileSize_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (entriesRead_ != 0L) { + output.writeInt64(1, entriesRead_); + } + if (entriesWritten_ != 0L) { + output.writeInt64(2, entriesWritten_); + } + if (fileSize_ != 0L) { + output.writeInt64(3, fileSize_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (entriesRead_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(1, entriesRead_); + } + if (entriesWritten_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(2, entriesWritten_); + } + if (fileSize_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(3, fileSize_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats other = (org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats) obj; + + if (getEntriesRead() + != other.getEntriesRead()) return false; + if (getEntriesWritten() + != other.getEntriesWritten()) return false; + if (getFileSize() + != other.getFileSize()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + ENTRIESREAD_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getEntriesRead()); + hash = (37 * hash) + ENTRIESWRITTEN_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getEntriesWritten()); + hash = (37 * hash) + FILESIZE_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getFileSize()); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code tabletserver.PCompactionStats} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:tabletserver.PCompactionStats) + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatsOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PCompactionStats_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PCompactionStats_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.class, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + entriesRead_ = 0L; + entriesWritten_ = 0L; + fileSize_ = 0L; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PCompactionStats_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats build() { + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats result = new org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.entriesRead_ = entriesRead_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.entriesWritten_ = entriesWritten_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.fileSize_ = fileSize_; + } + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats.getDefaultInstance()) return this; + if (other.getEntriesRead() != 0L) { + setEntriesRead(other.getEntriesRead()); + } + if (other.getEntriesWritten() != 0L) { + setEntriesWritten(other.getEntriesWritten()); + } + if (other.getFileSize() != 0L) { + setFileSize(other.getFileSize()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + entriesRead_ = input.readInt64(); + bitField0_ |= 0x00000001; + break; + } // case 8 + case 16: { + entriesWritten_ = input.readInt64(); + bitField0_ |= 0x00000002; + break; + } // case 16 + case 24: { + fileSize_ = input.readInt64(); + bitField0_ |= 0x00000004; + break; + } // case 24 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private long entriesRead_ ; + /** + * int64 entriesRead = 1; + * @return The entriesRead. + */ + @java.lang.Override + public long getEntriesRead() { + return entriesRead_; + } + /** + * int64 entriesRead = 1; + * @param value The entriesRead to set. + * @return This builder for chaining. + */ + public Builder setEntriesRead(long value) { + + entriesRead_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * int64 entriesRead = 1; + * @return This builder for chaining. + */ + public Builder clearEntriesRead() { + bitField0_ = (bitField0_ & ~0x00000001); + entriesRead_ = 0L; + onChanged(); + return this; + } + + private long entriesWritten_ ; + /** + * int64 entriesWritten = 2; + * @return The entriesWritten. + */ + @java.lang.Override + public long getEntriesWritten() { + return entriesWritten_; + } + /** + * int64 entriesWritten = 2; + * @param value The entriesWritten to set. + * @return This builder for chaining. + */ + public Builder setEntriesWritten(long value) { + + entriesWritten_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * int64 entriesWritten = 2; + * @return This builder for chaining. + */ + public Builder clearEntriesWritten() { + bitField0_ = (bitField0_ & ~0x00000002); + entriesWritten_ = 0L; + onChanged(); + return this; + } + + private long fileSize_ ; + /** + * int64 fileSize = 3; + * @return The fileSize. + */ + @java.lang.Override + public long getFileSize() { + return fileSize_; + } + /** + * int64 fileSize = 3; + * @param value The fileSize to set. + * @return This builder for chaining. + */ + public Builder setFileSize(long value) { + + fileSize_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * int64 fileSize = 3; + * @return This builder for chaining. + */ + public Builder clearFileSize() { + bitField0_ = (bitField0_ & ~0x00000004); + fileSize_ = 0L; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:tabletserver.PCompactionStats) + } + + // @@protoc_insertion_point(class_scope:tabletserver.PCompactionStats) + private static final org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PCompactionStats parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStatsOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStatsOrBuilder.java new file mode 100644 index 00000000000..cff10a214e0 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStatsOrBuilder.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PCompactionStatsOrBuilder extends + // @@protoc_insertion_point(interface_extends:tabletserver.PCompactionStats) + com.google.protobuf.MessageOrBuilder { + + /** + * int64 entriesRead = 1; + * @return The entriesRead. + */ + long getEntriesRead(); + + /** + * int64 entriesWritten = 2; + * @return The entriesWritten. + */ + long getEntriesWritten(); + + /** + * int64 fileSize = 3; + * @return The fileSize. + */ + long getFileSize(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStatusUpdate.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStatusUpdate.java new file mode 100644 index 00000000000..e17858593e5 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStatusUpdate.java @@ -0,0 +1,922 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.PCompactionStatusUpdate} + */ +public final class PCompactionStatusUpdate extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.PCompactionStatusUpdate) + PCompactionStatusUpdateOrBuilder { +private static final long serialVersionUID = 0L; + // Use PCompactionStatusUpdate.newBuilder() to construct. + private PCompactionStatusUpdate(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PCompactionStatusUpdate() { + state_ = 0; + message_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PCompactionStatusUpdate(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PCompactionStatusUpdate_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PCompactionStatusUpdate_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.class, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder.class); + } + + public static final int STATE_FIELD_NUMBER = 1; + private int state_ = 0; + /** + * .compaction_coordinator.PCompactionState state = 1; + * @return The enum numeric value on the wire for state. + */ + @java.lang.Override public int getStateValue() { + return state_; + } + /** + * .compaction_coordinator.PCompactionState state = 1; + * @return The state. + */ + @java.lang.Override public org.apache.accumulo.grpc.compaction.protobuf.PCompactionState getState() { + org.apache.accumulo.grpc.compaction.protobuf.PCompactionState result = org.apache.accumulo.grpc.compaction.protobuf.PCompactionState.forNumber(state_); + return result == null ? org.apache.accumulo.grpc.compaction.protobuf.PCompactionState.UNRECOGNIZED : result; + } + + public static final int MESSAGE_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object message_ = ""; + /** + * string message = 2; + * @return The message. + */ + @java.lang.Override + public java.lang.String getMessage() { + java.lang.Object ref = message_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + message_ = s; + return s; + } + } + /** + * string message = 2; + * @return The bytes for message. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getMessageBytes() { + java.lang.Object ref = message_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + message_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int ENTRIESTOBECOMPACTED_FIELD_NUMBER = 3; + private long entriesToBeCompacted_ = 0L; + /** + * int64 entriesToBeCompacted = 3; + * @return The entriesToBeCompacted. + */ + @java.lang.Override + public long getEntriesToBeCompacted() { + return entriesToBeCompacted_; + } + + public static final int ENTRIESREAD_FIELD_NUMBER = 4; + private long entriesRead_ = 0L; + /** + * int64 entriesRead = 4; + * @return The entriesRead. + */ + @java.lang.Override + public long getEntriesRead() { + return entriesRead_; + } + + public static final int ENTRIESWRITTEN_FIELD_NUMBER = 5; + private long entriesWritten_ = 0L; + /** + * int64 entriesWritten = 5; + * @return The entriesWritten. + */ + @java.lang.Override + public long getEntriesWritten() { + return entriesWritten_; + } + + public static final int COMPACTIONAGENANOS_FIELD_NUMBER = 6; + private long compactionAgeNanos_ = 0L; + /** + * int64 compactionAgeNanos = 6; + * @return The compactionAgeNanos. + */ + @java.lang.Override + public long getCompactionAgeNanos() { + return compactionAgeNanos_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (state_ != org.apache.accumulo.grpc.compaction.protobuf.PCompactionState.ASSIGNED.getNumber()) { + output.writeEnum(1, state_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(message_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, message_); + } + if (entriesToBeCompacted_ != 0L) { + output.writeInt64(3, entriesToBeCompacted_); + } + if (entriesRead_ != 0L) { + output.writeInt64(4, entriesRead_); + } + if (entriesWritten_ != 0L) { + output.writeInt64(5, entriesWritten_); + } + if (compactionAgeNanos_ != 0L) { + output.writeInt64(6, compactionAgeNanos_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (state_ != org.apache.accumulo.grpc.compaction.protobuf.PCompactionState.ASSIGNED.getNumber()) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(1, state_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(message_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, message_); + } + if (entriesToBeCompacted_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(3, entriesToBeCompacted_); + } + if (entriesRead_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(4, entriesRead_); + } + if (entriesWritten_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(5, entriesWritten_); + } + if (compactionAgeNanos_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(6, compactionAgeNanos_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate other = (org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate) obj; + + if (state_ != other.state_) return false; + if (!getMessage() + .equals(other.getMessage())) return false; + if (getEntriesToBeCompacted() + != other.getEntriesToBeCompacted()) return false; + if (getEntriesRead() + != other.getEntriesRead()) return false; + if (getEntriesWritten() + != other.getEntriesWritten()) return false; + if (getCompactionAgeNanos() + != other.getCompactionAgeNanos()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + STATE_FIELD_NUMBER; + hash = (53 * hash) + state_; + hash = (37 * hash) + MESSAGE_FIELD_NUMBER; + hash = (53 * hash) + getMessage().hashCode(); + hash = (37 * hash) + ENTRIESTOBECOMPACTED_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getEntriesToBeCompacted()); + hash = (37 * hash) + ENTRIESREAD_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getEntriesRead()); + hash = (37 * hash) + ENTRIESWRITTEN_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getEntriesWritten()); + hash = (37 * hash) + COMPACTIONAGENANOS_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getCompactionAgeNanos()); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.PCompactionStatusUpdate} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.PCompactionStatusUpdate) + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdateOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PCompactionStatusUpdate_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PCompactionStatusUpdate_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.class, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + state_ = 0; + message_ = ""; + entriesToBeCompacted_ = 0L; + entriesRead_ = 0L; + entriesWritten_ = 0L; + compactionAgeNanos_ = 0L; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PCompactionStatusUpdate_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate build() { + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate result = new org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.state_ = state_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.message_ = message_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.entriesToBeCompacted_ = entriesToBeCompacted_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.entriesRead_ = entriesRead_; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.entriesWritten_ = entriesWritten_; + } + if (((from_bitField0_ & 0x00000020) != 0)) { + result.compactionAgeNanos_ = compactionAgeNanos_; + } + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.getDefaultInstance()) return this; + if (other.state_ != 0) { + setStateValue(other.getStateValue()); + } + if (!other.getMessage().isEmpty()) { + message_ = other.message_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (other.getEntriesToBeCompacted() != 0L) { + setEntriesToBeCompacted(other.getEntriesToBeCompacted()); + } + if (other.getEntriesRead() != 0L) { + setEntriesRead(other.getEntriesRead()); + } + if (other.getEntriesWritten() != 0L) { + setEntriesWritten(other.getEntriesWritten()); + } + if (other.getCompactionAgeNanos() != 0L) { + setCompactionAgeNanos(other.getCompactionAgeNanos()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + state_ = input.readEnum(); + bitField0_ |= 0x00000001; + break; + } // case 8 + case 18: { + message_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + entriesToBeCompacted_ = input.readInt64(); + bitField0_ |= 0x00000004; + break; + } // case 24 + case 32: { + entriesRead_ = input.readInt64(); + bitField0_ |= 0x00000008; + break; + } // case 32 + case 40: { + entriesWritten_ = input.readInt64(); + bitField0_ |= 0x00000010; + break; + } // case 40 + case 48: { + compactionAgeNanos_ = input.readInt64(); + bitField0_ |= 0x00000020; + break; + } // case 48 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private int state_ = 0; + /** + * .compaction_coordinator.PCompactionState state = 1; + * @return The enum numeric value on the wire for state. + */ + @java.lang.Override public int getStateValue() { + return state_; + } + /** + * .compaction_coordinator.PCompactionState state = 1; + * @param value The enum numeric value on the wire for state to set. + * @return This builder for chaining. + */ + public Builder setStateValue(int value) { + state_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .compaction_coordinator.PCompactionState state = 1; + * @return The state. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionState getState() { + org.apache.accumulo.grpc.compaction.protobuf.PCompactionState result = org.apache.accumulo.grpc.compaction.protobuf.PCompactionState.forNumber(state_); + return result == null ? org.apache.accumulo.grpc.compaction.protobuf.PCompactionState.UNRECOGNIZED : result; + } + /** + * .compaction_coordinator.PCompactionState state = 1; + * @param value The state to set. + * @return This builder for chaining. + */ + public Builder setState(org.apache.accumulo.grpc.compaction.protobuf.PCompactionState value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + state_ = value.getNumber(); + onChanged(); + return this; + } + /** + * .compaction_coordinator.PCompactionState state = 1; + * @return This builder for chaining. + */ + public Builder clearState() { + bitField0_ = (bitField0_ & ~0x00000001); + state_ = 0; + onChanged(); + return this; + } + + private java.lang.Object message_ = ""; + /** + * string message = 2; + * @return The message. + */ + public java.lang.String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + message_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string message = 2; + * @return The bytes for message. + */ + public com.google.protobuf.ByteString + getMessageBytes() { + java.lang.Object ref = message_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + message_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string message = 2; + * @param value The message to set. + * @return This builder for chaining. + */ + public Builder setMessage( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + message_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string message = 2; + * @return This builder for chaining. + */ + public Builder clearMessage() { + message_ = getDefaultInstance().getMessage(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string message = 2; + * @param value The bytes for message to set. + * @return This builder for chaining. + */ + public Builder setMessageBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + message_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private long entriesToBeCompacted_ ; + /** + * int64 entriesToBeCompacted = 3; + * @return The entriesToBeCompacted. + */ + @java.lang.Override + public long getEntriesToBeCompacted() { + return entriesToBeCompacted_; + } + /** + * int64 entriesToBeCompacted = 3; + * @param value The entriesToBeCompacted to set. + * @return This builder for chaining. + */ + public Builder setEntriesToBeCompacted(long value) { + + entriesToBeCompacted_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * int64 entriesToBeCompacted = 3; + * @return This builder for chaining. + */ + public Builder clearEntriesToBeCompacted() { + bitField0_ = (bitField0_ & ~0x00000004); + entriesToBeCompacted_ = 0L; + onChanged(); + return this; + } + + private long entriesRead_ ; + /** + * int64 entriesRead = 4; + * @return The entriesRead. + */ + @java.lang.Override + public long getEntriesRead() { + return entriesRead_; + } + /** + * int64 entriesRead = 4; + * @param value The entriesRead to set. + * @return This builder for chaining. + */ + public Builder setEntriesRead(long value) { + + entriesRead_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * int64 entriesRead = 4; + * @return This builder for chaining. + */ + public Builder clearEntriesRead() { + bitField0_ = (bitField0_ & ~0x00000008); + entriesRead_ = 0L; + onChanged(); + return this; + } + + private long entriesWritten_ ; + /** + * int64 entriesWritten = 5; + * @return The entriesWritten. + */ + @java.lang.Override + public long getEntriesWritten() { + return entriesWritten_; + } + /** + * int64 entriesWritten = 5; + * @param value The entriesWritten to set. + * @return This builder for chaining. + */ + public Builder setEntriesWritten(long value) { + + entriesWritten_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * int64 entriesWritten = 5; + * @return This builder for chaining. + */ + public Builder clearEntriesWritten() { + bitField0_ = (bitField0_ & ~0x00000010); + entriesWritten_ = 0L; + onChanged(); + return this; + } + + private long compactionAgeNanos_ ; + /** + * int64 compactionAgeNanos = 6; + * @return The compactionAgeNanos. + */ + @java.lang.Override + public long getCompactionAgeNanos() { + return compactionAgeNanos_; + } + /** + * int64 compactionAgeNanos = 6; + * @param value The compactionAgeNanos to set. + * @return This builder for chaining. + */ + public Builder setCompactionAgeNanos(long value) { + + compactionAgeNanos_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + /** + * int64 compactionAgeNanos = 6; + * @return This builder for chaining. + */ + public Builder clearCompactionAgeNanos() { + bitField0_ = (bitField0_ & ~0x00000020); + compactionAgeNanos_ = 0L; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.PCompactionStatusUpdate) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.PCompactionStatusUpdate) + private static final org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PCompactionStatusUpdate parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStatusUpdateOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStatusUpdateOrBuilder.java new file mode 100644 index 00000000000..b01dd1382c0 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCompactionStatusUpdateOrBuilder.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PCompactionStatusUpdateOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.PCompactionStatusUpdate) + com.google.protobuf.MessageOrBuilder { + + /** + * .compaction_coordinator.PCompactionState state = 1; + * @return The enum numeric value on the wire for state. + */ + int getStateValue(); + /** + * .compaction_coordinator.PCompactionState state = 1; + * @return The state. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCompactionState getState(); + + /** + * string message = 2; + * @return The message. + */ + java.lang.String getMessage(); + /** + * string message = 2; + * @return The bytes for message. + */ + com.google.protobuf.ByteString + getMessageBytes(); + + /** + * int64 entriesToBeCompacted = 3; + * @return The entriesToBeCompacted. + */ + long getEntriesToBeCompacted(); + + /** + * int64 entriesRead = 4; + * @return The entriesRead. + */ + long getEntriesRead(); + + /** + * int64 entriesWritten = 5; + * @return The entriesWritten. + */ + long getEntriesWritten(); + + /** + * int64 compactionAgeNanos = 6; + * @return The compactionAgeNanos. + */ + long getCompactionAgeNanos(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCredentials.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCredentials.java new file mode 100644 index 00000000000..bbea49564d2 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCredentials.java @@ -0,0 +1,1001 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: security.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + *
+ * There are no nulls with Protobuf3, so the default will be an empty string
+ * Our TCredentials version with thrift currently is using/checking for null so
+ * using the optional field will generate "has" methods which we can use to
+ * check if the value was set
+ * 
+ * + * Protobuf type {@code security.PCredentials} + */ +public final class PCredentials extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:security.PCredentials) + PCredentialsOrBuilder { +private static final long serialVersionUID = 0L; + // Use PCredentials.newBuilder() to construct. + private PCredentials(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PCredentials() { + principal_ = ""; + tokenClassName_ = ""; + token_ = com.google.protobuf.ByteString.EMPTY; + instanceId_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PCredentials(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.SecurityProto.internal_static_security_PCredentials_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.SecurityProto.internal_static_security_PCredentials_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.class, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder.class); + } + + private int bitField0_; + public static final int PRINCIPAL_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object principal_ = ""; + /** + * optional string principal = 1; + * @return Whether the principal field is set. + */ + @java.lang.Override + public boolean hasPrincipal() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * optional string principal = 1; + * @return The principal. + */ + @java.lang.Override + public java.lang.String getPrincipal() { + java.lang.Object ref = principal_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + principal_ = s; + return s; + } + } + /** + * optional string principal = 1; + * @return The bytes for principal. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getPrincipalBytes() { + java.lang.Object ref = principal_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + principal_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TOKENCLASSNAME_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object tokenClassName_ = ""; + /** + * optional string tokenClassName = 2; + * @return Whether the tokenClassName field is set. + */ + @java.lang.Override + public boolean hasTokenClassName() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * optional string tokenClassName = 2; + * @return The tokenClassName. + */ + @java.lang.Override + public java.lang.String getTokenClassName() { + java.lang.Object ref = tokenClassName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + tokenClassName_ = s; + return s; + } + } + /** + * optional string tokenClassName = 2; + * @return The bytes for tokenClassName. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getTokenClassNameBytes() { + java.lang.Object ref = tokenClassName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + tokenClassName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int TOKEN_FIELD_NUMBER = 3; + private com.google.protobuf.ByteString token_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes token = 3; + * @return Whether the token field is set. + */ + @java.lang.Override + public boolean hasToken() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * optional bytes token = 3; + * @return The token. + */ + @java.lang.Override + public com.google.protobuf.ByteString getToken() { + return token_; + } + + public static final int INSTANCEID_FIELD_NUMBER = 4; + @SuppressWarnings("serial") + private volatile java.lang.Object instanceId_ = ""; + /** + * optional string instanceId = 4; + * @return Whether the instanceId field is set. + */ + @java.lang.Override + public boolean hasInstanceId() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * optional string instanceId = 4; + * @return The instanceId. + */ + @java.lang.Override + public java.lang.String getInstanceId() { + java.lang.Object ref = instanceId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + instanceId_ = s; + return s; + } + } + /** + * optional string instanceId = 4; + * @return The bytes for instanceId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getInstanceIdBytes() { + java.lang.Object ref = instanceId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + instanceId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, principal_); + } + if (((bitField0_ & 0x00000002) != 0)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, tokenClassName_); + } + if (((bitField0_ & 0x00000004) != 0)) { + output.writeBytes(3, token_); + } + if (((bitField0_ & 0x00000008) != 0)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 4, instanceId_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, principal_); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, tokenClassName_); + } + if (((bitField0_ & 0x00000004) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, token_); + } + if (((bitField0_ & 0x00000008) != 0)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, instanceId_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PCredentials)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PCredentials other = (org.apache.accumulo.grpc.compaction.protobuf.PCredentials) obj; + + if (hasPrincipal() != other.hasPrincipal()) return false; + if (hasPrincipal()) { + if (!getPrincipal() + .equals(other.getPrincipal())) return false; + } + if (hasTokenClassName() != other.hasTokenClassName()) return false; + if (hasTokenClassName()) { + if (!getTokenClassName() + .equals(other.getTokenClassName())) return false; + } + if (hasToken() != other.hasToken()) return false; + if (hasToken()) { + if (!getToken() + .equals(other.getToken())) return false; + } + if (hasInstanceId() != other.hasInstanceId()) return false; + if (hasInstanceId()) { + if (!getInstanceId() + .equals(other.getInstanceId())) return false; + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasPrincipal()) { + hash = (37 * hash) + PRINCIPAL_FIELD_NUMBER; + hash = (53 * hash) + getPrincipal().hashCode(); + } + if (hasTokenClassName()) { + hash = (37 * hash) + TOKENCLASSNAME_FIELD_NUMBER; + hash = (53 * hash) + getTokenClassName().hashCode(); + } + if (hasToken()) { + hash = (37 * hash) + TOKEN_FIELD_NUMBER; + hash = (53 * hash) + getToken().hashCode(); + } + if (hasInstanceId()) { + hash = (37 * hash) + INSTANCEID_FIELD_NUMBER; + hash = (53 * hash) + getInstanceId().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PCredentials prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + *
+   * There are no nulls with Protobuf3, so the default will be an empty string
+   * Our TCredentials version with thrift currently is using/checking for null so
+   * using the optional field will generate "has" methods which we can use to
+   * check if the value was set
+   * 
+ * + * Protobuf type {@code security.PCredentials} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:security.PCredentials) + org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.SecurityProto.internal_static_security_PCredentials_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.SecurityProto.internal_static_security_PCredentials_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.class, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PCredentials.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + principal_ = ""; + tokenClassName_ = ""; + token_ = com.google.protobuf.ByteString.EMPTY; + instanceId_ = ""; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.SecurityProto.internal_static_security_PCredentials_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials build() { + org.apache.accumulo.grpc.compaction.protobuf.PCredentials result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PCredentials result = new org.apache.accumulo.grpc.compaction.protobuf.PCredentials(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PCredentials result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.principal_ = principal_; + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.tokenClassName_ = tokenClassName_; + to_bitField0_ |= 0x00000002; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.token_ = token_; + to_bitField0_ |= 0x00000004; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.instanceId_ = instanceId_; + to_bitField0_ |= 0x00000008; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PCredentials) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PCredentials)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PCredentials other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance()) return this; + if (other.hasPrincipal()) { + principal_ = other.principal_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (other.hasTokenClassName()) { + tokenClassName_ = other.tokenClassName_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (other.hasToken()) { + setToken(other.getToken()); + } + if (other.hasInstanceId()) { + instanceId_ = other.instanceId_; + bitField0_ |= 0x00000008; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + principal_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + tokenClassName_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + token_ = input.readBytes(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + instanceId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000008; + break; + } // case 34 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object principal_ = ""; + /** + * optional string principal = 1; + * @return Whether the principal field is set. + */ + public boolean hasPrincipal() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * optional string principal = 1; + * @return The principal. + */ + public java.lang.String getPrincipal() { + java.lang.Object ref = principal_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + principal_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string principal = 1; + * @return The bytes for principal. + */ + public com.google.protobuf.ByteString + getPrincipalBytes() { + java.lang.Object ref = principal_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + principal_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * optional string principal = 1; + * @param value The principal to set. + * @return This builder for chaining. + */ + public Builder setPrincipal( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + principal_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * optional string principal = 1; + * @return This builder for chaining. + */ + public Builder clearPrincipal() { + principal_ = getDefaultInstance().getPrincipal(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * optional string principal = 1; + * @param value The bytes for principal to set. + * @return This builder for chaining. + */ + public Builder setPrincipalBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + principal_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private java.lang.Object tokenClassName_ = ""; + /** + * optional string tokenClassName = 2; + * @return Whether the tokenClassName field is set. + */ + public boolean hasTokenClassName() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * optional string tokenClassName = 2; + * @return The tokenClassName. + */ + public java.lang.String getTokenClassName() { + java.lang.Object ref = tokenClassName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + tokenClassName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string tokenClassName = 2; + * @return The bytes for tokenClassName. + */ + public com.google.protobuf.ByteString + getTokenClassNameBytes() { + java.lang.Object ref = tokenClassName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + tokenClassName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * optional string tokenClassName = 2; + * @param value The tokenClassName to set. + * @return This builder for chaining. + */ + public Builder setTokenClassName( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + tokenClassName_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * optional string tokenClassName = 2; + * @return This builder for chaining. + */ + public Builder clearTokenClassName() { + tokenClassName_ = getDefaultInstance().getTokenClassName(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * optional string tokenClassName = 2; + * @param value The bytes for tokenClassName to set. + * @return This builder for chaining. + */ + public Builder setTokenClassNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + tokenClassName_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private com.google.protobuf.ByteString token_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes token = 3; + * @return Whether the token field is set. + */ + @java.lang.Override + public boolean hasToken() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * optional bytes token = 3; + * @return The token. + */ + @java.lang.Override + public com.google.protobuf.ByteString getToken() { + return token_; + } + /** + * optional bytes token = 3; + * @param value The token to set. + * @return This builder for chaining. + */ + public Builder setToken(com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + token_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * optional bytes token = 3; + * @return This builder for chaining. + */ + public Builder clearToken() { + bitField0_ = (bitField0_ & ~0x00000004); + token_ = getDefaultInstance().getToken(); + onChanged(); + return this; + } + + private java.lang.Object instanceId_ = ""; + /** + * optional string instanceId = 4; + * @return Whether the instanceId field is set. + */ + public boolean hasInstanceId() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * optional string instanceId = 4; + * @return The instanceId. + */ + public java.lang.String getInstanceId() { + java.lang.Object ref = instanceId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + instanceId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string instanceId = 4; + * @return The bytes for instanceId. + */ + public com.google.protobuf.ByteString + getInstanceIdBytes() { + java.lang.Object ref = instanceId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + instanceId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * optional string instanceId = 4; + * @param value The instanceId to set. + * @return This builder for chaining. + */ + public Builder setInstanceId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + instanceId_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * optional string instanceId = 4; + * @return This builder for chaining. + */ + public Builder clearInstanceId() { + instanceId_ = getDefaultInstance().getInstanceId(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * optional string instanceId = 4; + * @param value The bytes for instanceId to set. + * @return This builder for chaining. + */ + public Builder setInstanceIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + instanceId_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:security.PCredentials) + } + + // @@protoc_insertion_point(class_scope:security.PCredentials) + private static final org.apache.accumulo.grpc.compaction.protobuf.PCredentials DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PCredentials(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PCredentials getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PCredentials parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCredentialsOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCredentialsOrBuilder.java new file mode 100644 index 00000000000..cb6c2d89296 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PCredentialsOrBuilder.java @@ -0,0 +1,90 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: security.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PCredentialsOrBuilder extends + // @@protoc_insertion_point(interface_extends:security.PCredentials) + com.google.protobuf.MessageOrBuilder { + + /** + * optional string principal = 1; + * @return Whether the principal field is set. + */ + boolean hasPrincipal(); + /** + * optional string principal = 1; + * @return The principal. + */ + java.lang.String getPrincipal(); + /** + * optional string principal = 1; + * @return The bytes for principal. + */ + com.google.protobuf.ByteString + getPrincipalBytes(); + + /** + * optional string tokenClassName = 2; + * @return Whether the tokenClassName field is set. + */ + boolean hasTokenClassName(); + /** + * optional string tokenClassName = 2; + * @return The tokenClassName. + */ + java.lang.String getTokenClassName(); + /** + * optional string tokenClassName = 2; + * @return The bytes for tokenClassName. + */ + com.google.protobuf.ByteString + getTokenClassNameBytes(); + + /** + * optional bytes token = 3; + * @return Whether the token field is set. + */ + boolean hasToken(); + /** + * optional bytes token = 3; + * @return The token. + */ + com.google.protobuf.ByteString getToken(); + + /** + * optional string instanceId = 4; + * @return Whether the instanceId field is set. + */ + boolean hasInstanceId(); + /** + * optional string instanceId = 4; + * @return The instanceId. + */ + java.lang.String getInstanceId(); + /** + * optional string instanceId = 4; + * @return The bytes for instanceId. + */ + com.google.protobuf.ByteString + getInstanceIdBytes(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompaction.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompaction.java new file mode 100644 index 00000000000..db925152a50 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompaction.java @@ -0,0 +1,1195 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.PExternalCompaction} + */ +public final class PExternalCompaction extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.PExternalCompaction) + PExternalCompactionOrBuilder { +private static final long serialVersionUID = 0L; + // Use PExternalCompaction.newBuilder() to construct. + private PExternalCompaction(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PExternalCompaction() { + groupName_ = ""; + compactor_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PExternalCompaction(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompaction_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + @java.lang.Override + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMapFieldReflection( + int number) { + switch (number) { + case 3: + return internalGetUpdates(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompaction_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.class, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.Builder.class); + } + + private int bitField0_; + public static final int GROUPNAME_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object groupName_ = ""; + /** + * string groupName = 1; + * @return The groupName. + */ + @java.lang.Override + public java.lang.String getGroupName() { + java.lang.Object ref = groupName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + groupName_ = s; + return s; + } + } + /** + * string groupName = 1; + * @return The bytes for groupName. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getGroupNameBytes() { + java.lang.Object ref = groupName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + groupName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int COMPACTOR_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object compactor_ = ""; + /** + * string compactor = 2; + * @return The compactor. + */ + @java.lang.Override + public java.lang.String getCompactor() { + java.lang.Object ref = compactor_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + compactor_ = s; + return s; + } + } + /** + * string compactor = 2; + * @return The bytes for compactor. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getCompactorBytes() { + java.lang.Object ref = compactor_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + compactor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int UPDATES_FIELD_NUMBER = 3; + private static final class UpdatesDefaultEntryHolder { + static final com.google.protobuf.MapEntry< + java.lang.Long, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate> defaultEntry = + com.google.protobuf.MapEntry + .newDefaultInstance( + org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompaction_UpdatesEntry_descriptor, + com.google.protobuf.WireFormat.FieldType.INT64, + 0L, + com.google.protobuf.WireFormat.FieldType.MESSAGE, + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.getDefaultInstance()); + } + @SuppressWarnings("serial") + private com.google.protobuf.MapField< + java.lang.Long, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate> updates_; + private com.google.protobuf.MapField + internalGetUpdates() { + if (updates_ == null) { + return com.google.protobuf.MapField.emptyMapField( + UpdatesDefaultEntryHolder.defaultEntry); + } + return updates_; + } + public int getUpdatesCount() { + return internalGetUpdates().getMap().size(); + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + @java.lang.Override + public boolean containsUpdates( + long key) { + + return internalGetUpdates().getMap().containsKey(key); + } + /** + * Use {@link #getUpdatesMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getUpdates() { + return getUpdatesMap(); + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + @java.lang.Override + public java.util.Map getUpdatesMap() { + return internalGetUpdates().getMap(); + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + @java.lang.Override + public /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getUpdatesOrDefault( + long key, + /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate defaultValue) { + + java.util.Map map = + internalGetUpdates().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getUpdatesOrThrow( + long key) { + + java.util.Map map = + internalGetUpdates().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + public static final int JOB_FIELD_NUMBER = 4; + private org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob job_; + /** + * .tabletserver.PExternalCompactionJob job = 4; + * @return Whether the job field is set. + */ + @java.lang.Override + public boolean hasJob() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .tabletserver.PExternalCompactionJob job = 4; + * @return The job. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob getJob() { + return job_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance() : job_; + } + /** + * .tabletserver.PExternalCompactionJob job = 4; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder getJobOrBuilder() { + return job_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance() : job_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(groupName_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, groupName_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(compactor_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, compactor_); + } + com.google.protobuf.GeneratedMessageV3 + .serializeLongMapTo( + output, + internalGetUpdates(), + UpdatesDefaultEntryHolder.defaultEntry, + 3); + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(4, getJob()); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(groupName_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, groupName_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(compactor_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, compactor_); + } + for (java.util.Map.Entry entry + : internalGetUpdates().getMap().entrySet()) { + com.google.protobuf.MapEntry + updates__ = UpdatesDefaultEntryHolder.defaultEntry.newBuilderForType() + .setKey(entry.getKey()) + .setValue(entry.getValue()) + .build(); + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, updates__); + } + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(4, getJob()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction other = (org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction) obj; + + if (!getGroupName() + .equals(other.getGroupName())) return false; + if (!getCompactor() + .equals(other.getCompactor())) return false; + if (!internalGetUpdates().equals( + other.internalGetUpdates())) return false; + if (hasJob() != other.hasJob()) return false; + if (hasJob()) { + if (!getJob() + .equals(other.getJob())) return false; + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + GROUPNAME_FIELD_NUMBER; + hash = (53 * hash) + getGroupName().hashCode(); + hash = (37 * hash) + COMPACTOR_FIELD_NUMBER; + hash = (53 * hash) + getCompactor().hashCode(); + if (!internalGetUpdates().getMap().isEmpty()) { + hash = (37 * hash) + UPDATES_FIELD_NUMBER; + hash = (53 * hash) + internalGetUpdates().hashCode(); + } + if (hasJob()) { + hash = (37 * hash) + JOB_FIELD_NUMBER; + hash = (53 * hash) + getJob().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.PExternalCompaction} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.PExternalCompaction) + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompaction_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMapFieldReflection( + int number) { + switch (number) { + case 3: + return internalGetUpdates(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMutableMapFieldReflection( + int number) { + switch (number) { + case 3: + return internalGetMutableUpdates(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompaction_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.class, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getJobFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + groupName_ = ""; + compactor_ = ""; + internalGetMutableUpdates().clear(); + job_ = null; + if (jobBuilder_ != null) { + jobBuilder_.dispose(); + jobBuilder_ = null; + } + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompaction_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction build() { + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction result = new org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.groupName_ = groupName_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.compactor_ = compactor_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.updates_ = internalGetUpdates().build(UpdatesDefaultEntryHolder.defaultEntry); + } + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000008) != 0)) { + result.job_ = jobBuilder_ == null + ? job_ + : jobBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.getDefaultInstance()) return this; + if (!other.getGroupName().isEmpty()) { + groupName_ = other.groupName_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (!other.getCompactor().isEmpty()) { + compactor_ = other.compactor_; + bitField0_ |= 0x00000002; + onChanged(); + } + internalGetMutableUpdates().mergeFrom( + other.internalGetUpdates()); + bitField0_ |= 0x00000004; + if (other.hasJob()) { + mergeJob(other.getJob()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + groupName_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + compactor_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + com.google.protobuf.MapEntry + updates__ = input.readMessage( + UpdatesDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry); + internalGetMutableUpdates().ensureBuilderMap().put( + updates__.getKey(), updates__.getValue()); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + input.readMessage( + getJobFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000008; + break; + } // case 34 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object groupName_ = ""; + /** + * string groupName = 1; + * @return The groupName. + */ + public java.lang.String getGroupName() { + java.lang.Object ref = groupName_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + groupName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string groupName = 1; + * @return The bytes for groupName. + */ + public com.google.protobuf.ByteString + getGroupNameBytes() { + java.lang.Object ref = groupName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + groupName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string groupName = 1; + * @param value The groupName to set. + * @return This builder for chaining. + */ + public Builder setGroupName( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + groupName_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * string groupName = 1; + * @return This builder for chaining. + */ + public Builder clearGroupName() { + groupName_ = getDefaultInstance().getGroupName(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * string groupName = 1; + * @param value The bytes for groupName to set. + * @return This builder for chaining. + */ + public Builder setGroupNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + groupName_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private java.lang.Object compactor_ = ""; + /** + * string compactor = 2; + * @return The compactor. + */ + public java.lang.String getCompactor() { + java.lang.Object ref = compactor_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + compactor_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string compactor = 2; + * @return The bytes for compactor. + */ + public com.google.protobuf.ByteString + getCompactorBytes() { + java.lang.Object ref = compactor_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + compactor_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string compactor = 2; + * @param value The compactor to set. + * @return This builder for chaining. + */ + public Builder setCompactor( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + compactor_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string compactor = 2; + * @return This builder for chaining. + */ + public Builder clearCompactor() { + compactor_ = getDefaultInstance().getCompactor(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string compactor = 2; + * @param value The bytes for compactor to set. + * @return This builder for chaining. + */ + public Builder setCompactorBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + compactor_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private static final class UpdatesConverter implements com.google.protobuf.MapFieldBuilder.Converter { + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate build(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdateOrBuilder val) { + if (val instanceof org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate) { return (org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate) val; } + return ((org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder) val).build(); + } + + @java.lang.Override + public com.google.protobuf.MapEntry defaultEntry() { + return UpdatesDefaultEntryHolder.defaultEntry; + } + }; + private static final UpdatesConverter updatesConverter = new UpdatesConverter(); + + private com.google.protobuf.MapFieldBuilder< + java.lang.Long, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdateOrBuilder, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder> updates_; + private com.google.protobuf.MapFieldBuilder + internalGetUpdates() { + if (updates_ == null) { + return new com.google.protobuf.MapFieldBuilder<>(updatesConverter); + } + return updates_; + } + private com.google.protobuf.MapFieldBuilder + internalGetMutableUpdates() { + if (updates_ == null) { + updates_ = new com.google.protobuf.MapFieldBuilder<>(updatesConverter); + } + bitField0_ |= 0x00000004; + onChanged(); + return updates_; + } + public int getUpdatesCount() { + return internalGetUpdates().ensureBuilderMap().size(); + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + @java.lang.Override + public boolean containsUpdates( + long key) { + + return internalGetUpdates().ensureBuilderMap().containsKey(key); + } + /** + * Use {@link #getUpdatesMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getUpdates() { + return getUpdatesMap(); + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + @java.lang.Override + public java.util.Map getUpdatesMap() { + return internalGetUpdates().getImmutableMap(); + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + @java.lang.Override + public /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getUpdatesOrDefault( + long key, + /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate defaultValue) { + + java.util.Map map = internalGetMutableUpdates().ensureBuilderMap(); + return map.containsKey(key) ? updatesConverter.build(map.get(key)) : defaultValue; + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getUpdatesOrThrow( + long key) { + + java.util.Map map = internalGetMutableUpdates().ensureBuilderMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return updatesConverter.build(map.get(key)); + } + public Builder clearUpdates() { + bitField0_ = (bitField0_ & ~0x00000004); + internalGetMutableUpdates().clear(); + return this; + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + public Builder removeUpdates( + long key) { + + internalGetMutableUpdates().ensureBuilderMap() + .remove(key); + return this; + } + /** + * Use alternate mutation accessors instead. + */ + @java.lang.Deprecated + public java.util.Map + getMutableUpdates() { + bitField0_ |= 0x00000004; + return internalGetMutableUpdates().ensureMessageMap(); + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + public Builder putUpdates( + long key, + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate value) { + + if (value == null) { throw new NullPointerException("map value"); } + internalGetMutableUpdates().ensureBuilderMap() + .put(key, value); + bitField0_ |= 0x00000004; + return this; + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + public Builder putAllUpdates( + java.util.Map values) { + for (java.util.Map.Entry e : values.entrySet()) { + if (e.getKey() == null || e.getValue() == null) { + throw new NullPointerException(); + } + } + internalGetMutableUpdates().ensureBuilderMap() + .putAll(values); + bitField0_ |= 0x00000004; + return this; + } + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder putUpdatesBuilderIfAbsent( + long key) { + java.util.Map builderMap = internalGetMutableUpdates().ensureBuilderMap(); + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdateOrBuilder entry = builderMap.get(key); + if (entry == null) { + entry = org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.newBuilder(); + builderMap.put(key, entry); + } + if (entry instanceof org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate) { + entry = ((org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate) entry).toBuilder(); + builderMap.put(key, entry); + } + return (org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder) entry; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob job_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder> jobBuilder_; + /** + * .tabletserver.PExternalCompactionJob job = 4; + * @return Whether the job field is set. + */ + public boolean hasJob() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * .tabletserver.PExternalCompactionJob job = 4; + * @return The job. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob getJob() { + if (jobBuilder_ == null) { + return job_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance() : job_; + } else { + return jobBuilder_.getMessage(); + } + } + /** + * .tabletserver.PExternalCompactionJob job = 4; + */ + public Builder setJob(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob value) { + if (jobBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + job_ = value; + } else { + jobBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * .tabletserver.PExternalCompactionJob job = 4; + */ + public Builder setJob( + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder builderForValue) { + if (jobBuilder_ == null) { + job_ = builderForValue.build(); + } else { + jobBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * .tabletserver.PExternalCompactionJob job = 4; + */ + public Builder mergeJob(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob value) { + if (jobBuilder_ == null) { + if (((bitField0_ & 0x00000008) != 0) && + job_ != null && + job_ != org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance()) { + getJobBuilder().mergeFrom(value); + } else { + job_ = value; + } + } else { + jobBuilder_.mergeFrom(value); + } + if (job_ != null) { + bitField0_ |= 0x00000008; + onChanged(); + } + return this; + } + /** + * .tabletserver.PExternalCompactionJob job = 4; + */ + public Builder clearJob() { + bitField0_ = (bitField0_ & ~0x00000008); + job_ = null; + if (jobBuilder_ != null) { + jobBuilder_.dispose(); + jobBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .tabletserver.PExternalCompactionJob job = 4; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder getJobBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getJobFieldBuilder().getBuilder(); + } + /** + * .tabletserver.PExternalCompactionJob job = 4; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder getJobOrBuilder() { + if (jobBuilder_ != null) { + return jobBuilder_.getMessageOrBuilder(); + } else { + return job_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance() : job_; + } + } + /** + * .tabletserver.PExternalCompactionJob job = 4; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder> + getJobFieldBuilder() { + if (jobBuilder_ == null) { + jobBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder>( + getJob(), + getParentForChildren(), + isClean()); + job_ = null; + } + return jobBuilder_; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.PExternalCompaction) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.PExternalCompaction) + private static final org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PExternalCompaction parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionJob.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionJob.java new file mode 100644 index 00000000000..a4fd873e059 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionJob.java @@ -0,0 +1,2138 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code tabletserver.PExternalCompactionJob} + */ +public final class PExternalCompactionJob extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:tabletserver.PExternalCompactionJob) + PExternalCompactionJobOrBuilder { +private static final long serialVersionUID = 0L; + // Use PExternalCompactionJob.newBuilder() to construct. + private PExternalCompactionJob(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PExternalCompactionJob() { + externalCompactionId_ = ""; + files_ = java.util.Collections.emptyList(); + outputFile_ = ""; + kind_ = 0; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PExternalCompactionJob(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PExternalCompactionJob_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + @java.lang.Override + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMapFieldReflection( + int number) { + switch (number) { + case 9: + return internalGetOverrides(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PExternalCompactionJob_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.class, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder.class); + } + + private int bitField0_; + public static final int EXTERNALCOMPACTIONID_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object externalCompactionId_ = ""; + /** + * optional string externalCompactionId = 1; + * @return Whether the externalCompactionId field is set. + */ + @java.lang.Override + public boolean hasExternalCompactionId() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * optional string externalCompactionId = 1; + * @return The externalCompactionId. + */ + @java.lang.Override + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } + } + /** + * optional string externalCompactionId = 1; + * @return The bytes for externalCompactionId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int EXTENT_FIELD_NUMBER = 2; + private org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent extent_; + /** + * optional .data.PKeyExtent extent = 2; + * @return Whether the extent field is set. + */ + @java.lang.Override + public boolean hasExtent() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * optional .data.PKeyExtent extent = 2; + * @return The extent. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getExtent() { + return extent_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } + /** + * optional .data.PKeyExtent extent = 2; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder getExtentOrBuilder() { + return extent_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } + + public static final int FILES_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private java.util.List files_; + /** + * repeated .tabletserver.PInputFile files = 3; + */ + @java.lang.Override + public java.util.List getFilesList() { + return files_; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + @java.lang.Override + public java.util.List + getFilesOrBuilderList() { + return files_; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + @java.lang.Override + public int getFilesCount() { + return files_.size(); + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PInputFile getFiles(int index) { + return files_.get(index); + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PInputFileOrBuilder getFilesOrBuilder( + int index) { + return files_.get(index); + } + + public static final int ITERATORSETTINGS_FIELD_NUMBER = 4; + private org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig iteratorSettings_; + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + * @return Whether the iteratorSettings field is set. + */ + @java.lang.Override + public boolean hasIteratorSettings() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + * @return The iteratorSettings. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig getIteratorSettings() { + return iteratorSettings_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.getDefaultInstance() : iteratorSettings_; + } + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfigOrBuilder getIteratorSettingsOrBuilder() { + return iteratorSettings_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.getDefaultInstance() : iteratorSettings_; + } + + public static final int OUTPUTFILE_FIELD_NUMBER = 5; + @SuppressWarnings("serial") + private volatile java.lang.Object outputFile_ = ""; + /** + * optional string outputFile = 5; + * @return Whether the outputFile field is set. + */ + @java.lang.Override + public boolean hasOutputFile() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * optional string outputFile = 5; + * @return The outputFile. + */ + @java.lang.Override + public java.lang.String getOutputFile() { + java.lang.Object ref = outputFile_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + outputFile_ = s; + return s; + } + } + /** + * optional string outputFile = 5; + * @return The bytes for outputFile. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getOutputFileBytes() { + java.lang.Object ref = outputFile_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + outputFile_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int PROPAGATEDELETES_FIELD_NUMBER = 6; + private boolean propagateDeletes_ = false; + /** + * optional bool propagateDeletes = 6; + * @return Whether the propagateDeletes field is set. + */ + @java.lang.Override + public boolean hasPropagateDeletes() { + return ((bitField0_ & 0x00000010) != 0); + } + /** + * optional bool propagateDeletes = 6; + * @return The propagateDeletes. + */ + @java.lang.Override + public boolean getPropagateDeletes() { + return propagateDeletes_; + } + + public static final int KIND_FIELD_NUMBER = 7; + private int kind_ = 0; + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @return Whether the kind field is set. + */ + @java.lang.Override public boolean hasKind() { + return ((bitField0_ & 0x00000020) != 0); + } + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @return The enum numeric value on the wire for kind. + */ + @java.lang.Override public int getKindValue() { + return kind_; + } + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @return The kind. + */ + @java.lang.Override public org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind getKind() { + org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind result = org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind.forNumber(kind_); + return result == null ? org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind.UNRECOGNIZED : result; + } + + public static final int FATEID_FIELD_NUMBER = 8; + private org.apache.accumulo.grpc.compaction.protobuf.PFateId fateId_; + /** + * optional .manager.PFateId fateId = 8; + * @return Whether the fateId field is set. + */ + @java.lang.Override + public boolean hasFateId() { + return ((bitField0_ & 0x00000040) != 0); + } + /** + * optional .manager.PFateId fateId = 8; + * @return The fateId. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PFateId getFateId() { + return fateId_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PFateId.getDefaultInstance() : fateId_; + } + /** + * optional .manager.PFateId fateId = 8; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PFateIdOrBuilder getFateIdOrBuilder() { + return fateId_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PFateId.getDefaultInstance() : fateId_; + } + + public static final int OVERRIDES_FIELD_NUMBER = 9; + private static final class OverridesDefaultEntryHolder { + static final com.google.protobuf.MapEntry< + java.lang.String, java.lang.String> defaultEntry = + com.google.protobuf.MapEntry + .newDefaultInstance( + org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PExternalCompactionJob_OverridesEntry_descriptor, + com.google.protobuf.WireFormat.FieldType.STRING, + "", + com.google.protobuf.WireFormat.FieldType.STRING, + ""); + } + @SuppressWarnings("serial") + private com.google.protobuf.MapField< + java.lang.String, java.lang.String> overrides_; + private com.google.protobuf.MapField + internalGetOverrides() { + if (overrides_ == null) { + return com.google.protobuf.MapField.emptyMapField( + OverridesDefaultEntryHolder.defaultEntry); + } + return overrides_; + } + public int getOverridesCount() { + return internalGetOverrides().getMap().size(); + } + /** + * map<string, string> overrides = 9; + */ + @java.lang.Override + public boolean containsOverrides( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + return internalGetOverrides().getMap().containsKey(key); + } + /** + * Use {@link #getOverridesMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getOverrides() { + return getOverridesMap(); + } + /** + * map<string, string> overrides = 9; + */ + @java.lang.Override + public java.util.Map getOverridesMap() { + return internalGetOverrides().getMap(); + } + /** + * map<string, string> overrides = 9; + */ + @java.lang.Override + public /* nullable */ +java.lang.String getOverridesOrDefault( + java.lang.String key, + /* nullable */ +java.lang.String defaultValue) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetOverrides().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + * map<string, string> overrides = 9; + */ + @java.lang.Override + public java.lang.String getOverridesOrThrow( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetOverrides().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, externalCompactionId_); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getExtent()); + } + for (int i = 0; i < files_.size(); i++) { + output.writeMessage(3, files_.get(i)); + } + if (((bitField0_ & 0x00000004) != 0)) { + output.writeMessage(4, getIteratorSettings()); + } + if (((bitField0_ & 0x00000008) != 0)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 5, outputFile_); + } + if (((bitField0_ & 0x00000010) != 0)) { + output.writeBool(6, propagateDeletes_); + } + if (((bitField0_ & 0x00000020) != 0)) { + output.writeEnum(7, kind_); + } + if (((bitField0_ & 0x00000040) != 0)) { + output.writeMessage(8, getFateId()); + } + com.google.protobuf.GeneratedMessageV3 + .serializeStringMapTo( + output, + internalGetOverrides(), + OverridesDefaultEntryHolder.defaultEntry, + 9); + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, externalCompactionId_); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, getExtent()); + } + for (int i = 0; i < files_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, files_.get(i)); + } + if (((bitField0_ & 0x00000004) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(4, getIteratorSettings()); + } + if (((bitField0_ & 0x00000008) != 0)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(5, outputFile_); + } + if (((bitField0_ & 0x00000010) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(6, propagateDeletes_); + } + if (((bitField0_ & 0x00000020) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(7, kind_); + } + if (((bitField0_ & 0x00000040) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(8, getFateId()); + } + for (java.util.Map.Entry entry + : internalGetOverrides().getMap().entrySet()) { + com.google.protobuf.MapEntry + overrides__ = OverridesDefaultEntryHolder.defaultEntry.newBuilderForType() + .setKey(entry.getKey()) + .setValue(entry.getValue()) + .build(); + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(9, overrides__); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob other = (org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob) obj; + + if (hasExternalCompactionId() != other.hasExternalCompactionId()) return false; + if (hasExternalCompactionId()) { + if (!getExternalCompactionId() + .equals(other.getExternalCompactionId())) return false; + } + if (hasExtent() != other.hasExtent()) return false; + if (hasExtent()) { + if (!getExtent() + .equals(other.getExtent())) return false; + } + if (!getFilesList() + .equals(other.getFilesList())) return false; + if (hasIteratorSettings() != other.hasIteratorSettings()) return false; + if (hasIteratorSettings()) { + if (!getIteratorSettings() + .equals(other.getIteratorSettings())) return false; + } + if (hasOutputFile() != other.hasOutputFile()) return false; + if (hasOutputFile()) { + if (!getOutputFile() + .equals(other.getOutputFile())) return false; + } + if (hasPropagateDeletes() != other.hasPropagateDeletes()) return false; + if (hasPropagateDeletes()) { + if (getPropagateDeletes() + != other.getPropagateDeletes()) return false; + } + if (hasKind() != other.hasKind()) return false; + if (hasKind()) { + if (kind_ != other.kind_) return false; + } + if (hasFateId() != other.hasFateId()) return false; + if (hasFateId()) { + if (!getFateId() + .equals(other.getFateId())) return false; + } + if (!internalGetOverrides().equals( + other.internalGetOverrides())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasExternalCompactionId()) { + hash = (37 * hash) + EXTERNALCOMPACTIONID_FIELD_NUMBER; + hash = (53 * hash) + getExternalCompactionId().hashCode(); + } + if (hasExtent()) { + hash = (37 * hash) + EXTENT_FIELD_NUMBER; + hash = (53 * hash) + getExtent().hashCode(); + } + if (getFilesCount() > 0) { + hash = (37 * hash) + FILES_FIELD_NUMBER; + hash = (53 * hash) + getFilesList().hashCode(); + } + if (hasIteratorSettings()) { + hash = (37 * hash) + ITERATORSETTINGS_FIELD_NUMBER; + hash = (53 * hash) + getIteratorSettings().hashCode(); + } + if (hasOutputFile()) { + hash = (37 * hash) + OUTPUTFILE_FIELD_NUMBER; + hash = (53 * hash) + getOutputFile().hashCode(); + } + if (hasPropagateDeletes()) { + hash = (37 * hash) + PROPAGATEDELETES_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( + getPropagateDeletes()); + } + if (hasKind()) { + hash = (37 * hash) + KIND_FIELD_NUMBER; + hash = (53 * hash) + kind_; + } + if (hasFateId()) { + hash = (37 * hash) + FATEID_FIELD_NUMBER; + hash = (53 * hash) + getFateId().hashCode(); + } + if (!internalGetOverrides().getMap().isEmpty()) { + hash = (37 * hash) + OVERRIDES_FIELD_NUMBER; + hash = (53 * hash) + internalGetOverrides().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code tabletserver.PExternalCompactionJob} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:tabletserver.PExternalCompactionJob) + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PExternalCompactionJob_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMapFieldReflection( + int number) { + switch (number) { + case 9: + return internalGetOverrides(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMutableMapFieldReflection( + int number) { + switch (number) { + case 9: + return internalGetMutableOverrides(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PExternalCompactionJob_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.class, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getExtentFieldBuilder(); + getFilesFieldBuilder(); + getIteratorSettingsFieldBuilder(); + getFateIdFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + externalCompactionId_ = ""; + extent_ = null; + if (extentBuilder_ != null) { + extentBuilder_.dispose(); + extentBuilder_ = null; + } + if (filesBuilder_ == null) { + files_ = java.util.Collections.emptyList(); + } else { + files_ = null; + filesBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + iteratorSettings_ = null; + if (iteratorSettingsBuilder_ != null) { + iteratorSettingsBuilder_.dispose(); + iteratorSettingsBuilder_ = null; + } + outputFile_ = ""; + propagateDeletes_ = false; + kind_ = 0; + fateId_ = null; + if (fateIdBuilder_ != null) { + fateIdBuilder_.dispose(); + fateIdBuilder_ = null; + } + internalGetMutableOverrides().clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PExternalCompactionJob_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob build() { + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob result = new org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob(this); + buildPartialRepeatedFields(result); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartialRepeatedFields(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob result) { + if (filesBuilder_ == null) { + if (((bitField0_ & 0x00000004) != 0)) { + files_ = java.util.Collections.unmodifiableList(files_); + bitField0_ = (bitField0_ & ~0x00000004); + } + result.files_ = files_; + } else { + result.files_ = filesBuilder_.build(); + } + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.externalCompactionId_ = externalCompactionId_; + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.extent_ = extentBuilder_ == null + ? extent_ + : extentBuilder_.build(); + to_bitField0_ |= 0x00000002; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.iteratorSettings_ = iteratorSettingsBuilder_ == null + ? iteratorSettings_ + : iteratorSettingsBuilder_.build(); + to_bitField0_ |= 0x00000004; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.outputFile_ = outputFile_; + to_bitField0_ |= 0x00000008; + } + if (((from_bitField0_ & 0x00000020) != 0)) { + result.propagateDeletes_ = propagateDeletes_; + to_bitField0_ |= 0x00000010; + } + if (((from_bitField0_ & 0x00000040) != 0)) { + result.kind_ = kind_; + to_bitField0_ |= 0x00000020; + } + if (((from_bitField0_ & 0x00000080) != 0)) { + result.fateId_ = fateIdBuilder_ == null + ? fateId_ + : fateIdBuilder_.build(); + to_bitField0_ |= 0x00000040; + } + if (((from_bitField0_ & 0x00000100) != 0)) { + result.overrides_ = internalGetOverrides(); + result.overrides_.makeImmutable(); + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance()) return this; + if (other.hasExternalCompactionId()) { + externalCompactionId_ = other.externalCompactionId_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (other.hasExtent()) { + mergeExtent(other.getExtent()); + } + if (filesBuilder_ == null) { + if (!other.files_.isEmpty()) { + if (files_.isEmpty()) { + files_ = other.files_; + bitField0_ = (bitField0_ & ~0x00000004); + } else { + ensureFilesIsMutable(); + files_.addAll(other.files_); + } + onChanged(); + } + } else { + if (!other.files_.isEmpty()) { + if (filesBuilder_.isEmpty()) { + filesBuilder_.dispose(); + filesBuilder_ = null; + files_ = other.files_; + bitField0_ = (bitField0_ & ~0x00000004); + filesBuilder_ = + com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getFilesFieldBuilder() : null; + } else { + filesBuilder_.addAllMessages(other.files_); + } + } + } + if (other.hasIteratorSettings()) { + mergeIteratorSettings(other.getIteratorSettings()); + } + if (other.hasOutputFile()) { + outputFile_ = other.outputFile_; + bitField0_ |= 0x00000010; + onChanged(); + } + if (other.hasPropagateDeletes()) { + setPropagateDeletes(other.getPropagateDeletes()); + } + if (other.hasKind()) { + setKind(other.getKind()); + } + if (other.hasFateId()) { + mergeFateId(other.getFateId()); + } + internalGetMutableOverrides().mergeFrom( + other.internalGetOverrides()); + bitField0_ |= 0x00000100; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + externalCompactionId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getExtentFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + org.apache.accumulo.grpc.compaction.protobuf.PInputFile m = + input.readMessage( + org.apache.accumulo.grpc.compaction.protobuf.PInputFile.parser(), + extensionRegistry); + if (filesBuilder_ == null) { + ensureFilesIsMutable(); + files_.add(m); + } else { + filesBuilder_.addMessage(m); + } + break; + } // case 26 + case 34: { + input.readMessage( + getIteratorSettingsFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 42: { + outputFile_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000010; + break; + } // case 42 + case 48: { + propagateDeletes_ = input.readBool(); + bitField0_ |= 0x00000020; + break; + } // case 48 + case 56: { + kind_ = input.readEnum(); + bitField0_ |= 0x00000040; + break; + } // case 56 + case 66: { + input.readMessage( + getFateIdFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000080; + break; + } // case 66 + case 74: { + com.google.protobuf.MapEntry + overrides__ = input.readMessage( + OverridesDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry); + internalGetMutableOverrides().getMutableMap().put( + overrides__.getKey(), overrides__.getValue()); + bitField0_ |= 0x00000100; + break; + } // case 74 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object externalCompactionId_ = ""; + /** + * optional string externalCompactionId = 1; + * @return Whether the externalCompactionId field is set. + */ + public boolean hasExternalCompactionId() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * optional string externalCompactionId = 1; + * @return The externalCompactionId. + */ + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string externalCompactionId = 1; + * @return The bytes for externalCompactionId. + */ + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * optional string externalCompactionId = 1; + * @param value The externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + externalCompactionId_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * optional string externalCompactionId = 1; + * @return This builder for chaining. + */ + public Builder clearExternalCompactionId() { + externalCompactionId_ = getDefaultInstance().getExternalCompactionId(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + * optional string externalCompactionId = 1; + * @param value The bytes for externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + externalCompactionId_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent extent_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder> extentBuilder_; + /** + * optional .data.PKeyExtent extent = 2; + * @return Whether the extent field is set. + */ + public boolean hasExtent() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * optional .data.PKeyExtent extent = 2; + * @return The extent. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getExtent() { + if (extentBuilder_ == null) { + return extent_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } else { + return extentBuilder_.getMessage(); + } + } + /** + * optional .data.PKeyExtent extent = 2; + */ + public Builder setExtent(org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent value) { + if (extentBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + extent_ = value; + } else { + extentBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * optional .data.PKeyExtent extent = 2; + */ + public Builder setExtent( + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder builderForValue) { + if (extentBuilder_ == null) { + extent_ = builderForValue.build(); + } else { + extentBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * optional .data.PKeyExtent extent = 2; + */ + public Builder mergeExtent(org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent value) { + if (extentBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) && + extent_ != null && + extent_ != org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance()) { + getExtentBuilder().mergeFrom(value); + } else { + extent_ = value; + } + } else { + extentBuilder_.mergeFrom(value); + } + if (extent_ != null) { + bitField0_ |= 0x00000002; + onChanged(); + } + return this; + } + /** + * optional .data.PKeyExtent extent = 2; + */ + public Builder clearExtent() { + bitField0_ = (bitField0_ & ~0x00000002); + extent_ = null; + if (extentBuilder_ != null) { + extentBuilder_.dispose(); + extentBuilder_ = null; + } + onChanged(); + return this; + } + /** + * optional .data.PKeyExtent extent = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder getExtentBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getExtentFieldBuilder().getBuilder(); + } + /** + * optional .data.PKeyExtent extent = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder getExtentOrBuilder() { + if (extentBuilder_ != null) { + return extentBuilder_.getMessageOrBuilder(); + } else { + return extent_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance() : extent_; + } + } + /** + * optional .data.PKeyExtent extent = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder> + getExtentFieldBuilder() { + if (extentBuilder_ == null) { + extentBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder>( + getExtent(), + getParentForChildren(), + isClean()); + extent_ = null; + } + return extentBuilder_; + } + + private java.util.List files_ = + java.util.Collections.emptyList(); + private void ensureFilesIsMutable() { + if (!((bitField0_ & 0x00000004) != 0)) { + files_ = new java.util.ArrayList(files_); + bitField0_ |= 0x00000004; + } + } + + private com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PInputFile, org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder, org.apache.accumulo.grpc.compaction.protobuf.PInputFileOrBuilder> filesBuilder_; + + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public java.util.List getFilesList() { + if (filesBuilder_ == null) { + return java.util.Collections.unmodifiableList(files_); + } else { + return filesBuilder_.getMessageList(); + } + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public int getFilesCount() { + if (filesBuilder_ == null) { + return files_.size(); + } else { + return filesBuilder_.getCount(); + } + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PInputFile getFiles(int index) { + if (filesBuilder_ == null) { + return files_.get(index); + } else { + return filesBuilder_.getMessage(index); + } + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public Builder setFiles( + int index, org.apache.accumulo.grpc.compaction.protobuf.PInputFile value) { + if (filesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureFilesIsMutable(); + files_.set(index, value); + onChanged(); + } else { + filesBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public Builder setFiles( + int index, org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder builderForValue) { + if (filesBuilder_ == null) { + ensureFilesIsMutable(); + files_.set(index, builderForValue.build()); + onChanged(); + } else { + filesBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public Builder addFiles(org.apache.accumulo.grpc.compaction.protobuf.PInputFile value) { + if (filesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureFilesIsMutable(); + files_.add(value); + onChanged(); + } else { + filesBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public Builder addFiles( + int index, org.apache.accumulo.grpc.compaction.protobuf.PInputFile value) { + if (filesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureFilesIsMutable(); + files_.add(index, value); + onChanged(); + } else { + filesBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public Builder addFiles( + org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder builderForValue) { + if (filesBuilder_ == null) { + ensureFilesIsMutable(); + files_.add(builderForValue.build()); + onChanged(); + } else { + filesBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public Builder addFiles( + int index, org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder builderForValue) { + if (filesBuilder_ == null) { + ensureFilesIsMutable(); + files_.add(index, builderForValue.build()); + onChanged(); + } else { + filesBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public Builder addAllFiles( + java.lang.Iterable values) { + if (filesBuilder_ == null) { + ensureFilesIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, files_); + onChanged(); + } else { + filesBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public Builder clearFiles() { + if (filesBuilder_ == null) { + files_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + } else { + filesBuilder_.clear(); + } + return this; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public Builder removeFiles(int index) { + if (filesBuilder_ == null) { + ensureFilesIsMutable(); + files_.remove(index); + onChanged(); + } else { + filesBuilder_.remove(index); + } + return this; + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder getFilesBuilder( + int index) { + return getFilesFieldBuilder().getBuilder(index); + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PInputFileOrBuilder getFilesOrBuilder( + int index) { + if (filesBuilder_ == null) { + return files_.get(index); } else { + return filesBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public java.util.List + getFilesOrBuilderList() { + if (filesBuilder_ != null) { + return filesBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(files_); + } + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder addFilesBuilder() { + return getFilesFieldBuilder().addBuilder( + org.apache.accumulo.grpc.compaction.protobuf.PInputFile.getDefaultInstance()); + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder addFilesBuilder( + int index) { + return getFilesFieldBuilder().addBuilder( + index, org.apache.accumulo.grpc.compaction.protobuf.PInputFile.getDefaultInstance()); + } + /** + * repeated .tabletserver.PInputFile files = 3; + */ + public java.util.List + getFilesBuilderList() { + return getFilesFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PInputFile, org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder, org.apache.accumulo.grpc.compaction.protobuf.PInputFileOrBuilder> + getFilesFieldBuilder() { + if (filesBuilder_ == null) { + filesBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PInputFile, org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder, org.apache.accumulo.grpc.compaction.protobuf.PInputFileOrBuilder>( + files_, + ((bitField0_ & 0x00000004) != 0), + getParentForChildren(), + isClean()); + files_ = null; + } + return filesBuilder_; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig iteratorSettings_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig, org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.Builder, org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfigOrBuilder> iteratorSettingsBuilder_; + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + * @return Whether the iteratorSettings field is set. + */ + public boolean hasIteratorSettings() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + * @return The iteratorSettings. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig getIteratorSettings() { + if (iteratorSettingsBuilder_ == null) { + return iteratorSettings_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.getDefaultInstance() : iteratorSettings_; + } else { + return iteratorSettingsBuilder_.getMessage(); + } + } + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + */ + public Builder setIteratorSettings(org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig value) { + if (iteratorSettingsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + iteratorSettings_ = value; + } else { + iteratorSettingsBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + */ + public Builder setIteratorSettings( + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.Builder builderForValue) { + if (iteratorSettingsBuilder_ == null) { + iteratorSettings_ = builderForValue.build(); + } else { + iteratorSettingsBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + */ + public Builder mergeIteratorSettings(org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig value) { + if (iteratorSettingsBuilder_ == null) { + if (((bitField0_ & 0x00000008) != 0) && + iteratorSettings_ != null && + iteratorSettings_ != org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.getDefaultInstance()) { + getIteratorSettingsBuilder().mergeFrom(value); + } else { + iteratorSettings_ = value; + } + } else { + iteratorSettingsBuilder_.mergeFrom(value); + } + if (iteratorSettings_ != null) { + bitField0_ |= 0x00000008; + onChanged(); + } + return this; + } + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + */ + public Builder clearIteratorSettings() { + bitField0_ = (bitField0_ & ~0x00000008); + iteratorSettings_ = null; + if (iteratorSettingsBuilder_ != null) { + iteratorSettingsBuilder_.dispose(); + iteratorSettingsBuilder_ = null; + } + onChanged(); + return this; + } + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.Builder getIteratorSettingsBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getIteratorSettingsFieldBuilder().getBuilder(); + } + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfigOrBuilder getIteratorSettingsOrBuilder() { + if (iteratorSettingsBuilder_ != null) { + return iteratorSettingsBuilder_.getMessageOrBuilder(); + } else { + return iteratorSettings_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.getDefaultInstance() : iteratorSettings_; + } + } + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig, org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.Builder, org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfigOrBuilder> + getIteratorSettingsFieldBuilder() { + if (iteratorSettingsBuilder_ == null) { + iteratorSettingsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig, org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.Builder, org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfigOrBuilder>( + getIteratorSettings(), + getParentForChildren(), + isClean()); + iteratorSettings_ = null; + } + return iteratorSettingsBuilder_; + } + + private java.lang.Object outputFile_ = ""; + /** + * optional string outputFile = 5; + * @return Whether the outputFile field is set. + */ + public boolean hasOutputFile() { + return ((bitField0_ & 0x00000010) != 0); + } + /** + * optional string outputFile = 5; + * @return The outputFile. + */ + public java.lang.String getOutputFile() { + java.lang.Object ref = outputFile_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + outputFile_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string outputFile = 5; + * @return The bytes for outputFile. + */ + public com.google.protobuf.ByteString + getOutputFileBytes() { + java.lang.Object ref = outputFile_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + outputFile_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * optional string outputFile = 5; + * @param value The outputFile to set. + * @return This builder for chaining. + */ + public Builder setOutputFile( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + outputFile_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * optional string outputFile = 5; + * @return This builder for chaining. + */ + public Builder clearOutputFile() { + outputFile_ = getDefaultInstance().getOutputFile(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + return this; + } + /** + * optional string outputFile = 5; + * @param value The bytes for outputFile to set. + * @return This builder for chaining. + */ + public Builder setOutputFileBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + outputFile_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + + private boolean propagateDeletes_ ; + /** + * optional bool propagateDeletes = 6; + * @return Whether the propagateDeletes field is set. + */ + @java.lang.Override + public boolean hasPropagateDeletes() { + return ((bitField0_ & 0x00000020) != 0); + } + /** + * optional bool propagateDeletes = 6; + * @return The propagateDeletes. + */ + @java.lang.Override + public boolean getPropagateDeletes() { + return propagateDeletes_; + } + /** + * optional bool propagateDeletes = 6; + * @param value The propagateDeletes to set. + * @return This builder for chaining. + */ + public Builder setPropagateDeletes(boolean value) { + + propagateDeletes_ = value; + bitField0_ |= 0x00000020; + onChanged(); + return this; + } + /** + * optional bool propagateDeletes = 6; + * @return This builder for chaining. + */ + public Builder clearPropagateDeletes() { + bitField0_ = (bitField0_ & ~0x00000020); + propagateDeletes_ = false; + onChanged(); + return this; + } + + private int kind_ = 0; + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @return Whether the kind field is set. + */ + @java.lang.Override public boolean hasKind() { + return ((bitField0_ & 0x00000040) != 0); + } + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @return The enum numeric value on the wire for kind. + */ + @java.lang.Override public int getKindValue() { + return kind_; + } + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @param value The enum numeric value on the wire for kind to set. + * @return This builder for chaining. + */ + public Builder setKindValue(int value) { + kind_ = value; + bitField0_ |= 0x00000040; + onChanged(); + return this; + } + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @return The kind. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind getKind() { + org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind result = org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind.forNumber(kind_); + return result == null ? org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind.UNRECOGNIZED : result; + } + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @param value The kind to set. + * @return This builder for chaining. + */ + public Builder setKind(org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000040; + kind_ = value.getNumber(); + onChanged(); + return this; + } + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @return This builder for chaining. + */ + public Builder clearKind() { + bitField0_ = (bitField0_ & ~0x00000040); + kind_ = 0; + onChanged(); + return this; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PFateId fateId_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PFateId, org.apache.accumulo.grpc.compaction.protobuf.PFateId.Builder, org.apache.accumulo.grpc.compaction.protobuf.PFateIdOrBuilder> fateIdBuilder_; + /** + * optional .manager.PFateId fateId = 8; + * @return Whether the fateId field is set. + */ + public boolean hasFateId() { + return ((bitField0_ & 0x00000080) != 0); + } + /** + * optional .manager.PFateId fateId = 8; + * @return The fateId. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PFateId getFateId() { + if (fateIdBuilder_ == null) { + return fateId_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PFateId.getDefaultInstance() : fateId_; + } else { + return fateIdBuilder_.getMessage(); + } + } + /** + * optional .manager.PFateId fateId = 8; + */ + public Builder setFateId(org.apache.accumulo.grpc.compaction.protobuf.PFateId value) { + if (fateIdBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + fateId_ = value; + } else { + fateIdBuilder_.setMessage(value); + } + bitField0_ |= 0x00000080; + onChanged(); + return this; + } + /** + * optional .manager.PFateId fateId = 8; + */ + public Builder setFateId( + org.apache.accumulo.grpc.compaction.protobuf.PFateId.Builder builderForValue) { + if (fateIdBuilder_ == null) { + fateId_ = builderForValue.build(); + } else { + fateIdBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000080; + onChanged(); + return this; + } + /** + * optional .manager.PFateId fateId = 8; + */ + public Builder mergeFateId(org.apache.accumulo.grpc.compaction.protobuf.PFateId value) { + if (fateIdBuilder_ == null) { + if (((bitField0_ & 0x00000080) != 0) && + fateId_ != null && + fateId_ != org.apache.accumulo.grpc.compaction.protobuf.PFateId.getDefaultInstance()) { + getFateIdBuilder().mergeFrom(value); + } else { + fateId_ = value; + } + } else { + fateIdBuilder_.mergeFrom(value); + } + if (fateId_ != null) { + bitField0_ |= 0x00000080; + onChanged(); + } + return this; + } + /** + * optional .manager.PFateId fateId = 8; + */ + public Builder clearFateId() { + bitField0_ = (bitField0_ & ~0x00000080); + fateId_ = null; + if (fateIdBuilder_ != null) { + fateIdBuilder_.dispose(); + fateIdBuilder_ = null; + } + onChanged(); + return this; + } + /** + * optional .manager.PFateId fateId = 8; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PFateId.Builder getFateIdBuilder() { + bitField0_ |= 0x00000080; + onChanged(); + return getFateIdFieldBuilder().getBuilder(); + } + /** + * optional .manager.PFateId fateId = 8; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PFateIdOrBuilder getFateIdOrBuilder() { + if (fateIdBuilder_ != null) { + return fateIdBuilder_.getMessageOrBuilder(); + } else { + return fateId_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PFateId.getDefaultInstance() : fateId_; + } + } + /** + * optional .manager.PFateId fateId = 8; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PFateId, org.apache.accumulo.grpc.compaction.protobuf.PFateId.Builder, org.apache.accumulo.grpc.compaction.protobuf.PFateIdOrBuilder> + getFateIdFieldBuilder() { + if (fateIdBuilder_ == null) { + fateIdBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PFateId, org.apache.accumulo.grpc.compaction.protobuf.PFateId.Builder, org.apache.accumulo.grpc.compaction.protobuf.PFateIdOrBuilder>( + getFateId(), + getParentForChildren(), + isClean()); + fateId_ = null; + } + return fateIdBuilder_; + } + + private com.google.protobuf.MapField< + java.lang.String, java.lang.String> overrides_; + private com.google.protobuf.MapField + internalGetOverrides() { + if (overrides_ == null) { + return com.google.protobuf.MapField.emptyMapField( + OverridesDefaultEntryHolder.defaultEntry); + } + return overrides_; + } + private com.google.protobuf.MapField + internalGetMutableOverrides() { + if (overrides_ == null) { + overrides_ = com.google.protobuf.MapField.newMapField( + OverridesDefaultEntryHolder.defaultEntry); + } + if (!overrides_.isMutable()) { + overrides_ = overrides_.copy(); + } + bitField0_ |= 0x00000100; + onChanged(); + return overrides_; + } + public int getOverridesCount() { + return internalGetOverrides().getMap().size(); + } + /** + * map<string, string> overrides = 9; + */ + @java.lang.Override + public boolean containsOverrides( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + return internalGetOverrides().getMap().containsKey(key); + } + /** + * Use {@link #getOverridesMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getOverrides() { + return getOverridesMap(); + } + /** + * map<string, string> overrides = 9; + */ + @java.lang.Override + public java.util.Map getOverridesMap() { + return internalGetOverrides().getMap(); + } + /** + * map<string, string> overrides = 9; + */ + @java.lang.Override + public /* nullable */ +java.lang.String getOverridesOrDefault( + java.lang.String key, + /* nullable */ +java.lang.String defaultValue) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetOverrides().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + * map<string, string> overrides = 9; + */ + @java.lang.Override + public java.lang.String getOverridesOrThrow( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetOverrides().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + public Builder clearOverrides() { + bitField0_ = (bitField0_ & ~0x00000100); + internalGetMutableOverrides().getMutableMap() + .clear(); + return this; + } + /** + * map<string, string> overrides = 9; + */ + public Builder removeOverrides( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + internalGetMutableOverrides().getMutableMap() + .remove(key); + return this; + } + /** + * Use alternate mutation accessors instead. + */ + @java.lang.Deprecated + public java.util.Map + getMutableOverrides() { + bitField0_ |= 0x00000100; + return internalGetMutableOverrides().getMutableMap(); + } + /** + * map<string, string> overrides = 9; + */ + public Builder putOverrides( + java.lang.String key, + java.lang.String value) { + if (key == null) { throw new NullPointerException("map key"); } + if (value == null) { throw new NullPointerException("map value"); } + internalGetMutableOverrides().getMutableMap() + .put(key, value); + bitField0_ |= 0x00000100; + return this; + } + /** + * map<string, string> overrides = 9; + */ + public Builder putAllOverrides( + java.util.Map values) { + internalGetMutableOverrides().getMutableMap() + .putAll(values); + bitField0_ |= 0x00000100; + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:tabletserver.PExternalCompactionJob) + } + + // @@protoc_insertion_point(class_scope:tabletserver.PExternalCompactionJob) + private static final org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PExternalCompactionJob parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionJobOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionJobOrBuilder.java new file mode 100644 index 00000000000..0efc29627b4 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionJobOrBuilder.java @@ -0,0 +1,192 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PExternalCompactionJobOrBuilder extends + // @@protoc_insertion_point(interface_extends:tabletserver.PExternalCompactionJob) + com.google.protobuf.MessageOrBuilder { + + /** + * optional string externalCompactionId = 1; + * @return Whether the externalCompactionId field is set. + */ + boolean hasExternalCompactionId(); + /** + * optional string externalCompactionId = 1; + * @return The externalCompactionId. + */ + java.lang.String getExternalCompactionId(); + /** + * optional string externalCompactionId = 1; + * @return The bytes for externalCompactionId. + */ + com.google.protobuf.ByteString + getExternalCompactionIdBytes(); + + /** + * optional .data.PKeyExtent extent = 2; + * @return Whether the extent field is set. + */ + boolean hasExtent(); + /** + * optional .data.PKeyExtent extent = 2; + * @return The extent. + */ + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getExtent(); + /** + * optional .data.PKeyExtent extent = 2; + */ + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder getExtentOrBuilder(); + + /** + * repeated .tabletserver.PInputFile files = 3; + */ + java.util.List + getFilesList(); + /** + * repeated .tabletserver.PInputFile files = 3; + */ + org.apache.accumulo.grpc.compaction.protobuf.PInputFile getFiles(int index); + /** + * repeated .tabletserver.PInputFile files = 3; + */ + int getFilesCount(); + /** + * repeated .tabletserver.PInputFile files = 3; + */ + java.util.List + getFilesOrBuilderList(); + /** + * repeated .tabletserver.PInputFile files = 3; + */ + org.apache.accumulo.grpc.compaction.protobuf.PInputFileOrBuilder getFilesOrBuilder( + int index); + + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + * @return Whether the iteratorSettings field is set. + */ + boolean hasIteratorSettings(); + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + * @return The iteratorSettings. + */ + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig getIteratorSettings(); + /** + * optional .tabletserver.PIteratorConfig iteratorSettings = 4; + */ + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfigOrBuilder getIteratorSettingsOrBuilder(); + + /** + * optional string outputFile = 5; + * @return Whether the outputFile field is set. + */ + boolean hasOutputFile(); + /** + * optional string outputFile = 5; + * @return The outputFile. + */ + java.lang.String getOutputFile(); + /** + * optional string outputFile = 5; + * @return The bytes for outputFile. + */ + com.google.protobuf.ByteString + getOutputFileBytes(); + + /** + * optional bool propagateDeletes = 6; + * @return Whether the propagateDeletes field is set. + */ + boolean hasPropagateDeletes(); + /** + * optional bool propagateDeletes = 6; + * @return The propagateDeletes. + */ + boolean getPropagateDeletes(); + + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @return Whether the kind field is set. + */ + boolean hasKind(); + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @return The enum numeric value on the wire for kind. + */ + int getKindValue(); + /** + * optional .tabletserver.PCompactionKind kind = 7; + * @return The kind. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind getKind(); + + /** + * optional .manager.PFateId fateId = 8; + * @return Whether the fateId field is set. + */ + boolean hasFateId(); + /** + * optional .manager.PFateId fateId = 8; + * @return The fateId. + */ + org.apache.accumulo.grpc.compaction.protobuf.PFateId getFateId(); + /** + * optional .manager.PFateId fateId = 8; + */ + org.apache.accumulo.grpc.compaction.protobuf.PFateIdOrBuilder getFateIdOrBuilder(); + + /** + * map<string, string> overrides = 9; + */ + int getOverridesCount(); + /** + * map<string, string> overrides = 9; + */ + boolean containsOverrides( + java.lang.String key); + /** + * Use {@link #getOverridesMap()} instead. + */ + @java.lang.Deprecated + java.util.Map + getOverrides(); + /** + * map<string, string> overrides = 9; + */ + java.util.Map + getOverridesMap(); + /** + * map<string, string> overrides = 9; + */ + /* nullable */ +java.lang.String getOverridesOrDefault( + java.lang.String key, + /* nullable */ +java.lang.String defaultValue); + /** + * map<string, string> overrides = 9; + */ + java.lang.String getOverridesOrThrow( + java.lang.String key); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionList.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionList.java new file mode 100644 index 00000000000..b9d60ee0fdd --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionList.java @@ -0,0 +1,730 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.PExternalCompactionList} + */ +public final class PExternalCompactionList extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.PExternalCompactionList) + PExternalCompactionListOrBuilder { +private static final long serialVersionUID = 0L; + // Use PExternalCompactionList.newBuilder() to construct. + private PExternalCompactionList(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PExternalCompactionList() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PExternalCompactionList(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompactionList_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + @java.lang.Override + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMapFieldReflection( + int number) { + switch (number) { + case 1: + return internalGetCompactions(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompactionList_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.class, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.Builder.class); + } + + public static final int COMPACTIONS_FIELD_NUMBER = 1; + private static final class CompactionsDefaultEntryHolder { + static final com.google.protobuf.MapEntry< + java.lang.String, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction> defaultEntry = + com.google.protobuf.MapEntry + .newDefaultInstance( + org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompactionList_CompactionsEntry_descriptor, + com.google.protobuf.WireFormat.FieldType.STRING, + "", + com.google.protobuf.WireFormat.FieldType.MESSAGE, + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.getDefaultInstance()); + } + @SuppressWarnings("serial") + private com.google.protobuf.MapField< + java.lang.String, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction> compactions_; + private com.google.protobuf.MapField + internalGetCompactions() { + if (compactions_ == null) { + return com.google.protobuf.MapField.emptyMapField( + CompactionsDefaultEntryHolder.defaultEntry); + } + return compactions_; + } + public int getCompactionsCount() { + return internalGetCompactions().getMap().size(); + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + @java.lang.Override + public boolean containsCompactions( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + return internalGetCompactions().getMap().containsKey(key); + } + /** + * Use {@link #getCompactionsMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getCompactions() { + return getCompactionsMap(); + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + @java.lang.Override + public java.util.Map getCompactionsMap() { + return internalGetCompactions().getMap(); + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + @java.lang.Override + public /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction getCompactionsOrDefault( + java.lang.String key, + /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction defaultValue) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetCompactions().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction getCompactionsOrThrow( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetCompactions().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + com.google.protobuf.GeneratedMessageV3 + .serializeStringMapTo( + output, + internalGetCompactions(), + CompactionsDefaultEntryHolder.defaultEntry, + 1); + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (java.util.Map.Entry entry + : internalGetCompactions().getMap().entrySet()) { + com.google.protobuf.MapEntry + compactions__ = CompactionsDefaultEntryHolder.defaultEntry.newBuilderForType() + .setKey(entry.getKey()) + .setValue(entry.getValue()) + .build(); + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, compactions__); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList other = (org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList) obj; + + if (!internalGetCompactions().equals( + other.internalGetCompactions())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (!internalGetCompactions().getMap().isEmpty()) { + hash = (37 * hash) + COMPACTIONS_FIELD_NUMBER; + hash = (53 * hash) + internalGetCompactions().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.PExternalCompactionList} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.PExternalCompactionList) + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionListOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompactionList_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMapFieldReflection( + int number) { + switch (number) { + case 1: + return internalGetCompactions(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMutableMapFieldReflection( + int number) { + switch (number) { + case 1: + return internalGetMutableCompactions(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompactionList_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.class, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + internalGetMutableCompactions().clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PExternalCompactionList_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList build() { + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList result = new org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.compactions_ = internalGetCompactions().build(CompactionsDefaultEntryHolder.defaultEntry); + } + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList.getDefaultInstance()) return this; + internalGetMutableCompactions().mergeFrom( + other.internalGetCompactions()); + bitField0_ |= 0x00000001; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + com.google.protobuf.MapEntry + compactions__ = input.readMessage( + CompactionsDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry); + internalGetMutableCompactions().ensureBuilderMap().put( + compactions__.getKey(), compactions__.getValue()); + bitField0_ |= 0x00000001; + break; + } // case 10 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private static final class CompactionsConverter implements com.google.protobuf.MapFieldBuilder.Converter { + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction build(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionOrBuilder val) { + if (val instanceof org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction) { return (org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction) val; } + return ((org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.Builder) val).build(); + } + + @java.lang.Override + public com.google.protobuf.MapEntry defaultEntry() { + return CompactionsDefaultEntryHolder.defaultEntry; + } + }; + private static final CompactionsConverter compactionsConverter = new CompactionsConverter(); + + private com.google.protobuf.MapFieldBuilder< + java.lang.String, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionOrBuilder, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.Builder> compactions_; + private com.google.protobuf.MapFieldBuilder + internalGetCompactions() { + if (compactions_ == null) { + return new com.google.protobuf.MapFieldBuilder<>(compactionsConverter); + } + return compactions_; + } + private com.google.protobuf.MapFieldBuilder + internalGetMutableCompactions() { + if (compactions_ == null) { + compactions_ = new com.google.protobuf.MapFieldBuilder<>(compactionsConverter); + } + bitField0_ |= 0x00000001; + onChanged(); + return compactions_; + } + public int getCompactionsCount() { + return internalGetCompactions().ensureBuilderMap().size(); + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + @java.lang.Override + public boolean containsCompactions( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + return internalGetCompactions().ensureBuilderMap().containsKey(key); + } + /** + * Use {@link #getCompactionsMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getCompactions() { + return getCompactionsMap(); + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + @java.lang.Override + public java.util.Map getCompactionsMap() { + return internalGetCompactions().getImmutableMap(); + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + @java.lang.Override + public /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction getCompactionsOrDefault( + java.lang.String key, + /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction defaultValue) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = internalGetMutableCompactions().ensureBuilderMap(); + return map.containsKey(key) ? compactionsConverter.build(map.get(key)) : defaultValue; + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction getCompactionsOrThrow( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = internalGetMutableCompactions().ensureBuilderMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return compactionsConverter.build(map.get(key)); + } + public Builder clearCompactions() { + bitField0_ = (bitField0_ & ~0x00000001); + internalGetMutableCompactions().clear(); + return this; + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + public Builder removeCompactions( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + internalGetMutableCompactions().ensureBuilderMap() + .remove(key); + return this; + } + /** + * Use alternate mutation accessors instead. + */ + @java.lang.Deprecated + public java.util.Map + getMutableCompactions() { + bitField0_ |= 0x00000001; + return internalGetMutableCompactions().ensureMessageMap(); + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + public Builder putCompactions( + java.lang.String key, + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction value) { + if (key == null) { throw new NullPointerException("map key"); } + if (value == null) { throw new NullPointerException("map value"); } + internalGetMutableCompactions().ensureBuilderMap() + .put(key, value); + bitField0_ |= 0x00000001; + return this; + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + public Builder putAllCompactions( + java.util.Map values) { + for (java.util.Map.Entry e : values.entrySet()) { + if (e.getKey() == null || e.getValue() == null) { + throw new NullPointerException(); + } + } + internalGetMutableCompactions().ensureBuilderMap() + .putAll(values); + bitField0_ |= 0x00000001; + return this; + } + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.Builder putCompactionsBuilderIfAbsent( + java.lang.String key) { + java.util.Map builderMap = internalGetMutableCompactions().ensureBuilderMap(); + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionOrBuilder entry = builderMap.get(key); + if (entry == null) { + entry = org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.newBuilder(); + builderMap.put(key, entry); + } + if (entry instanceof org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction) { + entry = ((org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction) entry).toBuilder(); + builderMap.put(key, entry); + } + return (org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction.Builder) entry; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.PExternalCompactionList) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.PExternalCompactionList) + private static final org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PExternalCompactionList parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionListOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionListOrBuilder.java new file mode 100644 index 00000000000..d806142828e --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionListOrBuilder.java @@ -0,0 +1,62 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PExternalCompactionListOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.PExternalCompactionList) + com.google.protobuf.MessageOrBuilder { + + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + int getCompactionsCount(); + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + boolean containsCompactions( + java.lang.String key); + /** + * Use {@link #getCompactionsMap()} instead. + */ + @java.lang.Deprecated + java.util.Map + getCompactions(); + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + java.util.Map + getCompactionsMap(); + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction getCompactionsOrDefault( + java.lang.String key, + /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction defaultValue); + /** + * map<string, .compaction_coordinator.PExternalCompaction> compactions = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction getCompactionsOrThrow( + java.lang.String key); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionOrBuilder.java new file mode 100644 index 00000000000..445cfef6259 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PExternalCompactionOrBuilder.java @@ -0,0 +1,101 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PExternalCompactionOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.PExternalCompaction) + com.google.protobuf.MessageOrBuilder { + + /** + * string groupName = 1; + * @return The groupName. + */ + java.lang.String getGroupName(); + /** + * string groupName = 1; + * @return The bytes for groupName. + */ + com.google.protobuf.ByteString + getGroupNameBytes(); + + /** + * string compactor = 2; + * @return The compactor. + */ + java.lang.String getCompactor(); + /** + * string compactor = 2; + * @return The bytes for compactor. + */ + com.google.protobuf.ByteString + getCompactorBytes(); + + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + int getUpdatesCount(); + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + boolean containsUpdates( + long key); + /** + * Use {@link #getUpdatesMap()} instead. + */ + @java.lang.Deprecated + java.util.Map + getUpdates(); + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + java.util.Map + getUpdatesMap(); + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getUpdatesOrDefault( + long key, + /* nullable */ +org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate defaultValue); + /** + * map<int64, .compaction_coordinator.PCompactionStatusUpdate> updates = 3; + */ + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getUpdatesOrThrow( + long key); + + /** + * .tabletserver.PExternalCompactionJob job = 4; + * @return Whether the job field is set. + */ + boolean hasJob(); + /** + * .tabletserver.PExternalCompactionJob job = 4; + * @return The job. + */ + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob getJob(); + /** + * .tabletserver.PExternalCompactionJob job = 4; + */ + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder getJobOrBuilder(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PFateId.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PFateId.java new file mode 100644 index 00000000000..e24f32b3204 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PFateId.java @@ -0,0 +1,654 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: manager.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code manager.PFateId} + */ +public final class PFateId extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:manager.PFateId) + PFateIdOrBuilder { +private static final long serialVersionUID = 0L; + // Use PFateId.newBuilder() to construct. + private PFateId(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PFateId() { + type_ = 0; + txUUIDStr_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PFateId(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.ManagerProto.internal_static_manager_PFateId_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.ManagerProto.internal_static_manager_PFateId_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PFateId.class, org.apache.accumulo.grpc.compaction.protobuf.PFateId.Builder.class); + } + + public static final int TYPE_FIELD_NUMBER = 1; + private int type_ = 0; + /** + * .manager.PFateInstanceType type = 1; + * @return The enum numeric value on the wire for type. + */ + @java.lang.Override public int getTypeValue() { + return type_; + } + /** + * .manager.PFateInstanceType type = 1; + * @return The type. + */ + @java.lang.Override public org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType getType() { + org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType result = org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType.forNumber(type_); + return result == null ? org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType.UNRECOGNIZED : result; + } + + public static final int TXUUIDSTR_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object txUUIDStr_ = ""; + /** + * string txUUIDStr = 2; + * @return The txUUIDStr. + */ + @java.lang.Override + public java.lang.String getTxUUIDStr() { + java.lang.Object ref = txUUIDStr_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + txUUIDStr_ = s; + return s; + } + } + /** + * string txUUIDStr = 2; + * @return The bytes for txUUIDStr. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getTxUUIDStrBytes() { + java.lang.Object ref = txUUIDStr_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + txUUIDStr_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (type_ != org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType.UNKNOWN.getNumber()) { + output.writeEnum(1, type_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(txUUIDStr_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, txUUIDStr_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (type_ != org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType.UNKNOWN.getNumber()) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(1, type_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(txUUIDStr_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, txUUIDStr_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PFateId)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PFateId other = (org.apache.accumulo.grpc.compaction.protobuf.PFateId) obj; + + if (type_ != other.type_) return false; + if (!getTxUUIDStr() + .equals(other.getTxUUIDStr())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + TYPE_FIELD_NUMBER; + hash = (53 * hash) + type_; + hash = (37 * hash) + TXUUIDSTR_FIELD_NUMBER; + hash = (53 * hash) + getTxUUIDStr().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PFateId prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code manager.PFateId} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:manager.PFateId) + org.apache.accumulo.grpc.compaction.protobuf.PFateIdOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.ManagerProto.internal_static_manager_PFateId_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.ManagerProto.internal_static_manager_PFateId_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PFateId.class, org.apache.accumulo.grpc.compaction.protobuf.PFateId.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PFateId.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + type_ = 0; + txUUIDStr_ = ""; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.ManagerProto.internal_static_manager_PFateId_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PFateId getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PFateId.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PFateId build() { + org.apache.accumulo.grpc.compaction.protobuf.PFateId result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PFateId buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PFateId result = new org.apache.accumulo.grpc.compaction.protobuf.PFateId(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PFateId result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.type_ = type_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.txUUIDStr_ = txUUIDStr_; + } + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PFateId) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PFateId)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PFateId other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PFateId.getDefaultInstance()) return this; + if (other.type_ != 0) { + setTypeValue(other.getTypeValue()); + } + if (!other.getTxUUIDStr().isEmpty()) { + txUUIDStr_ = other.txUUIDStr_; + bitField0_ |= 0x00000002; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + type_ = input.readEnum(); + bitField0_ |= 0x00000001; + break; + } // case 8 + case 18: { + txUUIDStr_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private int type_ = 0; + /** + * .manager.PFateInstanceType type = 1; + * @return The enum numeric value on the wire for type. + */ + @java.lang.Override public int getTypeValue() { + return type_; + } + /** + * .manager.PFateInstanceType type = 1; + * @param value The enum numeric value on the wire for type to set. + * @return This builder for chaining. + */ + public Builder setTypeValue(int value) { + type_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .manager.PFateInstanceType type = 1; + * @return The type. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType getType() { + org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType result = org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType.forNumber(type_); + return result == null ? org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType.UNRECOGNIZED : result; + } + /** + * .manager.PFateInstanceType type = 1; + * @param value The type to set. + * @return This builder for chaining. + */ + public Builder setType(org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + type_ = value.getNumber(); + onChanged(); + return this; + } + /** + * .manager.PFateInstanceType type = 1; + * @return This builder for chaining. + */ + public Builder clearType() { + bitField0_ = (bitField0_ & ~0x00000001); + type_ = 0; + onChanged(); + return this; + } + + private java.lang.Object txUUIDStr_ = ""; + /** + * string txUUIDStr = 2; + * @return The txUUIDStr. + */ + public java.lang.String getTxUUIDStr() { + java.lang.Object ref = txUUIDStr_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + txUUIDStr_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string txUUIDStr = 2; + * @return The bytes for txUUIDStr. + */ + public com.google.protobuf.ByteString + getTxUUIDStrBytes() { + java.lang.Object ref = txUUIDStr_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + txUUIDStr_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string txUUIDStr = 2; + * @param value The txUUIDStr to set. + * @return This builder for chaining. + */ + public Builder setTxUUIDStr( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + txUUIDStr_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string txUUIDStr = 2; + * @return This builder for chaining. + */ + public Builder clearTxUUIDStr() { + txUUIDStr_ = getDefaultInstance().getTxUUIDStr(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string txUUIDStr = 2; + * @param value The bytes for txUUIDStr to set. + * @return This builder for chaining. + */ + public Builder setTxUUIDStrBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + txUUIDStr_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:manager.PFateId) + } + + // @@protoc_insertion_point(class_scope:manager.PFateId) + private static final org.apache.accumulo.grpc.compaction.protobuf.PFateId DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PFateId(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PFateId getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PFateId parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PFateId getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PFateIdOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PFateIdOrBuilder.java new file mode 100644 index 00000000000..728fd610068 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PFateIdOrBuilder.java @@ -0,0 +1,51 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: manager.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PFateIdOrBuilder extends + // @@protoc_insertion_point(interface_extends:manager.PFateId) + com.google.protobuf.MessageOrBuilder { + + /** + * .manager.PFateInstanceType type = 1; + * @return The enum numeric value on the wire for type. + */ + int getTypeValue(); + /** + * .manager.PFateInstanceType type = 1; + * @return The type. + */ + org.apache.accumulo.grpc.compaction.protobuf.PFateInstanceType getType(); + + /** + * string txUUIDStr = 2; + * @return The txUUIDStr. + */ + java.lang.String getTxUUIDStr(); + /** + * string txUUIDStr = 2; + * @return The bytes for txUUIDStr. + */ + com.google.protobuf.ByteString + getTxUUIDStrBytes(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PFateInstanceType.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PFateInstanceType.java new file mode 100644 index 00000000000..2b825e42259 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PFateInstanceType.java @@ -0,0 +1,141 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: manager.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf enum {@code manager.PFateInstanceType} + */ +public enum PFateInstanceType + implements com.google.protobuf.ProtocolMessageEnum { + /** + * UNKNOWN = 0; + */ + UNKNOWN(0), + /** + * META = 1; + */ + META(1), + /** + * USER = 2; + */ + USER(2), + UNRECOGNIZED(-1), + ; + + /** + * UNKNOWN = 0; + */ + public static final int UNKNOWN_VALUE = 0; + /** + * META = 1; + */ + public static final int META_VALUE = 1; + /** + * USER = 2; + */ + public static final int USER_VALUE = 2; + + + public final int getNumber() { + if (this == UNRECOGNIZED) { + throw new java.lang.IllegalArgumentException( + "Can't get the number of an unknown enum value."); + } + return value; + } + + /** + * @param value The numeric wire value of the corresponding enum entry. + * @return The enum associated with the given numeric wire value. + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static PFateInstanceType valueOf(int value) { + return forNumber(value); + } + + /** + * @param value The numeric wire value of the corresponding enum entry. + * @return The enum associated with the given numeric wire value. + */ + public static PFateInstanceType forNumber(int value) { + switch (value) { + case 0: return UNKNOWN; + case 1: return META; + case 2: return USER; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static final com.google.protobuf.Internal.EnumLiteMap< + PFateInstanceType> internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public PFateInstanceType findValueByNumber(int number) { + return PFateInstanceType.forNumber(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + if (this == UNRECOGNIZED) { + throw new java.lang.IllegalStateException( + "Can't get the descriptor of an unrecognized enum value."); + } + return getDescriptor().getValues().get(ordinal()); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.ManagerProto.getDescriptor().getEnumTypes().get(0); + } + + private static final PFateInstanceType[] VALUES = values(); + + public static PFateInstanceType valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + if (desc.getIndex() == -1) { + return UNRECOGNIZED; + } + return VALUES[desc.getIndex()]; + } + + private final int value; + + private PFateInstanceType(int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:manager.PFateInstanceType) +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PInputFile.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PInputFile.java new file mode 100644 index 00000000000..5e432c632bc --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PInputFile.java @@ -0,0 +1,796 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code tabletserver.PInputFile} + */ +public final class PInputFile extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:tabletserver.PInputFile) + PInputFileOrBuilder { +private static final long serialVersionUID = 0L; + // Use PInputFile.newBuilder() to construct. + private PInputFile(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PInputFile() { + metadataFileEntry_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PInputFile(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PInputFile_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PInputFile_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PInputFile.class, org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder.class); + } + + public static final int METADATAFILEENTRY_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object metadataFileEntry_ = ""; + /** + *
+   * This is nullable in Thrift but I am not marking as optional as
+   * I think it should always be set
+   * 
+ * + * string metadataFileEntry = 1; + * @return The metadataFileEntry. + */ + @java.lang.Override + public java.lang.String getMetadataFileEntry() { + java.lang.Object ref = metadataFileEntry_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + metadataFileEntry_ = s; + return s; + } + } + /** + *
+   * This is nullable in Thrift but I am not marking as optional as
+   * I think it should always be set
+   * 
+ * + * string metadataFileEntry = 1; + * @return The bytes for metadataFileEntry. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getMetadataFileEntryBytes() { + java.lang.Object ref = metadataFileEntry_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + metadataFileEntry_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int SIZE_FIELD_NUMBER = 2; + private long size_ = 0L; + /** + * int64 size = 2; + * @return The size. + */ + @java.lang.Override + public long getSize() { + return size_; + } + + public static final int ENTRIES_FIELD_NUMBER = 3; + private long entries_ = 0L; + /** + * int64 entries = 3; + * @return The entries. + */ + @java.lang.Override + public long getEntries() { + return entries_; + } + + public static final int TIMESTAMP_FIELD_NUMBER = 4; + private long timestamp_ = 0L; + /** + * int64 timestamp = 4; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(metadataFileEntry_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, metadataFileEntry_); + } + if (size_ != 0L) { + output.writeInt64(2, size_); + } + if (entries_ != 0L) { + output.writeInt64(3, entries_); + } + if (timestamp_ != 0L) { + output.writeInt64(4, timestamp_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(metadataFileEntry_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, metadataFileEntry_); + } + if (size_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(2, size_); + } + if (entries_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(3, entries_); + } + if (timestamp_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(4, timestamp_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PInputFile)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PInputFile other = (org.apache.accumulo.grpc.compaction.protobuf.PInputFile) obj; + + if (!getMetadataFileEntry() + .equals(other.getMetadataFileEntry())) return false; + if (getSize() + != other.getSize()) return false; + if (getEntries() + != other.getEntries()) return false; + if (getTimestamp() + != other.getTimestamp()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + METADATAFILEENTRY_FIELD_NUMBER; + hash = (53 * hash) + getMetadataFileEntry().hashCode(); + hash = (37 * hash) + SIZE_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getSize()); + hash = (37 * hash) + ENTRIES_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getEntries()); + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getTimestamp()); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PInputFile prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code tabletserver.PInputFile} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:tabletserver.PInputFile) + org.apache.accumulo.grpc.compaction.protobuf.PInputFileOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PInputFile_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PInputFile_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PInputFile.class, org.apache.accumulo.grpc.compaction.protobuf.PInputFile.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PInputFile.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + metadataFileEntry_ = ""; + size_ = 0L; + entries_ = 0L; + timestamp_ = 0L; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PInputFile_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PInputFile getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PInputFile.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PInputFile build() { + org.apache.accumulo.grpc.compaction.protobuf.PInputFile result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PInputFile buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PInputFile result = new org.apache.accumulo.grpc.compaction.protobuf.PInputFile(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PInputFile result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.metadataFileEntry_ = metadataFileEntry_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.size_ = size_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.entries_ = entries_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.timestamp_ = timestamp_; + } + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PInputFile) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PInputFile)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PInputFile other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PInputFile.getDefaultInstance()) return this; + if (!other.getMetadataFileEntry().isEmpty()) { + metadataFileEntry_ = other.metadataFileEntry_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (other.getSize() != 0L) { + setSize(other.getSize()); + } + if (other.getEntries() != 0L) { + setEntries(other.getEntries()); + } + if (other.getTimestamp() != 0L) { + setTimestamp(other.getTimestamp()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + metadataFileEntry_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 16: { + size_ = input.readInt64(); + bitField0_ |= 0x00000002; + break; + } // case 16 + case 24: { + entries_ = input.readInt64(); + bitField0_ |= 0x00000004; + break; + } // case 24 + case 32: { + timestamp_ = input.readInt64(); + bitField0_ |= 0x00000008; + break; + } // case 32 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object metadataFileEntry_ = ""; + /** + *
+     * This is nullable in Thrift but I am not marking as optional as
+     * I think it should always be set
+     * 
+ * + * string metadataFileEntry = 1; + * @return The metadataFileEntry. + */ + public java.lang.String getMetadataFileEntry() { + java.lang.Object ref = metadataFileEntry_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + metadataFileEntry_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + *
+     * This is nullable in Thrift but I am not marking as optional as
+     * I think it should always be set
+     * 
+ * + * string metadataFileEntry = 1; + * @return The bytes for metadataFileEntry. + */ + public com.google.protobuf.ByteString + getMetadataFileEntryBytes() { + java.lang.Object ref = metadataFileEntry_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + metadataFileEntry_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + *
+     * This is nullable in Thrift but I am not marking as optional as
+     * I think it should always be set
+     * 
+ * + * string metadataFileEntry = 1; + * @param value The metadataFileEntry to set. + * @return This builder for chaining. + */ + public Builder setMetadataFileEntry( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + metadataFileEntry_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + *
+     * This is nullable in Thrift but I am not marking as optional as
+     * I think it should always be set
+     * 
+ * + * string metadataFileEntry = 1; + * @return This builder for chaining. + */ + public Builder clearMetadataFileEntry() { + metadataFileEntry_ = getDefaultInstance().getMetadataFileEntry(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + *
+     * This is nullable in Thrift but I am not marking as optional as
+     * I think it should always be set
+     * 
+ * + * string metadataFileEntry = 1; + * @param value The bytes for metadataFileEntry to set. + * @return This builder for chaining. + */ + public Builder setMetadataFileEntryBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + metadataFileEntry_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private long size_ ; + /** + * int64 size = 2; + * @return The size. + */ + @java.lang.Override + public long getSize() { + return size_; + } + /** + * int64 size = 2; + * @param value The size to set. + * @return This builder for chaining. + */ + public Builder setSize(long value) { + + size_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * int64 size = 2; + * @return This builder for chaining. + */ + public Builder clearSize() { + bitField0_ = (bitField0_ & ~0x00000002); + size_ = 0L; + onChanged(); + return this; + } + + private long entries_ ; + /** + * int64 entries = 3; + * @return The entries. + */ + @java.lang.Override + public long getEntries() { + return entries_; + } + /** + * int64 entries = 3; + * @param value The entries to set. + * @return This builder for chaining. + */ + public Builder setEntries(long value) { + + entries_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * int64 entries = 3; + * @return This builder for chaining. + */ + public Builder clearEntries() { + bitField0_ = (bitField0_ & ~0x00000004); + entries_ = 0L; + onChanged(); + return this; + } + + private long timestamp_ ; + /** + * int64 timestamp = 4; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + /** + * int64 timestamp = 4; + * @param value The timestamp to set. + * @return This builder for chaining. + */ + public Builder setTimestamp(long value) { + + timestamp_ = value; + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * int64 timestamp = 4; + * @return This builder for chaining. + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000008); + timestamp_ = 0L; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:tabletserver.PInputFile) + } + + // @@protoc_insertion_point(class_scope:tabletserver.PInputFile) + private static final org.apache.accumulo.grpc.compaction.protobuf.PInputFile DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PInputFile(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PInputFile getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PInputFile parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PInputFile getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PInputFileOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PInputFileOrBuilder.java new file mode 100644 index 00000000000..75f9129462d --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PInputFileOrBuilder.java @@ -0,0 +1,68 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PInputFileOrBuilder extends + // @@protoc_insertion_point(interface_extends:tabletserver.PInputFile) + com.google.protobuf.MessageOrBuilder { + + /** + *
+   * This is nullable in Thrift but I am not marking as optional as
+   * I think it should always be set
+   * 
+ * + * string metadataFileEntry = 1; + * @return The metadataFileEntry. + */ + java.lang.String getMetadataFileEntry(); + /** + *
+   * This is nullable in Thrift but I am not marking as optional as
+   * I think it should always be set
+   * 
+ * + * string metadataFileEntry = 1; + * @return The bytes for metadataFileEntry. + */ + com.google.protobuf.ByteString + getMetadataFileEntryBytes(); + + /** + * int64 size = 2; + * @return The size. + */ + long getSize(); + + /** + * int64 entries = 3; + * @return The entries. + */ + long getEntries(); + + /** + * int64 timestamp = 4; + * @return The timestamp. + */ + long getTimestamp(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorConfig.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorConfig.java new file mode 100644 index 00000000000..fa4af3ad349 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorConfig.java @@ -0,0 +1,778 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code tabletserver.PIteratorConfig} + */ +public final class PIteratorConfig extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:tabletserver.PIteratorConfig) + PIteratorConfigOrBuilder { +private static final long serialVersionUID = 0L; + // Use PIteratorConfig.newBuilder() to construct. + private PIteratorConfig(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PIteratorConfig() { + iterators_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PIteratorConfig(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorConfig_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorConfig_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.class, org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.Builder.class); + } + + public static final int ITERATORS_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private java.util.List iterators_; + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + @java.lang.Override + public java.util.List getIteratorsList() { + return iterators_; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + @java.lang.Override + public java.util.List + getIteratorsOrBuilderList() { + return iterators_; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + @java.lang.Override + public int getIteratorsCount() { + return iterators_.size(); + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting getIterators(int index) { + return iterators_.get(index); + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSettingOrBuilder getIteratorsOrBuilder( + int index) { + return iterators_.get(index); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + for (int i = 0; i < iterators_.size(); i++) { + output.writeMessage(1, iterators_.get(i)); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < iterators_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, iterators_.get(i)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig other = (org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig) obj; + + if (!getIteratorsList() + .equals(other.getIteratorsList())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (getIteratorsCount() > 0) { + hash = (37 * hash) + ITERATORS_FIELD_NUMBER; + hash = (53 * hash) + getIteratorsList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code tabletserver.PIteratorConfig} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:tabletserver.PIteratorConfig) + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfigOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorConfig_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorConfig_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.class, org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + if (iteratorsBuilder_ == null) { + iterators_ = java.util.Collections.emptyList(); + } else { + iterators_ = null; + iteratorsBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorConfig_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig build() { + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig result = new org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig(this); + buildPartialRepeatedFields(result); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartialRepeatedFields(org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig result) { + if (iteratorsBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0)) { + iterators_ = java.util.Collections.unmodifiableList(iterators_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.iterators_ = iterators_; + } else { + result.iterators_ = iteratorsBuilder_.build(); + } + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig result) { + int from_bitField0_ = bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig.getDefaultInstance()) return this; + if (iteratorsBuilder_ == null) { + if (!other.iterators_.isEmpty()) { + if (iterators_.isEmpty()) { + iterators_ = other.iterators_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureIteratorsIsMutable(); + iterators_.addAll(other.iterators_); + } + onChanged(); + } + } else { + if (!other.iterators_.isEmpty()) { + if (iteratorsBuilder_.isEmpty()) { + iteratorsBuilder_.dispose(); + iteratorsBuilder_ = null; + iterators_ = other.iterators_; + bitField0_ = (bitField0_ & ~0x00000001); + iteratorsBuilder_ = + com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getIteratorsFieldBuilder() : null; + } else { + iteratorsBuilder_.addAllMessages(other.iterators_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting m = + input.readMessage( + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.parser(), + extensionRegistry); + if (iteratorsBuilder_ == null) { + ensureIteratorsIsMutable(); + iterators_.add(m); + } else { + iteratorsBuilder_.addMessage(m); + } + break; + } // case 10 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.util.List iterators_ = + java.util.Collections.emptyList(); + private void ensureIteratorsIsMutable() { + if (!((bitField0_ & 0x00000001) != 0)) { + iterators_ = new java.util.ArrayList(iterators_); + bitField0_ |= 0x00000001; + } + } + + private com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSettingOrBuilder> iteratorsBuilder_; + + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public java.util.List getIteratorsList() { + if (iteratorsBuilder_ == null) { + return java.util.Collections.unmodifiableList(iterators_); + } else { + return iteratorsBuilder_.getMessageList(); + } + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public int getIteratorsCount() { + if (iteratorsBuilder_ == null) { + return iterators_.size(); + } else { + return iteratorsBuilder_.getCount(); + } + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting getIterators(int index) { + if (iteratorsBuilder_ == null) { + return iterators_.get(index); + } else { + return iteratorsBuilder_.getMessage(index); + } + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public Builder setIterators( + int index, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting value) { + if (iteratorsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureIteratorsIsMutable(); + iterators_.set(index, value); + onChanged(); + } else { + iteratorsBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public Builder setIterators( + int index, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder builderForValue) { + if (iteratorsBuilder_ == null) { + ensureIteratorsIsMutable(); + iterators_.set(index, builderForValue.build()); + onChanged(); + } else { + iteratorsBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public Builder addIterators(org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting value) { + if (iteratorsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureIteratorsIsMutable(); + iterators_.add(value); + onChanged(); + } else { + iteratorsBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public Builder addIterators( + int index, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting value) { + if (iteratorsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureIteratorsIsMutable(); + iterators_.add(index, value); + onChanged(); + } else { + iteratorsBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public Builder addIterators( + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder builderForValue) { + if (iteratorsBuilder_ == null) { + ensureIteratorsIsMutable(); + iterators_.add(builderForValue.build()); + onChanged(); + } else { + iteratorsBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public Builder addIterators( + int index, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder builderForValue) { + if (iteratorsBuilder_ == null) { + ensureIteratorsIsMutable(); + iterators_.add(index, builderForValue.build()); + onChanged(); + } else { + iteratorsBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public Builder addAllIterators( + java.lang.Iterable values) { + if (iteratorsBuilder_ == null) { + ensureIteratorsIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, iterators_); + onChanged(); + } else { + iteratorsBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public Builder clearIterators() { + if (iteratorsBuilder_ == null) { + iterators_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + iteratorsBuilder_.clear(); + } + return this; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public Builder removeIterators(int index) { + if (iteratorsBuilder_ == null) { + ensureIteratorsIsMutable(); + iterators_.remove(index); + onChanged(); + } else { + iteratorsBuilder_.remove(index); + } + return this; + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder getIteratorsBuilder( + int index) { + return getIteratorsFieldBuilder().getBuilder(index); + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSettingOrBuilder getIteratorsOrBuilder( + int index) { + if (iteratorsBuilder_ == null) { + return iterators_.get(index); } else { + return iteratorsBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public java.util.List + getIteratorsOrBuilderList() { + if (iteratorsBuilder_ != null) { + return iteratorsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(iterators_); + } + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder addIteratorsBuilder() { + return getIteratorsFieldBuilder().addBuilder( + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.getDefaultInstance()); + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder addIteratorsBuilder( + int index) { + return getIteratorsFieldBuilder().addBuilder( + index, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.getDefaultInstance()); + } + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + public java.util.List + getIteratorsBuilderList() { + return getIteratorsFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSettingOrBuilder> + getIteratorsFieldBuilder() { + if (iteratorsBuilder_ == null) { + iteratorsBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSettingOrBuilder>( + iterators_, + ((bitField0_ & 0x00000001) != 0), + getParentForChildren(), + isClean()); + iterators_ = null; + } + return iteratorsBuilder_; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:tabletserver.PIteratorConfig) + } + + // @@protoc_insertion_point(class_scope:tabletserver.PIteratorConfig) + private static final org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PIteratorConfig parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorConfigOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorConfigOrBuilder.java new file mode 100644 index 00000000000..63654e2b8bc --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorConfigOrBuilder.java @@ -0,0 +1,52 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PIteratorConfigOrBuilder extends + // @@protoc_insertion_point(interface_extends:tabletserver.PIteratorConfig) + com.google.protobuf.MessageOrBuilder { + + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + java.util.List + getIteratorsList(); + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting getIterators(int index); + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + int getIteratorsCount(); + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + java.util.List + getIteratorsOrBuilderList(); + /** + * repeated .tabletserver.PIteratorSetting iterators = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSettingOrBuilder getIteratorsOrBuilder( + int index); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorSetting.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorSetting.java new file mode 100644 index 00000000000..d9f6ddbb5d7 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorSetting.java @@ -0,0 +1,1041 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code tabletserver.PIteratorSetting} + */ +public final class PIteratorSetting extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:tabletserver.PIteratorSetting) + PIteratorSettingOrBuilder { +private static final long serialVersionUID = 0L; + // Use PIteratorSetting.newBuilder() to construct. + private PIteratorSetting(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PIteratorSetting() { + name_ = ""; + iteratorClass_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PIteratorSetting(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorSetting_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + @java.lang.Override + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMapFieldReflection( + int number) { + switch (number) { + case 4: + return internalGetProperties(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorSetting_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.class, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder.class); + } + + public static final int PRIORITY_FIELD_NUMBER = 1; + private int priority_ = 0; + /** + * int32 priority = 1; + * @return The priority. + */ + @java.lang.Override + public int getPriority() { + return priority_; + } + + public static final int NAME_FIELD_NUMBER = 2; + @SuppressWarnings("serial") + private volatile java.lang.Object name_ = ""; + /** + * string name = 2; + * @return The name. + */ + @java.lang.Override + public java.lang.String getName() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + name_ = s; + return s; + } + } + /** + * string name = 2; + * @return The bytes for name. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int ITERATORCLASS_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object iteratorClass_ = ""; + /** + * string iteratorClass = 3; + * @return The iteratorClass. + */ + @java.lang.Override + public java.lang.String getIteratorClass() { + java.lang.Object ref = iteratorClass_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + iteratorClass_ = s; + return s; + } + } + /** + * string iteratorClass = 3; + * @return The bytes for iteratorClass. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getIteratorClassBytes() { + java.lang.Object ref = iteratorClass_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + iteratorClass_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int PROPERTIES_FIELD_NUMBER = 4; + private static final class PropertiesDefaultEntryHolder { + static final com.google.protobuf.MapEntry< + java.lang.String, java.lang.String> defaultEntry = + com.google.protobuf.MapEntry + .newDefaultInstance( + org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorSetting_PropertiesEntry_descriptor, + com.google.protobuf.WireFormat.FieldType.STRING, + "", + com.google.protobuf.WireFormat.FieldType.STRING, + ""); + } + @SuppressWarnings("serial") + private com.google.protobuf.MapField< + java.lang.String, java.lang.String> properties_; + private com.google.protobuf.MapField + internalGetProperties() { + if (properties_ == null) { + return com.google.protobuf.MapField.emptyMapField( + PropertiesDefaultEntryHolder.defaultEntry); + } + return properties_; + } + public int getPropertiesCount() { + return internalGetProperties().getMap().size(); + } + /** + * map<string, string> properties = 4; + */ + @java.lang.Override + public boolean containsProperties( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + return internalGetProperties().getMap().containsKey(key); + } + /** + * Use {@link #getPropertiesMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getProperties() { + return getPropertiesMap(); + } + /** + * map<string, string> properties = 4; + */ + @java.lang.Override + public java.util.Map getPropertiesMap() { + return internalGetProperties().getMap(); + } + /** + * map<string, string> properties = 4; + */ + @java.lang.Override + public /* nullable */ +java.lang.String getPropertiesOrDefault( + java.lang.String key, + /* nullable */ +java.lang.String defaultValue) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetProperties().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + * map<string, string> properties = 4; + */ + @java.lang.Override + public java.lang.String getPropertiesOrThrow( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetProperties().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (priority_ != 0) { + output.writeInt32(1, priority_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(name_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, name_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(iteratorClass_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 3, iteratorClass_); + } + com.google.protobuf.GeneratedMessageV3 + .serializeStringMapTo( + output, + internalGetProperties(), + PropertiesDefaultEntryHolder.defaultEntry, + 4); + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (priority_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(1, priority_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(name_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, name_); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(iteratorClass_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, iteratorClass_); + } + for (java.util.Map.Entry entry + : internalGetProperties().getMap().entrySet()) { + com.google.protobuf.MapEntry + properties__ = PropertiesDefaultEntryHolder.defaultEntry.newBuilderForType() + .setKey(entry.getKey()) + .setValue(entry.getValue()) + .build(); + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(4, properties__); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting other = (org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting) obj; + + if (getPriority() + != other.getPriority()) return false; + if (!getName() + .equals(other.getName())) return false; + if (!getIteratorClass() + .equals(other.getIteratorClass())) return false; + if (!internalGetProperties().equals( + other.internalGetProperties())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + PRIORITY_FIELD_NUMBER; + hash = (53 * hash) + getPriority(); + hash = (37 * hash) + NAME_FIELD_NUMBER; + hash = (53 * hash) + getName().hashCode(); + hash = (37 * hash) + ITERATORCLASS_FIELD_NUMBER; + hash = (53 * hash) + getIteratorClass().hashCode(); + if (!internalGetProperties().getMap().isEmpty()) { + hash = (37 * hash) + PROPERTIES_FIELD_NUMBER; + hash = (53 * hash) + internalGetProperties().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code tabletserver.PIteratorSetting} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:tabletserver.PIteratorSetting) + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSettingOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorSetting_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMapFieldReflection( + int number) { + switch (number) { + case 4: + return internalGetProperties(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMutableMapFieldReflection( + int number) { + switch (number) { + case 4: + return internalGetMutableProperties(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorSetting_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.class, org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + priority_ = 0; + name_ = ""; + iteratorClass_ = ""; + internalGetMutableProperties().clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.TabletServerProto.internal_static_tabletserver_PIteratorSetting_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting build() { + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting result = new org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.priority_ = priority_; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.name_ = name_; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.iteratorClass_ = iteratorClass_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.properties_ = internalGetProperties(); + result.properties_.makeImmutable(); + } + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting.getDefaultInstance()) return this; + if (other.getPriority() != 0) { + setPriority(other.getPriority()); + } + if (!other.getName().isEmpty()) { + name_ = other.name_; + bitField0_ |= 0x00000002; + onChanged(); + } + if (!other.getIteratorClass().isEmpty()) { + iteratorClass_ = other.iteratorClass_; + bitField0_ |= 0x00000004; + onChanged(); + } + internalGetMutableProperties().mergeFrom( + other.internalGetProperties()); + bitField0_ |= 0x00000008; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + priority_ = input.readInt32(); + bitField0_ |= 0x00000001; + break; + } // case 8 + case 18: { + name_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + iteratorClass_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + com.google.protobuf.MapEntry + properties__ = input.readMessage( + PropertiesDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry); + internalGetMutableProperties().getMutableMap().put( + properties__.getKey(), properties__.getValue()); + bitField0_ |= 0x00000008; + break; + } // case 34 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private int priority_ ; + /** + * int32 priority = 1; + * @return The priority. + */ + @java.lang.Override + public int getPriority() { + return priority_; + } + /** + * int32 priority = 1; + * @param value The priority to set. + * @return This builder for chaining. + */ + public Builder setPriority(int value) { + + priority_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * int32 priority = 1; + * @return This builder for chaining. + */ + public Builder clearPriority() { + bitField0_ = (bitField0_ & ~0x00000001); + priority_ = 0; + onChanged(); + return this; + } + + private java.lang.Object name_ = ""; + /** + * string name = 2; + * @return The name. + */ + public java.lang.String getName() { + java.lang.Object ref = name_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + name_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string name = 2; + * @return The bytes for name. + */ + public com.google.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string name = 2; + * @param value The name to set. + * @return This builder for chaining. + */ + public Builder setName( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + name_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * string name = 2; + * @return This builder for chaining. + */ + public Builder clearName() { + name_ = getDefaultInstance().getName(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + /** + * string name = 2; + * @param value The bytes for name to set. + * @return This builder for chaining. + */ + public Builder setNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + name_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + + private java.lang.Object iteratorClass_ = ""; + /** + * string iteratorClass = 3; + * @return The iteratorClass. + */ + public java.lang.String getIteratorClass() { + java.lang.Object ref = iteratorClass_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + iteratorClass_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string iteratorClass = 3; + * @return The bytes for iteratorClass. + */ + public com.google.protobuf.ByteString + getIteratorClassBytes() { + java.lang.Object ref = iteratorClass_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + iteratorClass_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string iteratorClass = 3; + * @param value The iteratorClass to set. + * @return This builder for chaining. + */ + public Builder setIteratorClass( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + iteratorClass_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string iteratorClass = 3; + * @return This builder for chaining. + */ + public Builder clearIteratorClass() { + iteratorClass_ = getDefaultInstance().getIteratorClass(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string iteratorClass = 3; + * @param value The bytes for iteratorClass to set. + * @return This builder for chaining. + */ + public Builder setIteratorClassBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + iteratorClass_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private com.google.protobuf.MapField< + java.lang.String, java.lang.String> properties_; + private com.google.protobuf.MapField + internalGetProperties() { + if (properties_ == null) { + return com.google.protobuf.MapField.emptyMapField( + PropertiesDefaultEntryHolder.defaultEntry); + } + return properties_; + } + private com.google.protobuf.MapField + internalGetMutableProperties() { + if (properties_ == null) { + properties_ = com.google.protobuf.MapField.newMapField( + PropertiesDefaultEntryHolder.defaultEntry); + } + if (!properties_.isMutable()) { + properties_ = properties_.copy(); + } + bitField0_ |= 0x00000008; + onChanged(); + return properties_; + } + public int getPropertiesCount() { + return internalGetProperties().getMap().size(); + } + /** + * map<string, string> properties = 4; + */ + @java.lang.Override + public boolean containsProperties( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + return internalGetProperties().getMap().containsKey(key); + } + /** + * Use {@link #getPropertiesMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getProperties() { + return getPropertiesMap(); + } + /** + * map<string, string> properties = 4; + */ + @java.lang.Override + public java.util.Map getPropertiesMap() { + return internalGetProperties().getMap(); + } + /** + * map<string, string> properties = 4; + */ + @java.lang.Override + public /* nullable */ +java.lang.String getPropertiesOrDefault( + java.lang.String key, + /* nullable */ +java.lang.String defaultValue) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetProperties().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + * map<string, string> properties = 4; + */ + @java.lang.Override + public java.lang.String getPropertiesOrThrow( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetProperties().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + public Builder clearProperties() { + bitField0_ = (bitField0_ & ~0x00000008); + internalGetMutableProperties().getMutableMap() + .clear(); + return this; + } + /** + * map<string, string> properties = 4; + */ + public Builder removeProperties( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + internalGetMutableProperties().getMutableMap() + .remove(key); + return this; + } + /** + * Use alternate mutation accessors instead. + */ + @java.lang.Deprecated + public java.util.Map + getMutableProperties() { + bitField0_ |= 0x00000008; + return internalGetMutableProperties().getMutableMap(); + } + /** + * map<string, string> properties = 4; + */ + public Builder putProperties( + java.lang.String key, + java.lang.String value) { + if (key == null) { throw new NullPointerException("map key"); } + if (value == null) { throw new NullPointerException("map value"); } + internalGetMutableProperties().getMutableMap() + .put(key, value); + bitField0_ |= 0x00000008; + return this; + } + /** + * map<string, string> properties = 4; + */ + public Builder putAllProperties( + java.util.Map values) { + internalGetMutableProperties().getMutableMap() + .putAll(values); + bitField0_ |= 0x00000008; + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:tabletserver.PIteratorSetting) + } + + // @@protoc_insertion_point(class_scope:tabletserver.PIteratorSetting) + private static final org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PIteratorSetting parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PIteratorSetting getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorSettingOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorSettingOrBuilder.java new file mode 100644 index 00000000000..3b0f568d7f7 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PIteratorSettingOrBuilder.java @@ -0,0 +1,92 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PIteratorSettingOrBuilder extends + // @@protoc_insertion_point(interface_extends:tabletserver.PIteratorSetting) + com.google.protobuf.MessageOrBuilder { + + /** + * int32 priority = 1; + * @return The priority. + */ + int getPriority(); + + /** + * string name = 2; + * @return The name. + */ + java.lang.String getName(); + /** + * string name = 2; + * @return The bytes for name. + */ + com.google.protobuf.ByteString + getNameBytes(); + + /** + * string iteratorClass = 3; + * @return The iteratorClass. + */ + java.lang.String getIteratorClass(); + /** + * string iteratorClass = 3; + * @return The bytes for iteratorClass. + */ + com.google.protobuf.ByteString + getIteratorClassBytes(); + + /** + * map<string, string> properties = 4; + */ + int getPropertiesCount(); + /** + * map<string, string> properties = 4; + */ + boolean containsProperties( + java.lang.String key); + /** + * Use {@link #getPropertiesMap()} instead. + */ + @java.lang.Deprecated + java.util.Map + getProperties(); + /** + * map<string, string> properties = 4; + */ + java.util.Map + getPropertiesMap(); + /** + * map<string, string> properties = 4; + */ + /* nullable */ +java.lang.String getPropertiesOrDefault( + java.lang.String key, + /* nullable */ +java.lang.String defaultValue); + /** + * map<string, string> properties = 4; + */ + java.lang.String getPropertiesOrThrow( + java.lang.String key); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PKeyExtent.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PKeyExtent.java new file mode 100644 index 00000000000..7e9aa4e6d39 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PKeyExtent.java @@ -0,0 +1,799 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: data.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code data.PKeyExtent} + */ +public final class PKeyExtent extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:data.PKeyExtent) + PKeyExtentOrBuilder { +private static final long serialVersionUID = 0L; + // Use PKeyExtent.newBuilder() to construct. + private PKeyExtent(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PKeyExtent() { + table_ = ""; + endRow_ = com.google.protobuf.ByteString.EMPTY; + prevEndRow_ = com.google.protobuf.ByteString.EMPTY; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PKeyExtent(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.DataProto.internal_static_data_PKeyExtent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.DataProto.internal_static_data_PKeyExtent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.class, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder.class); + } + + private int bitField0_; + public static final int TABLE_FIELD_NUMBER = 1; + @SuppressWarnings("serial") + private volatile java.lang.Object table_ = ""; + /** + *
+   * Use a string to simplify things (thrift uses a byte array)
+   * 
+ * + * string table = 1; + * @return The table. + */ + @java.lang.Override + public java.lang.String getTable() { + java.lang.Object ref = table_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + table_ = s; + return s; + } + } + /** + *
+   * Use a string to simplify things (thrift uses a byte array)
+   * 
+ * + * string table = 1; + * @return The bytes for table. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getTableBytes() { + java.lang.Object ref = table_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + table_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int ENDROW_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString endRow_ = com.google.protobuf.ByteString.EMPTY; + /** + *
+   * optional to generate hasEndRow() and hasPrevEndRow() because
+   * of no nulls in protobuf
+   * 
+ * + * optional bytes endRow = 2; + * @return Whether the endRow field is set. + */ + @java.lang.Override + public boolean hasEndRow() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + *
+   * optional to generate hasEndRow() and hasPrevEndRow() because
+   * of no nulls in protobuf
+   * 
+ * + * optional bytes endRow = 2; + * @return The endRow. + */ + @java.lang.Override + public com.google.protobuf.ByteString getEndRow() { + return endRow_; + } + + public static final int PREVENDROW_FIELD_NUMBER = 3; + private com.google.protobuf.ByteString prevEndRow_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes prevEndRow = 3; + * @return Whether the prevEndRow field is set. + */ + @java.lang.Override + public boolean hasPrevEndRow() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * optional bytes prevEndRow = 3; + * @return The prevEndRow. + */ + @java.lang.Override + public com.google.protobuf.ByteString getPrevEndRow() { + return prevEndRow_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(table_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, table_); + } + if (((bitField0_ & 0x00000001) != 0)) { + output.writeBytes(2, endRow_); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeBytes(3, prevEndRow_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(table_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, table_); + } + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, endRow_); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, prevEndRow_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent other = (org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent) obj; + + if (!getTable() + .equals(other.getTable())) return false; + if (hasEndRow() != other.hasEndRow()) return false; + if (hasEndRow()) { + if (!getEndRow() + .equals(other.getEndRow())) return false; + } + if (hasPrevEndRow() != other.hasPrevEndRow()) return false; + if (hasPrevEndRow()) { + if (!getPrevEndRow() + .equals(other.getPrevEndRow())) return false; + } + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + hash = (37 * hash) + TABLE_FIELD_NUMBER; + hash = (53 * hash) + getTable().hashCode(); + if (hasEndRow()) { + hash = (37 * hash) + ENDROW_FIELD_NUMBER; + hash = (53 * hash) + getEndRow().hashCode(); + } + if (hasPrevEndRow()) { + hash = (37 * hash) + PREVENDROW_FIELD_NUMBER; + hash = (53 * hash) + getPrevEndRow().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code data.PKeyExtent} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:data.PKeyExtent) + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtentOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.DataProto.internal_static_data_PKeyExtent_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.DataProto.internal_static_data_PKeyExtent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.class, org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + table_ = ""; + endRow_ = com.google.protobuf.ByteString.EMPTY; + prevEndRow_ = com.google.protobuf.ByteString.EMPTY; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.DataProto.internal_static_data_PKeyExtent_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent build() { + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent result = new org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.table_ = table_; + } + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000002) != 0)) { + result.endRow_ = endRow_; + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.prevEndRow_ = prevEndRow_; + to_bitField0_ |= 0x00000002; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent.getDefaultInstance()) return this; + if (!other.getTable().isEmpty()) { + table_ = other.table_; + bitField0_ |= 0x00000001; + onChanged(); + } + if (other.hasEndRow()) { + setEndRow(other.getEndRow()); + } + if (other.hasPrevEndRow()) { + setPrevEndRow(other.getPrevEndRow()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + table_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + endRow_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + prevEndRow_ = input.readBytes(); + bitField0_ |= 0x00000004; + break; + } // case 26 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private java.lang.Object table_ = ""; + /** + *
+     * Use a string to simplify things (thrift uses a byte array)
+     * 
+ * + * string table = 1; + * @return The table. + */ + public java.lang.String getTable() { + java.lang.Object ref = table_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + table_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + *
+     * Use a string to simplify things (thrift uses a byte array)
+     * 
+ * + * string table = 1; + * @return The bytes for table. + */ + public com.google.protobuf.ByteString + getTableBytes() { + java.lang.Object ref = table_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + table_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + *
+     * Use a string to simplify things (thrift uses a byte array)
+     * 
+ * + * string table = 1; + * @param value The table to set. + * @return This builder for chaining. + */ + public Builder setTable( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + table_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + *
+     * Use a string to simplify things (thrift uses a byte array)
+     * 
+ * + * string table = 1; + * @return This builder for chaining. + */ + public Builder clearTable() { + table_ = getDefaultInstance().getTable(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + return this; + } + /** + *
+     * Use a string to simplify things (thrift uses a byte array)
+     * 
+ * + * string table = 1; + * @param value The bytes for table to set. + * @return This builder for chaining. + */ + public Builder setTableBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + table_ = value; + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + + private com.google.protobuf.ByteString endRow_ = com.google.protobuf.ByteString.EMPTY; + /** + *
+     * optional to generate hasEndRow() and hasPrevEndRow() because
+     * of no nulls in protobuf
+     * 
+ * + * optional bytes endRow = 2; + * @return Whether the endRow field is set. + */ + @java.lang.Override + public boolean hasEndRow() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + *
+     * optional to generate hasEndRow() and hasPrevEndRow() because
+     * of no nulls in protobuf
+     * 
+ * + * optional bytes endRow = 2; + * @return The endRow. + */ + @java.lang.Override + public com.google.protobuf.ByteString getEndRow() { + return endRow_; + } + /** + *
+     * optional to generate hasEndRow() and hasPrevEndRow() because
+     * of no nulls in protobuf
+     * 
+ * + * optional bytes endRow = 2; + * @param value The endRow to set. + * @return This builder for chaining. + */ + public Builder setEndRow(com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + endRow_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + *
+     * optional to generate hasEndRow() and hasPrevEndRow() because
+     * of no nulls in protobuf
+     * 
+ * + * optional bytes endRow = 2; + * @return This builder for chaining. + */ + public Builder clearEndRow() { + bitField0_ = (bitField0_ & ~0x00000002); + endRow_ = getDefaultInstance().getEndRow(); + onChanged(); + return this; + } + + private com.google.protobuf.ByteString prevEndRow_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes prevEndRow = 3; + * @return Whether the prevEndRow field is set. + */ + @java.lang.Override + public boolean hasPrevEndRow() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * optional bytes prevEndRow = 3; + * @return The prevEndRow. + */ + @java.lang.Override + public com.google.protobuf.ByteString getPrevEndRow() { + return prevEndRow_; + } + /** + * optional bytes prevEndRow = 3; + * @param value The prevEndRow to set. + * @return This builder for chaining. + */ + public Builder setPrevEndRow(com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + prevEndRow_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * optional bytes prevEndRow = 3; + * @return This builder for chaining. + */ + public Builder clearPrevEndRow() { + bitField0_ = (bitField0_ & ~0x00000004); + prevEndRow_ = getDefaultInstance().getPrevEndRow(); + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:data.PKeyExtent) + } + + // @@protoc_insertion_point(class_scope:data.PKeyExtent) + private static final org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PKeyExtent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PKeyExtent getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PKeyExtentOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PKeyExtentOrBuilder.java new file mode 100644 index 00000000000..f997b835f9a --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PKeyExtentOrBuilder.java @@ -0,0 +1,80 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: data.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PKeyExtentOrBuilder extends + // @@protoc_insertion_point(interface_extends:data.PKeyExtent) + com.google.protobuf.MessageOrBuilder { + + /** + *
+   * Use a string to simplify things (thrift uses a byte array)
+   * 
+ * + * string table = 1; + * @return The table. + */ + java.lang.String getTable(); + /** + *
+   * Use a string to simplify things (thrift uses a byte array)
+   * 
+ * + * string table = 1; + * @return The bytes for table. + */ + com.google.protobuf.ByteString + getTableBytes(); + + /** + *
+   * optional to generate hasEndRow() and hasPrevEndRow() because
+   * of no nulls in protobuf
+   * 
+ * + * optional bytes endRow = 2; + * @return Whether the endRow field is set. + */ + boolean hasEndRow(); + /** + *
+   * optional to generate hasEndRow() and hasPrevEndRow() because
+   * of no nulls in protobuf
+   * 
+ * + * optional bytes endRow = 2; + * @return The endRow. + */ + com.google.protobuf.ByteString getEndRow(); + + /** + * optional bytes prevEndRow = 3; + * @return Whether the prevEndRow field is set. + */ + boolean hasPrevEndRow(); + /** + * optional bytes prevEndRow = 3; + * @return The prevEndRow. + */ + com.google.protobuf.ByteString getPrevEndRow(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PNextCompactionJob.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PNextCompactionJob.java new file mode 100644 index 00000000000..586362f4534 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PNextCompactionJob.java @@ -0,0 +1,683 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.PNextCompactionJob} + */ +public final class PNextCompactionJob extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.PNextCompactionJob) + PNextCompactionJobOrBuilder { +private static final long serialVersionUID = 0L; + // Use PNextCompactionJob.newBuilder() to construct. + private PNextCompactionJob(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private PNextCompactionJob() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new PNextCompactionJob(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PNextCompactionJob_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PNextCompactionJob_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob.class, org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob.Builder.class); + } + + private int bitField0_; + public static final int JOB_FIELD_NUMBER = 1; + private org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob job_; + /** + * .tabletserver.PExternalCompactionJob job = 1; + * @return Whether the job field is set. + */ + @java.lang.Override + public boolean hasJob() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .tabletserver.PExternalCompactionJob job = 1; + * @return The job. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob getJob() { + return job_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance() : job_; + } + /** + * .tabletserver.PExternalCompactionJob job = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder getJobOrBuilder() { + return job_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance() : job_; + } + + public static final int COMPACTORCOUNT_FIELD_NUMBER = 2; + private int compactorCount_ = 0; + /** + * int32 compactorCount = 2; + * @return The compactorCount. + */ + @java.lang.Override + public int getCompactorCount() { + return compactorCount_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(1, getJob()); + } + if (compactorCount_ != 0) { + output.writeInt32(2, compactorCount_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getJob()); + } + if (compactorCount_ != 0) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(2, compactorCount_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob other = (org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob) obj; + + if (hasJob() != other.hasJob()) return false; + if (hasJob()) { + if (!getJob() + .equals(other.getJob())) return false; + } + if (getCompactorCount() + != other.getCompactorCount()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasJob()) { + hash = (37 * hash) + JOB_FIELD_NUMBER; + hash = (53 * hash) + getJob().hashCode(); + } + hash = (37 * hash) + COMPACTORCOUNT_FIELD_NUMBER; + hash = (53 * hash) + getCompactorCount(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.PNextCompactionJob} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.PNextCompactionJob) + org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJobOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PNextCompactionJob_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PNextCompactionJob_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob.class, org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getJobFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + job_ = null; + if (jobBuilder_ != null) { + jobBuilder_.dispose(); + jobBuilder_ = null; + } + compactorCount_ = 0; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_PNextCompactionJob_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob build() { + org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob result = new org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.job_ = jobBuilder_ == null + ? job_ + : jobBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.compactorCount_ = compactorCount_; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob.getDefaultInstance()) return this; + if (other.hasJob()) { + mergeJob(other.getJob()); + } + if (other.getCompactorCount() != 0) { + setCompactorCount(other.getCompactorCount()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getJobFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 16: { + compactorCount_ = input.readInt32(); + bitField0_ |= 0x00000002; + break; + } // case 16 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob job_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder> jobBuilder_; + /** + * .tabletserver.PExternalCompactionJob job = 1; + * @return Whether the job field is set. + */ + public boolean hasJob() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .tabletserver.PExternalCompactionJob job = 1; + * @return The job. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob getJob() { + if (jobBuilder_ == null) { + return job_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance() : job_; + } else { + return jobBuilder_.getMessage(); + } + } + /** + * .tabletserver.PExternalCompactionJob job = 1; + */ + public Builder setJob(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob value) { + if (jobBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + job_ = value; + } else { + jobBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .tabletserver.PExternalCompactionJob job = 1; + */ + public Builder setJob( + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder builderForValue) { + if (jobBuilder_ == null) { + job_ = builderForValue.build(); + } else { + jobBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .tabletserver.PExternalCompactionJob job = 1; + */ + public Builder mergeJob(org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob value) { + if (jobBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) && + job_ != null && + job_ != org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance()) { + getJobBuilder().mergeFrom(value); + } else { + job_ = value; + } + } else { + jobBuilder_.mergeFrom(value); + } + if (job_ != null) { + bitField0_ |= 0x00000001; + onChanged(); + } + return this; + } + /** + * .tabletserver.PExternalCompactionJob job = 1; + */ + public Builder clearJob() { + bitField0_ = (bitField0_ & ~0x00000001); + job_ = null; + if (jobBuilder_ != null) { + jobBuilder_.dispose(); + jobBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .tabletserver.PExternalCompactionJob job = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder getJobBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getJobFieldBuilder().getBuilder(); + } + /** + * .tabletserver.PExternalCompactionJob job = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder getJobOrBuilder() { + if (jobBuilder_ != null) { + return jobBuilder_.getMessageOrBuilder(); + } else { + return job_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.getDefaultInstance() : job_; + } + } + /** + * .tabletserver.PExternalCompactionJob job = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder> + getJobFieldBuilder() { + if (jobBuilder_ == null) { + jobBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob.Builder, org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder>( + getJob(), + getParentForChildren(), + isClean()); + job_ = null; + } + return jobBuilder_; + } + + private int compactorCount_ ; + /** + * int32 compactorCount = 2; + * @return The compactorCount. + */ + @java.lang.Override + public int getCompactorCount() { + return compactorCount_; + } + /** + * int32 compactorCount = 2; + * @param value The compactorCount to set. + * @return This builder for chaining. + */ + public Builder setCompactorCount(int value) { + + compactorCount_ = value; + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * int32 compactorCount = 2; + * @return This builder for chaining. + */ + public Builder clearCompactorCount() { + bitField0_ = (bitField0_ & ~0x00000002); + compactorCount_ = 0; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.PNextCompactionJob) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.PNextCompactionJob) + private static final org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public PNextCompactionJob parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PNextCompactionJobOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PNextCompactionJobOrBuilder.java new file mode 100644 index 00000000000..81c7e70b0b3 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/PNextCompactionJobOrBuilder.java @@ -0,0 +1,49 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface PNextCompactionJobOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.PNextCompactionJob) + com.google.protobuf.MessageOrBuilder { + + /** + * .tabletserver.PExternalCompactionJob job = 1; + * @return Whether the job field is set. + */ + boolean hasJob(); + /** + * .tabletserver.PExternalCompactionJob job = 1; + * @return The job. + */ + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob getJob(); + /** + * .tabletserver.PExternalCompactionJob job = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJobOrBuilder getJobOrBuilder(); + + /** + * int32 compactorCount = 2; + * @return The compactorCount. + */ + int getCompactorCount(); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ProtoTInfo.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ProtoTInfo.java new file mode 100644 index 00000000000..643b38240d9 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ProtoTInfo.java @@ -0,0 +1,703 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: client.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code client.ProtoTInfo} + */ +public final class ProtoTInfo extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:client.ProtoTInfo) + ProtoTInfoOrBuilder { +private static final long serialVersionUID = 0L; + // Use ProtoTInfo.newBuilder() to construct. + private ProtoTInfo(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private ProtoTInfo() { + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new ProtoTInfo(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.ClientProto.internal_static_client_ProtoTInfo_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + @java.lang.Override + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMapFieldReflection( + int number) { + switch (number) { + case 1: + return internalGetHeaders(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.ClientProto.internal_static_client_ProtoTInfo_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.class, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder.class); + } + + public static final int HEADERS_FIELD_NUMBER = 1; + private static final class HeadersDefaultEntryHolder { + static final com.google.protobuf.MapEntry< + java.lang.String, java.lang.String> defaultEntry = + com.google.protobuf.MapEntry + .newDefaultInstance( + org.apache.accumulo.grpc.compaction.protobuf.ClientProto.internal_static_client_ProtoTInfo_HeadersEntry_descriptor, + com.google.protobuf.WireFormat.FieldType.STRING, + "", + com.google.protobuf.WireFormat.FieldType.STRING, + ""); + } + @SuppressWarnings("serial") + private com.google.protobuf.MapField< + java.lang.String, java.lang.String> headers_; + private com.google.protobuf.MapField + internalGetHeaders() { + if (headers_ == null) { + return com.google.protobuf.MapField.emptyMapField( + HeadersDefaultEntryHolder.defaultEntry); + } + return headers_; + } + public int getHeadersCount() { + return internalGetHeaders().getMap().size(); + } + /** + * map<string, string> headers = 1; + */ + @java.lang.Override + public boolean containsHeaders( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + return internalGetHeaders().getMap().containsKey(key); + } + /** + * Use {@link #getHeadersMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getHeaders() { + return getHeadersMap(); + } + /** + * map<string, string> headers = 1; + */ + @java.lang.Override + public java.util.Map getHeadersMap() { + return internalGetHeaders().getMap(); + } + /** + * map<string, string> headers = 1; + */ + @java.lang.Override + public /* nullable */ +java.lang.String getHeadersOrDefault( + java.lang.String key, + /* nullable */ +java.lang.String defaultValue) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetHeaders().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + * map<string, string> headers = 1; + */ + @java.lang.Override + public java.lang.String getHeadersOrThrow( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetHeaders().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + com.google.protobuf.GeneratedMessageV3 + .serializeStringMapTo( + output, + internalGetHeaders(), + HeadersDefaultEntryHolder.defaultEntry, + 1); + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (java.util.Map.Entry entry + : internalGetHeaders().getMap().entrySet()) { + com.google.protobuf.MapEntry + headers__ = HeadersDefaultEntryHolder.defaultEntry.newBuilderForType() + .setKey(entry.getKey()) + .setValue(entry.getValue()) + .build(); + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, headers__); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo other = (org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo) obj; + + if (!internalGetHeaders().equals( + other.internalGetHeaders())) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (!internalGetHeaders().getMap().isEmpty()) { + hash = (37 * hash) + HEADERS_FIELD_NUMBER; + hash = (53 * hash) + internalGetHeaders().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code client.ProtoTInfo} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:client.ProtoTInfo) + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.ClientProto.internal_static_client_ProtoTInfo_descriptor; + } + + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMapFieldReflection( + int number) { + switch (number) { + case 1: + return internalGetHeaders(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @SuppressWarnings({"rawtypes"}) + protected com.google.protobuf.MapFieldReflectionAccessor internalGetMutableMapFieldReflection( + int number) { + switch (number) { + case 1: + return internalGetMutableHeaders(); + default: + throw new RuntimeException( + "Invalid map field number: " + number); + } + } + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.ClientProto.internal_static_client_ProtoTInfo_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.class, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.newBuilder() + private Builder() { + + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + internalGetMutableHeaders().clear(); + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.ClientProto.internal_static_client_ProtoTInfo_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo build() { + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo result = new org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo result) { + int from_bitField0_ = bitField0_; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.headers_ = internalGetHeaders(); + result.headers_.makeImmutable(); + } + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance()) return this; + internalGetMutableHeaders().mergeFrom( + other.internalGetHeaders()); + bitField0_ |= 0x00000001; + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + com.google.protobuf.MapEntry + headers__ = input.readMessage( + HeadersDefaultEntryHolder.defaultEntry.getParserForType(), extensionRegistry); + internalGetMutableHeaders().getMutableMap().put( + headers__.getKey(), headers__.getValue()); + bitField0_ |= 0x00000001; + break; + } // case 10 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private com.google.protobuf.MapField< + java.lang.String, java.lang.String> headers_; + private com.google.protobuf.MapField + internalGetHeaders() { + if (headers_ == null) { + return com.google.protobuf.MapField.emptyMapField( + HeadersDefaultEntryHolder.defaultEntry); + } + return headers_; + } + private com.google.protobuf.MapField + internalGetMutableHeaders() { + if (headers_ == null) { + headers_ = com.google.protobuf.MapField.newMapField( + HeadersDefaultEntryHolder.defaultEntry); + } + if (!headers_.isMutable()) { + headers_ = headers_.copy(); + } + bitField0_ |= 0x00000001; + onChanged(); + return headers_; + } + public int getHeadersCount() { + return internalGetHeaders().getMap().size(); + } + /** + * map<string, string> headers = 1; + */ + @java.lang.Override + public boolean containsHeaders( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + return internalGetHeaders().getMap().containsKey(key); + } + /** + * Use {@link #getHeadersMap()} instead. + */ + @java.lang.Override + @java.lang.Deprecated + public java.util.Map getHeaders() { + return getHeadersMap(); + } + /** + * map<string, string> headers = 1; + */ + @java.lang.Override + public java.util.Map getHeadersMap() { + return internalGetHeaders().getMap(); + } + /** + * map<string, string> headers = 1; + */ + @java.lang.Override + public /* nullable */ +java.lang.String getHeadersOrDefault( + java.lang.String key, + /* nullable */ +java.lang.String defaultValue) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetHeaders().getMap(); + return map.containsKey(key) ? map.get(key) : defaultValue; + } + /** + * map<string, string> headers = 1; + */ + @java.lang.Override + public java.lang.String getHeadersOrThrow( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + java.util.Map map = + internalGetHeaders().getMap(); + if (!map.containsKey(key)) { + throw new java.lang.IllegalArgumentException(); + } + return map.get(key); + } + public Builder clearHeaders() { + bitField0_ = (bitField0_ & ~0x00000001); + internalGetMutableHeaders().getMutableMap() + .clear(); + return this; + } + /** + * map<string, string> headers = 1; + */ + public Builder removeHeaders( + java.lang.String key) { + if (key == null) { throw new NullPointerException("map key"); } + internalGetMutableHeaders().getMutableMap() + .remove(key); + return this; + } + /** + * Use alternate mutation accessors instead. + */ + @java.lang.Deprecated + public java.util.Map + getMutableHeaders() { + bitField0_ |= 0x00000001; + return internalGetMutableHeaders().getMutableMap(); + } + /** + * map<string, string> headers = 1; + */ + public Builder putHeaders( + java.lang.String key, + java.lang.String value) { + if (key == null) { throw new NullPointerException("map key"); } + if (value == null) { throw new NullPointerException("map value"); } + internalGetMutableHeaders().getMutableMap() + .put(key, value); + bitField0_ |= 0x00000001; + return this; + } + /** + * map<string, string> headers = 1; + */ + public Builder putAllHeaders( + java.util.Map values) { + internalGetMutableHeaders().getMutableMap() + .putAll(values); + bitField0_ |= 0x00000001; + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:client.ProtoTInfo) + } + + // @@protoc_insertion_point(class_scope:client.ProtoTInfo) + private static final org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public ProtoTInfo parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ProtoTInfoOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ProtoTInfoOrBuilder.java new file mode 100644 index 00000000000..ec28993cdf5 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/ProtoTInfoOrBuilder.java @@ -0,0 +1,62 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: client.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface ProtoTInfoOrBuilder extends + // @@protoc_insertion_point(interface_extends:client.ProtoTInfo) + com.google.protobuf.MessageOrBuilder { + + /** + * map<string, string> headers = 1; + */ + int getHeadersCount(); + /** + * map<string, string> headers = 1; + */ + boolean containsHeaders( + java.lang.String key); + /** + * Use {@link #getHeadersMap()} instead. + */ + @java.lang.Deprecated + java.util.Map + getHeaders(); + /** + * map<string, string> headers = 1; + */ + java.util.Map + getHeadersMap(); + /** + * map<string, string> headers = 1; + */ + /* nullable */ +java.lang.String getHeadersOrDefault( + java.lang.String key, + /* nullable */ +java.lang.String defaultValue); + /** + * map<string, string> headers = 1; + */ + java.lang.String getHeadersOrThrow( + java.lang.String key); +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/SecurityProto.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/SecurityProto.java new file mode 100644 index 00000000000..61332b08442 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/SecurityProto.java @@ -0,0 +1,71 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: security.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public final class SecurityProto { + private SecurityProto() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_security_PCredentials_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_security_PCredentials_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\016security.proto\022\010security\"\252\001\n\014PCredenti" + + "als\022\026\n\tprincipal\030\001 \001(\tH\000\210\001\001\022\033\n\016tokenClas" + + "sName\030\002 \001(\tH\001\210\001\001\022\022\n\005token\030\003 \001(\014H\002\210\001\001\022\027\n\n" + + "instanceId\030\004 \001(\tH\003\210\001\001B\014\n\n_principalB\021\n\017_" + + "tokenClassNameB\010\n\006_tokenB\r\n\013_instanceIdB" + + "?\n,org.apache.accumulo.grpc.compaction.p" + + "rotobufB\rSecurityProtoP\001b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }); + internal_static_security_PCredentials_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_security_PCredentials_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_security_PCredentials_descriptor, + new java.lang.String[] { "Principal", "TokenClassName", "Token", "InstanceId", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/TabletServerProto.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/TabletServerProto.java new file mode 100644 index 00000000000..e6d7a160e0c --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/TabletServerProto.java @@ -0,0 +1,165 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: tabletserver.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public final class TabletServerProto { + private TabletServerProto() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + static final com.google.protobuf.Descriptors.Descriptor + internal_static_tabletserver_PExternalCompactionJob_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_tabletserver_PExternalCompactionJob_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_tabletserver_PExternalCompactionJob_OverridesEntry_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_tabletserver_PExternalCompactionJob_OverridesEntry_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_tabletserver_PInputFile_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_tabletserver_PInputFile_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_tabletserver_PIteratorSetting_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_tabletserver_PIteratorSetting_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_tabletserver_PIteratorSetting_PropertiesEntry_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_tabletserver_PIteratorSetting_PropertiesEntry_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_tabletserver_PIteratorConfig_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_tabletserver_PIteratorConfig_fieldAccessorTable; + static final com.google.protobuf.Descriptors.Descriptor + internal_static_tabletserver_PCompactionStats_descriptor; + static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_tabletserver_PCompactionStats_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\022tabletserver.proto\022\014tabletserver\032\ndata" + + ".proto\032\rmanager.proto\"\305\004\n\026PExternalCompa" + + "ctionJob\022!\n\024externalCompactionId\030\001 \001(\tH\000" + + "\210\001\001\022%\n\006extent\030\002 \001(\0132\020.data.PKeyExtentH\001\210" + + "\001\001\022\'\n\005files\030\003 \003(\0132\030.tabletserver.PInputF" + + "ile\022<\n\020iteratorSettings\030\004 \001(\0132\035.tabletse" + + "rver.PIteratorConfigH\002\210\001\001\022\027\n\noutputFile\030" + + "\005 \001(\tH\003\210\001\001\022\035\n\020propagateDeletes\030\006 \001(\010H\004\210\001" + + "\001\0220\n\004kind\030\007 \001(\0162\035.tabletserver.PCompacti" + + "onKindH\005\210\001\001\022%\n\006fateId\030\010 \001(\0132\020.manager.PF" + + "ateIdH\006\210\001\001\022F\n\toverrides\030\t \003(\01323.tabletse" + + "rver.PExternalCompactionJob.OverridesEnt" + + "ry\0320\n\016OverridesEntry\022\013\n\003key\030\001 \001(\t\022\r\n\005val" + + "ue\030\002 \001(\t:\0028\001B\027\n\025_externalCompactionIdB\t\n" + + "\007_extentB\023\n\021_iteratorSettingsB\r\n\013_output" + + "FileB\023\n\021_propagateDeletesB\007\n\005_kindB\t\n\007_f" + + "ateId\"Y\n\nPInputFile\022\031\n\021metadataFileEntry" + + "\030\001 \001(\t\022\014\n\004size\030\002 \001(\003\022\017\n\007entries\030\003 \001(\003\022\021\n" + + "\ttimestamp\030\004 \001(\003\"\300\001\n\020PIteratorSetting\022\020\n" + + "\010priority\030\001 \001(\005\022\014\n\004name\030\002 \001(\t\022\025\n\riterato" + + "rClass\030\003 \001(\t\022B\n\nproperties\030\004 \003(\0132..table" + + "tserver.PIteratorSetting.PropertiesEntry" + + "\0321\n\017PropertiesEntry\022\013\n\003key\030\001 \001(\t\022\r\n\005valu" + + "e\030\002 \001(\t:\0028\001\"D\n\017PIteratorConfig\0221\n\titerat" + + "ors\030\001 \003(\0132\036.tabletserver.PIteratorSettin" + + "g\"Q\n\020PCompactionStats\022\023\n\013entriesRead\030\001 \001" + + "(\003\022\026\n\016entriesWritten\030\002 \001(\003\022\020\n\010fileSize\030\003" + + " \001(\003*4\n\017PCompactionKind\022\013\n\007UNKNOWN\020\000\022\n\n\006" + + "SYSTEM\020\001\022\010\n\004USER\020\002BC\n,org.apache.accumul" + + "o.grpc.compaction.protobufB\021TabletServer" + + "ProtoP\001b\006proto3" + }; + descriptor = com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + org.apache.accumulo.grpc.compaction.protobuf.DataProto.getDescriptor(), + org.apache.accumulo.grpc.compaction.protobuf.ManagerProto.getDescriptor(), + }); + internal_static_tabletserver_PExternalCompactionJob_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_tabletserver_PExternalCompactionJob_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_tabletserver_PExternalCompactionJob_descriptor, + new java.lang.String[] { "ExternalCompactionId", "Extent", "Files", "IteratorSettings", "OutputFile", "PropagateDeletes", "Kind", "FateId", "Overrides", }); + internal_static_tabletserver_PExternalCompactionJob_OverridesEntry_descriptor = + internal_static_tabletserver_PExternalCompactionJob_descriptor.getNestedTypes().get(0); + internal_static_tabletserver_PExternalCompactionJob_OverridesEntry_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_tabletserver_PExternalCompactionJob_OverridesEntry_descriptor, + new java.lang.String[] { "Key", "Value", }); + internal_static_tabletserver_PInputFile_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_tabletserver_PInputFile_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_tabletserver_PInputFile_descriptor, + new java.lang.String[] { "MetadataFileEntry", "Size", "Entries", "Timestamp", }); + internal_static_tabletserver_PIteratorSetting_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_tabletserver_PIteratorSetting_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_tabletserver_PIteratorSetting_descriptor, + new java.lang.String[] { "Priority", "Name", "IteratorClass", "Properties", }); + internal_static_tabletserver_PIteratorSetting_PropertiesEntry_descriptor = + internal_static_tabletserver_PIteratorSetting_descriptor.getNestedTypes().get(0); + internal_static_tabletserver_PIteratorSetting_PropertiesEntry_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_tabletserver_PIteratorSetting_PropertiesEntry_descriptor, + new java.lang.String[] { "Key", "Value", }); + internal_static_tabletserver_PIteratorConfig_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_tabletserver_PIteratorConfig_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_tabletserver_PIteratorConfig_descriptor, + new java.lang.String[] { "Iterators", }); + internal_static_tabletserver_PCompactionStats_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_tabletserver_PCompactionStats_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_tabletserver_PCompactionStats_descriptor, + new java.lang.String[] { "EntriesRead", "EntriesWritten", "FileSize", }); + org.apache.accumulo.grpc.compaction.protobuf.DataProto.getDescriptor(); + org.apache.accumulo.grpc.compaction.protobuf.ManagerProto.getDescriptor(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/UpdateCompactionStatusRequest.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/UpdateCompactionStatusRequest.java new file mode 100644 index 00000000000..9548791b780 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/UpdateCompactionStatusRequest.java @@ -0,0 +1,1190 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +/** + * Protobuf type {@code compaction_coordinator.UpdateCompactionStatusRequest} + */ +public final class UpdateCompactionStatusRequest extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:compaction_coordinator.UpdateCompactionStatusRequest) + UpdateCompactionStatusRequestOrBuilder { +private static final long serialVersionUID = 0L; + // Use UpdateCompactionStatusRequest.newBuilder() to construct. + private UpdateCompactionStatusRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private UpdateCompactionStatusRequest() { + externalCompactionId_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + UnusedPrivateParameter unused) { + return new UpdateCompactionStatusRequest(); + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_UpdateCompactionStatusRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_UpdateCompactionStatusRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest.class, org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest.Builder.class); + } + + private int bitField0_; + public static final int PTINFO_FIELD_NUMBER = 1; + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + @java.lang.Override + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + + public static final int CREDENTIALS_FIELD_NUMBER = 2; + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + @java.lang.Override + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + /** + * .security.PCredentials credentials = 2; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + + public static final int EXTERNALCOMPACTIONID_FIELD_NUMBER = 3; + @SuppressWarnings("serial") + private volatile java.lang.Object externalCompactionId_ = ""; + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + @java.lang.Override + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } + } + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + @java.lang.Override + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int STATUS_FIELD_NUMBER = 4; + private org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate status_; + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + * @return Whether the status field is set. + */ + @java.lang.Override + public boolean hasStatus() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + * @return The status. + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getStatus() { + return status_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.getDefaultInstance() : status_; + } + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + */ + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdateOrBuilder getStatusOrBuilder() { + return status_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.getDefaultInstance() : status_; + } + + public static final int TIMESTAMP_FIELD_NUMBER = 5; + private long timestamp_ = 0L; + /** + * int64 timestamp = 5; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + + private byte memoizedIsInitialized = -1; + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getCredentials()); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(externalCompactionId_)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 3, externalCompactionId_); + } + if (((bitField0_ & 0x00000004) != 0)) { + output.writeMessage(4, getStatus()); + } + if (timestamp_ != 0L) { + output.writeInt64(5, timestamp_); + } + getUnknownFields().writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getPtinfo()); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, getCredentials()); + } + if (!com.google.protobuf.GeneratedMessageV3.isStringEmpty(externalCompactionId_)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, externalCompactionId_); + } + if (((bitField0_ & 0x00000004) != 0)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(4, getStatus()); + } + if (timestamp_ != 0L) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(5, timestamp_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest)) { + return super.equals(obj); + } + org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest other = (org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest) obj; + + if (hasPtinfo() != other.hasPtinfo()) return false; + if (hasPtinfo()) { + if (!getPtinfo() + .equals(other.getPtinfo())) return false; + } + if (hasCredentials() != other.hasCredentials()) return false; + if (hasCredentials()) { + if (!getCredentials() + .equals(other.getCredentials())) return false; + } + if (!getExternalCompactionId() + .equals(other.getExternalCompactionId())) return false; + if (hasStatus() != other.hasStatus()) return false; + if (hasStatus()) { + if (!getStatus() + .equals(other.getStatus())) return false; + } + if (getTimestamp() + != other.getTimestamp()) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasPtinfo()) { + hash = (37 * hash) + PTINFO_FIELD_NUMBER; + hash = (53 * hash) + getPtinfo().hashCode(); + } + if (hasCredentials()) { + hash = (37 * hash) + CREDENTIALS_FIELD_NUMBER; + hash = (53 * hash) + getCredentials().hashCode(); + } + hash = (37 * hash) + EXTERNALCOMPACTIONID_FIELD_NUMBER; + hash = (53 * hash) + getExternalCompactionId().hashCode(); + if (hasStatus()) { + hash = (37 * hash) + STATUS_FIELD_NUMBER; + hash = (53 * hash) + getStatus().hashCode(); + } + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getTimestamp()); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseFrom( + java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseFrom( + java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code compaction_coordinator.UpdateCompactionStatusRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:compaction_coordinator.UpdateCompactionStatusRequest) + org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_UpdateCompactionStatusRequest_descriptor; + } + + @java.lang.Override + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_UpdateCompactionStatusRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest.class, org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest.Builder.class); + } + + // Construct using org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getPtinfoFieldBuilder(); + getCredentialsFieldBuilder(); + getStatusFieldBuilder(); + } + } + @java.lang.Override + public Builder clear() { + super.clear(); + bitField0_ = 0; + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + externalCompactionId_ = ""; + status_ = null; + if (statusBuilder_ != null) { + statusBuilder_.dispose(); + statusBuilder_ = null; + } + timestamp_ = 0L; + return this; + } + + @java.lang.Override + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceProto.internal_static_compaction_coordinator_UpdateCompactionStatusRequest_descriptor; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest getDefaultInstanceForType() { + return org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest.getDefaultInstance(); + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest build() { + org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest buildPartial() { + org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest result = new org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest(this); + if (bitField0_ != 0) { buildPartial0(result); } + onBuilt(); + return result; + } + + private void buildPartial0(org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest result) { + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + result.ptinfo_ = ptinfoBuilder_ == null + ? ptinfo_ + : ptinfoBuilder_.build(); + to_bitField0_ |= 0x00000001; + } + if (((from_bitField0_ & 0x00000002) != 0)) { + result.credentials_ = credentialsBuilder_ == null + ? credentials_ + : credentialsBuilder_.build(); + to_bitField0_ |= 0x00000002; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + result.externalCompactionId_ = externalCompactionId_; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.status_ = statusBuilder_ == null + ? status_ + : statusBuilder_.build(); + to_bitField0_ |= 0x00000004; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + result.timestamp_ = timestamp_; + } + result.bitField0_ |= to_bitField0_; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + @java.lang.Override + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.setField(field, value); + } + @java.lang.Override + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + @java.lang.Override + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + @java.lang.Override + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + @java.lang.Override + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return super.addRepeatedField(field, value); + } + @java.lang.Override + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest) { + return mergeFrom((org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest other) { + if (other == org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest.getDefaultInstance()) return this; + if (other.hasPtinfo()) { + mergePtinfo(other.getPtinfo()); + } + if (other.hasCredentials()) { + mergeCredentials(other.getCredentials()); + } + if (!other.getExternalCompactionId().isEmpty()) { + externalCompactionId_ = other.externalCompactionId_; + bitField0_ |= 0x00000004; + onChanged(); + } + if (other.hasStatus()) { + mergeStatus(other.getStatus()); + } + if (other.getTimestamp() != 0L) { + setTimestamp(other.getTimestamp()); + } + this.mergeUnknownFields(other.getUnknownFields()); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + return true; + } + + @java.lang.Override + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getPtinfoFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getCredentialsFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + externalCompactionId_ = input.readStringRequireUtf8(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + input.readMessage( + getStatusFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 40: { + timestamp_ = input.readInt64(); + bitField0_ |= 0x00000010; + break; + } // case 40 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.unwrapIOException(); + } finally { + onChanged(); + } // finally + return this; + } + private int bitField0_; + + private org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo ptinfo_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> ptinfoBuilder_; + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + public boolean hasPtinfo() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo() { + if (ptinfoBuilder_ == null) { + return ptinfo_ == null ? org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } else { + return ptinfoBuilder_.getMessage(); + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ptinfo_ = value; + } else { + ptinfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder setPtinfo( + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder builderForValue) { + if (ptinfoBuilder_ == null) { + ptinfo_ = builderForValue.build(); + } else { + ptinfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder mergePtinfo(org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo value) { + if (ptinfoBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) && + ptinfo_ != null && + ptinfo_ != org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance()) { + getPtinfoBuilder().mergeFrom(value); + } else { + ptinfo_ = value; + } + } else { + ptinfoBuilder_.mergeFrom(value); + } + if (ptinfo_ != null) { + bitField0_ |= 0x00000001; + onChanged(); + } + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public Builder clearPtinfo() { + bitField0_ = (bitField0_ & ~0x00000001); + ptinfo_ = null; + if (ptinfoBuilder_ != null) { + ptinfoBuilder_.dispose(); + ptinfoBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder getPtinfoBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getPtinfoFieldBuilder().getBuilder(); + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + public org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder() { + if (ptinfoBuilder_ != null) { + return ptinfoBuilder_.getMessageOrBuilder(); + } else { + return ptinfo_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.getDefaultInstance() : ptinfo_; + } + } + /** + * .client.ProtoTInfo ptinfo = 1; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder> + getPtinfoFieldBuilder() { + if (ptinfoBuilder_ == null) { + ptinfoBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo.Builder, org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder>( + getPtinfo(), + getParentForChildren(), + isClean()); + ptinfo_ = null; + } + return ptinfoBuilder_; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PCredentials credentials_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> credentialsBuilder_; + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + public boolean hasCredentials() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials() { + if (credentialsBuilder_ == null) { + return credentials_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } else { + return credentialsBuilder_.getMessage(); + } + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + credentials_ = value; + } else { + credentialsBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder setCredentials( + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder builderForValue) { + if (credentialsBuilder_ == null) { + credentials_ = builderForValue.build(); + } else { + credentialsBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder mergeCredentials(org.apache.accumulo.grpc.compaction.protobuf.PCredentials value) { + if (credentialsBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) && + credentials_ != null && + credentials_ != org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance()) { + getCredentialsBuilder().mergeFrom(value); + } else { + credentials_ = value; + } + } else { + credentialsBuilder_.mergeFrom(value); + } + if (credentials_ != null) { + bitField0_ |= 0x00000002; + onChanged(); + } + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public Builder clearCredentials() { + bitField0_ = (bitField0_ & ~0x00000002); + credentials_ = null; + if (credentialsBuilder_ != null) { + credentialsBuilder_.dispose(); + credentialsBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder getCredentialsBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getCredentialsFieldBuilder().getBuilder(); + } + /** + * .security.PCredentials credentials = 2; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder() { + if (credentialsBuilder_ != null) { + return credentialsBuilder_.getMessageOrBuilder(); + } else { + return credentials_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PCredentials.getDefaultInstance() : credentials_; + } + } + /** + * .security.PCredentials credentials = 2; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder> + getCredentialsFieldBuilder() { + if (credentialsBuilder_ == null) { + credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCredentials, org.apache.accumulo.grpc.compaction.protobuf.PCredentials.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder>( + getCredentials(), + getParentForChildren(), + isClean()); + credentials_ = null; + } + return credentialsBuilder_; + } + + private java.lang.Object externalCompactionId_ = ""; + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + public java.lang.String getExternalCompactionId() { + java.lang.Object ref = externalCompactionId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + externalCompactionId_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + public com.google.protobuf.ByteString + getExternalCompactionIdBytes() { + java.lang.Object ref = externalCompactionId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + externalCompactionId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * string externalCompactionId = 3; + * @param value The externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionId( + java.lang.String value) { + if (value == null) { throw new NullPointerException(); } + externalCompactionId_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + /** + * string externalCompactionId = 3; + * @return This builder for chaining. + */ + public Builder clearExternalCompactionId() { + externalCompactionId_ = getDefaultInstance().getExternalCompactionId(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * string externalCompactionId = 3; + * @param value The bytes for externalCompactionId to set. + * @return This builder for chaining. + */ + public Builder setExternalCompactionIdBytes( + com.google.protobuf.ByteString value) { + if (value == null) { throw new NullPointerException(); } + checkByteStringIsUtf8(value); + externalCompactionId_ = value; + bitField0_ |= 0x00000004; + onChanged(); + return this; + } + + private org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate status_; + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdateOrBuilder> statusBuilder_; + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + * @return Whether the status field is set. + */ + public boolean hasStatus() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + * @return The status. + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getStatus() { + if (statusBuilder_ == null) { + return status_ == null ? org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.getDefaultInstance() : status_; + } else { + return statusBuilder_.getMessage(); + } + } + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + */ + public Builder setStatus(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate value) { + if (statusBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + status_ = value; + } else { + statusBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + */ + public Builder setStatus( + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder builderForValue) { + if (statusBuilder_ == null) { + status_ = builderForValue.build(); + } else { + statusBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + onChanged(); + return this; + } + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + */ + public Builder mergeStatus(org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate value) { + if (statusBuilder_ == null) { + if (((bitField0_ & 0x00000008) != 0) && + status_ != null && + status_ != org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.getDefaultInstance()) { + getStatusBuilder().mergeFrom(value); + } else { + status_ = value; + } + } else { + statusBuilder_.mergeFrom(value); + } + if (status_ != null) { + bitField0_ |= 0x00000008; + onChanged(); + } + return this; + } + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + */ + public Builder clearStatus() { + bitField0_ = (bitField0_ & ~0x00000008); + status_ = null; + if (statusBuilder_ != null) { + statusBuilder_.dispose(); + statusBuilder_ = null; + } + onChanged(); + return this; + } + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder getStatusBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getStatusFieldBuilder().getBuilder(); + } + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + */ + public org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdateOrBuilder getStatusOrBuilder() { + if (statusBuilder_ != null) { + return statusBuilder_.getMessageOrBuilder(); + } else { + return status_ == null ? + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.getDefaultInstance() : status_; + } + } + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + */ + private com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdateOrBuilder> + getStatusFieldBuilder() { + if (statusBuilder_ == null) { + statusBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate.Builder, org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdateOrBuilder>( + getStatus(), + getParentForChildren(), + isClean()); + status_ = null; + } + return statusBuilder_; + } + + private long timestamp_ ; + /** + * int64 timestamp = 5; + * @return The timestamp. + */ + @java.lang.Override + public long getTimestamp() { + return timestamp_; + } + /** + * int64 timestamp = 5; + * @param value The timestamp to set. + * @return This builder for chaining. + */ + public Builder setTimestamp(long value) { + + timestamp_ = value; + bitField0_ |= 0x00000010; + onChanged(); + return this; + } + /** + * int64 timestamp = 5; + * @return This builder for chaining. + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000010); + timestamp_ = 0L; + onChanged(); + return this; + } + @java.lang.Override + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:compaction_coordinator.UpdateCompactionStatusRequest) + } + + // @@protoc_insertion_point(class_scope:compaction_coordinator.UpdateCompactionStatusRequest) + private static final org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest(); + } + + public static org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + private static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() { + @java.lang.Override + public UpdateCompactionStatusRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + +} + diff --git a/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/UpdateCompactionStatusRequestOrBuilder.java b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/UpdateCompactionStatusRequestOrBuilder.java new file mode 100644 index 00000000000..02f50dbcd13 --- /dev/null +++ b/rpc/grpc/src/main/protobuf-gen-java/org/apache/accumulo/grpc/compaction/protobuf/UpdateCompactionStatusRequestOrBuilder.java @@ -0,0 +1,91 @@ +/* + * 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. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: compaction-coordinator.proto + +// Protobuf Java Version: 3.25.3 +package org.apache.accumulo.grpc.compaction.protobuf; + +public interface UpdateCompactionStatusRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:compaction_coordinator.UpdateCompactionStatusRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * .client.ProtoTInfo ptinfo = 1; + * @return Whether the ptinfo field is set. + */ + boolean hasPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + * @return The ptinfo. + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo getPtinfo(); + /** + * .client.ProtoTInfo ptinfo = 1; + */ + org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfoOrBuilder getPtinfoOrBuilder(); + + /** + * .security.PCredentials credentials = 2; + * @return Whether the credentials field is set. + */ + boolean hasCredentials(); + /** + * .security.PCredentials credentials = 2; + * @return The credentials. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentials getCredentials(); + /** + * .security.PCredentials credentials = 2; + */ + org.apache.accumulo.grpc.compaction.protobuf.PCredentialsOrBuilder getCredentialsOrBuilder(); + + /** + * string externalCompactionId = 3; + * @return The externalCompactionId. + */ + java.lang.String getExternalCompactionId(); + /** + * string externalCompactionId = 3; + * @return The bytes for externalCompactionId. + */ + com.google.protobuf.ByteString + getExternalCompactionIdBytes(); + + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + * @return Whether the status field is set. + */ + boolean hasStatus(); + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + * @return The status. + */ + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate getStatus(); + /** + * .compaction_coordinator.PCompactionStatusUpdate status = 4; + */ + org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdateOrBuilder getStatusOrBuilder(); + + /** + * int64 timestamp = 5; + * @return The timestamp. + */ + long getTimestamp(); +} diff --git a/rpc/grpc/src/main/spotbugs/exclude-filter.xml b/rpc/grpc/src/main/spotbugs/exclude-filter.xml new file mode 100644 index 00000000000..93fe2179bd8 --- /dev/null +++ b/rpc/grpc/src/main/spotbugs/exclude-filter.xml @@ -0,0 +1,37 @@ + + + + + + + + + + + + + + diff --git a/server/base/pom.xml b/server/base/pom.xml index c20b3e6edce..9da948f333c 100644 --- a/server/base/pom.xml +++ b/server/base/pom.xml @@ -52,10 +52,22 @@ com.google.guava guava
+ + com.google.protobuf + protobuf-java + commons-codec commons-codec + + io.grpc + grpc-api + + + io.grpc + grpc-netty-shaded + io.micrometer micrometer-core @@ -72,6 +84,10 @@ org.apache.accumulo accumulo-core + + org.apache.accumulo + accumulo-grpc + org.apache.accumulo accumulo-start diff --git a/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableRpcCall.java similarity index 96% rename from server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java rename to server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableRpcCall.java index f4389f9fada..a038e18ea39 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java +++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableRpcCall.java @@ -26,7 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class RetryableThriftCall { +public class RetryableRpcCall { public static class RetriesExceededException extends Exception { @@ -53,7 +53,7 @@ public RetriesExceededException(Throwable cause) { } - private static final Logger LOG = LoggerFactory.getLogger(RetryableThriftCall.class); + private static final Logger LOG = LoggerFactory.getLogger(RetryableRpcCall.class); public static final long MAX_WAIT_TIME = 60000; private final RetryableThriftFunction function; @@ -67,7 +67,7 @@ public RetriesExceededException(Throwable cause) { * @param maxNumRetries number of times to retry, 0 to retry forever * @param function function to execute */ - public RetryableThriftCall(long start, long maxWaitTime, int maxNumRetries, + public RetryableRpcCall(long start, long maxWaitTime, int maxNumRetries, RetryableThriftFunction function) { this.function = function; NeedsRetryDelay builder = null; diff --git a/server/base/src/main/java/org/apache/accumulo/server/grpc/CompactionCoordinatorServiceServer.java b/server/base/src/main/java/org/apache/accumulo/server/grpc/CompactionCoordinatorServiceServer.java new file mode 100644 index 00000000000..e8ff1b95a93 --- /dev/null +++ b/server/base/src/main/java/org/apache/accumulo/server/grpc/CompactionCoordinatorServiceServer.java @@ -0,0 +1,86 @@ +/* + * 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.server.grpc; + +import java.io.IOException; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import org.apache.accumulo.core.rpc.grpc.GrpcUtil; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceGrpc; +import org.apache.accumulo.server.ServerContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.grpc.Grpc; +import io.grpc.InsecureServerCredentials; +import io.grpc.Server; +import io.grpc.ServerBuilder; +import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder; + +/** + * Simple wrapper to start/stop the grpc server + */ +public class CompactionCoordinatorServiceServer { + + private static final Logger logger = + LoggerFactory.getLogger(CompactionCoordinatorServiceServer.class); + + private final int port; + private final Server server; + + public CompactionCoordinatorServiceServer( + CompactionCoordinatorServiceGrpc.CompactionCoordinatorServiceImplBase service, int port) + throws IOException { + this(Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()), service, port); + } + + public CompactionCoordinatorServiceServer(ServerBuilder serverBuilder, + CompactionCoordinatorServiceGrpc.CompactionCoordinatorServiceImplBase service, int port) { + this.port = port; + this.server = serverBuilder.addService(service).build(); + } + + public CompactionCoordinatorServiceServer(ServerContext context, + CompactionCoordinatorServiceGrpc.CompactionCoordinatorServiceImplBase service, int port) { + this.port = port; + var serverBuilder = NettyServerBuilder.forPort(port); + + Optional.ofNullable(context.getServerSslParams()).ifPresent( + sslParams -> serverBuilder.sslContext(GrpcUtil.buildSslContext(sslParams, true))); + + this.server = serverBuilder.addService(service).build(); + } + + public void start() throws IOException { + server.start(); + logger.info("Starting CompactionCoordinatorService, listening on {}", port); + } + + public void stop() { + logger.info("Stopping CompactionCoordinatorService"); + try { + server.shutdown().awaitTermination(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + // TODO Do we care or want to handle this? It's on shutdown so probably can just log + logger.debug(e.getMessage(), e); + } + } + +} diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java index 6f0a6086eba..733acf9c59e 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java +++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java @@ -114,14 +114,10 @@ public static TMultiplexedProcessor getGcTProcessor(GCMonitorService.Iface servi } public static TMultiplexedProcessor getManagerTProcessor(FateService.Iface fateServiceHandler, - CompactionCoordinatorService.Iface coordinatorServiceHandler, ManagerClientService.Iface managerServiceHandler, ServerContext context) { TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor(); muxProcessor.registerProcessor(FATE.getServiceName(), FATE.getTProcessor( FateService.Processor.class, FateService.Iface.class, fateServiceHandler, context)); - muxProcessor.registerProcessor(COORDINATOR.getServiceName(), - COORDINATOR.getTProcessor(CompactionCoordinatorService.Processor.class, - CompactionCoordinatorService.Iface.class, coordinatorServiceHandler, context)); muxProcessor.registerProcessor(MANAGER.getServiceName(), MANAGER.getTProcessor(ManagerClientService.Processor.class, ManagerClientService.Iface.class, managerServiceHandler, context)); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java index 416bbf39a2f..24fb24282c2 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java @@ -18,18 +18,20 @@ */ package org.apache.accumulo.server.util; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; -import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; -import org.apache.accumulo.core.rpc.ThriftUtil; -import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; +import org.apache.accumulo.core.rpc.grpc.GrpcUtil; import org.apache.accumulo.core.singletons.SingletonManager; import org.apache.accumulo.core.singletons.SingletonManager.Mode; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; import org.apache.accumulo.core.util.compaction.RunningCompaction; import org.apache.accumulo.core.util.compaction.RunningCompactionInfo; +import org.apache.accumulo.grpc.compaction.protobuf.CancelRequest; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceGrpc; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceGrpc.CompactionCoordinatorServiceBlockingStub; +import org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.cli.ServerUtilOpts; import org.apache.accumulo.start.spi.KeywordExecutable; @@ -131,16 +133,19 @@ public void execute(final String[] args) { } private void cancelCompaction(ServerContext context, String ecid) { - CompactionCoordinatorService.Client coordinatorClient = null; + CompactionCoordinatorServiceBlockingStub coordinatorClient = null; ecid = ExternalCompactionId.from(ecid).canonical(); try { coordinatorClient = getCoordinatorClient(context); - coordinatorClient.cancel(TraceUtil.traceInfo(), context.rpcCreds(), ecid); + var ignored = + coordinatorClient.cancel(CancelRequest.newBuilder().setPtinfo(TraceUtil.protoTraceInfo()) + .setCredentials(context.gRpcCreds()).setExternalCompactionId(ecid).build()); System.out.println("Cancel sent to coordinator for " + ecid); } catch (Exception e) { throw new IllegalStateException("Exception calling cancel compaction for " + ecid, e); } finally { - ThriftUtil.returnClient(coordinatorClient, context); + // TODO: cleanup if using grpc pooling in future + // ThriftUtil.returnClient(coordinatorClient, context); } } @@ -154,16 +159,17 @@ private void listCompactorsByQueue(ServerContext context) { } private void runningCompactions(ServerContext context, boolean details) { - CompactionCoordinatorService.Client coordinatorClient = null; - TExternalCompactionList running; + CompactionCoordinatorServiceBlockingStub coordinatorClient = null; + PExternalCompactionList running; try { coordinatorClient = getCoordinatorClient(context); - running = coordinatorClient.getRunningCompactions(TraceUtil.traceInfo(), context.rpcCreds()); + running = coordinatorClient.getRunningCompactions(GetRunningCompactionsRequest.newBuilder() + .setPtinfo(TraceUtil.protoTraceInfo()).setCredentials(context.gRpcCreds()).build()); if (running == null) { System.out.println("No running compactions found."); return; } - var ecidMap = running.getCompactions(); + var ecidMap = running.getCompactionsMap(); if (ecidMap == null) { System.out.println("No running compactions found."); return; @@ -172,9 +178,9 @@ private void runningCompactions(ServerContext context, boolean details) { if (ec != null) { var runningCompaction = new RunningCompaction(ec); var addr = runningCompaction.getCompactorAddress(); - var kind = runningCompaction.getJob().kind; + var kind = runningCompaction.getJob().getKind(); var group = runningCompaction.getGroupName(); - var ke = KeyExtent.fromThrift(runningCompaction.getJob().extent); + var ke = KeyExtent.fromProtobuf(runningCompaction.getJob().getExtent()); System.out.format("%s %s %s %s TableId: %s\n", ecid, addr, kind, group, ke.tableId()); if (details) { var runningCompactionInfo = new RunningCompactionInfo(ec); @@ -191,19 +197,24 @@ private void runningCompactions(ServerContext context, boolean details) { } catch (Exception e) { throw new IllegalStateException("Unable to get running compactions.", e); } finally { - ThriftUtil.returnClient(coordinatorClient, context); + // TODO: clean up if we use pooling with grpc + // ThriftUtil.returnClient(coordinatorClient, context); } } - private CompactionCoordinatorService.Client getCoordinatorClient(ServerContext context) { + private CompactionCoordinatorServiceBlockingStub getCoordinatorClient(ServerContext context) { var coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(context); if (coordinatorHost.isEmpty()) { throw new IllegalStateException("Unable to find coordinator. Check that it is running."); } HostAndPort address = coordinatorHost.orElseThrow(); - CompactionCoordinatorService.Client coordinatorClient; + CompactionCoordinatorServiceBlockingStub coordinatorClient; try { - coordinatorClient = ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, address, context); + // TODO: coordinatorHost contains the Thrift port so right now only host is used. + // we eventually need the gRPC port and will need to store than in Zk. + // GrpcUtil for now just uses the property in the context for the port + coordinatorClient = CompactionCoordinatorServiceGrpc + .newBlockingStub(GrpcUtil.getChannel(coordinatorHost.orElseThrow(), context)); } catch (Exception e) { throw new IllegalStateException("Unable to get Compaction coordinator at " + address, e); } diff --git a/server/compactor/pom.xml b/server/compactor/pom.xml index 7779271cefb..b22a1a2f83c 100644 --- a/server/compactor/pom.xml +++ b/server/compactor/pom.xml @@ -39,6 +39,14 @@ com.google.guava guava + + com.google.protobuf + protobuf-java + + + io.grpc + grpc-api + io.micrometer micrometer-core @@ -47,6 +55,10 @@ org.apache.accumulo accumulo-core + + org.apache.accumulo + accumulo-grpc + org.apache.accumulo accumulo-server-base diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/CompactionJobHolder.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/CompactionJobHolder.java index 38d03faaf2d..9d5605a7486 100644 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/CompactionJobHolder.java +++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/CompactionJobHolder.java @@ -23,17 +23,17 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; -import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob; import org.apache.accumulo.server.compaction.FileCompactor; public class CompactionJobHolder { - private TExternalCompactionJob job; + private PExternalCompactionJob job; private Thread compactionThread; private AtomicReference compactor; private volatile boolean cancelled = false; - private volatile TCompactionStats stats = null; + private volatile PCompactionStats stats = null; CompactionJobHolder() {} @@ -45,20 +45,20 @@ public synchronized void reset() { stats = null; } - public synchronized TExternalCompactionJob getJob() { + public synchronized PExternalCompactionJob getJob() { return job; } public TableId getTableId() { - var tKeyExtent = getJob().getExtent(); - return KeyExtent.fromThrift(tKeyExtent).tableId(); + var pKeyExtent = getJob().getExtent(); + return KeyExtent.fromProtobuf(pKeyExtent).tableId(); } - public TCompactionStats getStats() { + public PCompactionStats getStats() { return stats; } - public void setStats(TCompactionStats stats) { + public void setStats(PCompactionStats stats) { this.stats = stats; } @@ -82,7 +82,7 @@ public synchronized boolean isSet() { return (null != this.job); } - public synchronized void set(TExternalCompactionJob job, Thread compactionThread, + public synchronized void set(PExternalCompactionJob job, Thread compactionThread, AtomicReference compactor) { Objects.requireNonNull(job, "CompactionJob is null"); Objects.requireNonNull(compactionThread, "Compaction thread is null"); diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java index ea8a90bfe4e..eab31499ff4 100644 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java +++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java @@ -19,6 +19,7 @@ package org.apache.accumulo.compactor; import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; +import static org.apache.accumulo.core.rpc.grpc.ThriftProtobufUtil.convert; import static org.apache.accumulo.core.util.LazySingletons.RANDOM; import java.io.IOException; @@ -49,12 +50,7 @@ import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService.Client; import org.apache.accumulo.core.compaction.thrift.CompactorService; -import org.apache.accumulo.core.compaction.thrift.TCompactionState; -import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; -import org.apache.accumulo.core.compaction.thrift.TNextCompactionJob; import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.ConfigurationCopy; @@ -85,13 +81,10 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; import org.apache.accumulo.core.metrics.MetricsInfo; import org.apache.accumulo.core.metrics.MetricsProducer; -import org.apache.accumulo.core.rpc.ThriftUtil; -import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; +import org.apache.accumulo.core.rpc.grpc.GrpcUtil; import org.apache.accumulo.core.securityImpl.thrift.TCredentials; import org.apache.accumulo.core.spi.crypto.CryptoService; import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.Halt; @@ -100,6 +93,18 @@ import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.util.threads.Threads; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceGrpc; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceGrpc.CompactionCoordinatorServiceBlockingStub; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionState; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest; import org.apache.accumulo.server.AbstractServer; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.client.ClientServiceHandler; @@ -108,8 +113,8 @@ import org.apache.accumulo.server.compaction.CompactionWatcher; import org.apache.accumulo.server.compaction.FileCompactor; import org.apache.accumulo.server.compaction.PausedCompactionMetrics; -import org.apache.accumulo.server.compaction.RetryableThriftCall; -import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException; +import org.apache.accumulo.server.compaction.RetryableRpcCall; +import org.apache.accumulo.server.compaction.RetryableRpcCall.RetriesExceededException; import org.apache.accumulo.server.conf.TableConfiguration; import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.rpc.ServerAddress; @@ -127,6 +132,7 @@ import com.google.common.base.Preconditions; import com.google.common.net.HostAndPort; +import io.grpc.StatusRuntimeException; import io.micrometer.core.instrument.FunctionCounter; import io.micrometer.core.instrument.LongTaskTimer; import io.micrometer.core.instrument.MeterRegistry; @@ -203,10 +209,11 @@ protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor, } protected void checkIfCanceled() { - TExternalCompactionJob job = JOB_HOLDER.getJob(); + PExternalCompactionJob job = JOB_HOLDER.getJob(); if (job != null) { try { - var extent = KeyExtent.fromThrift(job.getExtent()); + var extent = KeyExtent.fromProtobuf(job.getExtent()); + var ecid = ExternalCompactionId.of(job.getExternalCompactionId()); TabletMetadata tabletMeta = @@ -227,20 +234,20 @@ protected void checkIfCanceled() { return; } - if (job.getKind() == TCompactionKind.USER) { + if (job.getKind() == PCompactionKind.USER) { - var cconf = - CompactionConfigStorage.getConfig(getContext(), FateId.fromThrift(job.getFateId())); + var cconf = CompactionConfigStorage.getConfig(getContext(), + FateId.fromThrift(convert(job.getFateId()))); if (cconf == null) { LOG.info("Cancelling compaction {} for user compaction that no longer exists {} {}", - ecid, FateId.fromThrift(job.getFateId()), extent); + ecid, FateId.fromThrift(convert(job.getFateId())), extent); JOB_HOLDER.cancel(job.getExternalCompactionId()); } } } catch (RuntimeException | KeeperException e) { LOG.warn("Failed to check if compaction {} for {} was canceled.", - job.getExternalCompactionId(), KeyExtent.fromThrift(job.getExtent()), e); + job.getExternalCompactionId(), KeyExtent.fromProtobuf(job.getExtent()), e); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); @@ -384,20 +391,34 @@ public void cancel(TInfo tinfo, TCredentials credentials, String externalCompact * @param update status update * @throws RetriesExceededException thrown when retries have been exceeded */ - protected void updateCompactionState(TExternalCompactionJob job, TCompactionStatusUpdate update) + protected void updateCompactionState(PExternalCompactionJob job, PCompactionStatusUpdate update) throws RetriesExceededException { - RetryableThriftCall thriftCall = - new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 25, () -> { - Client coordinatorClient = getCoordinatorClient(); + RetryableRpcCall thriftCall = + new RetryableRpcCall<>(1000, RetryableRpcCall.MAX_WAIT_TIME, 25, () -> { + var coordinatorClient = getCoordinatorClient(); try { - coordinatorClient.updateCompactionStatus(TraceUtil.traceInfo(), getContext().rpcCreds(), - job.getExternalCompactionId(), update, System.currentTimeMillis()); + var u = UpdateCompactionStatusRequest.newBuilder().setPtinfo(TraceUtil.protoTraceInfo()) + .setCredentials(getContext().gRpcCreds()) + .setExternalCompactionId(job.getExternalCompactionId()).setStatus(update) + .setTimestamp(System.currentTimeMillis()).build(); + LOG.debug("Compactor status update, id: {}, timestamp: {}, update: {}", + u.getExternalCompactionId(), u.getTimestamp(), u.getStatus()); + var ignored = coordinatorClient.updateCompactionStatus(u); return ""; + } catch (StatusRuntimeException e) { + // TODO: Do we need to handle this any better? + LOG.debug(e.getMessage()); + throw e; } finally { - ThriftUtil.returnClient(coordinatorClient, getContext()); + // TODO: Channel is currently cached and shared, look into using pooling } }); - thriftCall.run(); + try { + thriftCall.run(); + } catch (Exception e) { + LOG.error("thriftRun error: {}", e.getMessage()); + } + } /** @@ -406,17 +427,23 @@ protected void updateCompactionState(TExternalCompactionJob job, TCompactionStat * @param job current compaction job * @throws RetriesExceededException thrown when retries have been exceeded */ - protected void updateCompactionFailed(TExternalCompactionJob job) + protected void updateCompactionFailed(PExternalCompactionJob job) throws RetriesExceededException { - RetryableThriftCall thriftCall = - new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 25, () -> { - Client coordinatorClient = getCoordinatorClient(); + RetryableRpcCall thriftCall = + new RetryableRpcCall<>(1000, RetryableRpcCall.MAX_WAIT_TIME, 25, () -> { + var coordinatorClient = getCoordinatorClient(); try { - coordinatorClient.compactionFailed(TraceUtil.traceInfo(), getContext().rpcCreds(), - job.getExternalCompactionId(), job.extent); + var ignored = coordinatorClient.compactionFailed(CompactionFailedRequest.newBuilder() + .setPtinfo(TraceUtil.protoTraceInfo()).setCredentials(getContext().gRpcCreds()) + .setExternalCompactionId(job.getExternalCompactionId()).setExtent(job.getExtent()) + .build()); return ""; + } catch (StatusRuntimeException e) { + // TODO: Do we need to handle this any better? + LOG.debug(e.getMessage()); + throw e; } finally { - ThriftUtil.returnClient(coordinatorClient, getContext()); + // TODO: Channel is currently cached and shared, look into using pooling } }); thriftCall.run(); @@ -429,20 +456,28 @@ protected void updateCompactionFailed(TExternalCompactionJob job) * @param stats compaction stats * @throws RetriesExceededException thrown when retries have been exceeded */ - protected void updateCompactionCompleted(TExternalCompactionJob job, TCompactionStats stats) + protected void updateCompactionCompleted(PExternalCompactionJob job, PCompactionStats stats) throws RetriesExceededException { - RetryableThriftCall thriftCall = - new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 25, () -> { - Client coordinatorClient = getCoordinatorClient(); + RetryableRpcCall rpcCall = + new RetryableRpcCall<>(1000, RetryableRpcCall.MAX_WAIT_TIME, 25, () -> { + var coordinatorClient = getCoordinatorClient(); try { - coordinatorClient.compactionCompleted(TraceUtil.traceInfo(), getContext().rpcCreds(), - job.getExternalCompactionId(), job.extent, stats); + LOG.info("Job: {}", job.getExtent()); + var ignored = + coordinatorClient.compactionCompleted(CompactionCompletedRequest.newBuilder() + .setPtinfo(TraceUtil.protoTraceInfo()).setCredentials(getContext().gRpcCreds()) + .setExternalCompactionId(job.getExternalCompactionId()) + .setExtent(job.getExtent()).setStats(stats).build()); return ""; + } catch (StatusRuntimeException e) { + // TODO: Do we need to handle this any better? + LOG.debug(e.getMessage()); + throw e; } finally { - ThriftUtil.returnClient(coordinatorClient, getContext()); + // TODO: Channel is currently cached and shared, look into using pooling } }); - thriftCall.run(); + rpcCall.run(); } /** @@ -452,47 +487,53 @@ protected void updateCompactionCompleted(TExternalCompactionJob job, TCompaction * @return CompactionJob * @throws RetriesExceededException thrown when retries have been exceeded */ - protected TNextCompactionJob getNextJob(Supplier uuid) throws RetriesExceededException { + protected PNextCompactionJob getNextJob(Supplier uuid) throws RetriesExceededException { final long startingWaitTime = getConfiguration().getTimeInMillis(Property.COMPACTOR_MIN_JOB_WAIT_TIME); final long maxWaitTime = getConfiguration().getTimeInMillis(Property.COMPACTOR_MAX_JOB_WAIT_TIME); - RetryableThriftCall nextJobThriftCall = - new RetryableThriftCall<>(startingWaitTime, maxWaitTime, 0, () -> { - Client coordinatorClient = getCoordinatorClient(); + RetryableRpcCall nextJobRpcCall = + new RetryableRpcCall<>(startingWaitTime, maxWaitTime, 0, () -> { + var grpcClient = getCoordinatorClient(); try { ExternalCompactionId eci = ExternalCompactionId.generate(uuid.get()); LOG.trace("Attempting to get next job, eci = {}", eci); currentCompactionId.set(eci); - return coordinatorClient.getCompactionJob(TraceUtil.traceInfo(), - getContext().rpcCreds(), this.getResourceGroup(), - ExternalCompactionUtil.getHostPortString(compactorAddress.getAddress()), - eci.toString()); + + // gRPC is now used to make the compaction job request. + var request = CompactionJobRequest.newBuilder().setPtinfo(TraceUtil.protoTraceInfo()) + .setCredentials(getContext().gRpcCreds()).setGroupName(this.getResourceGroup()) + .setCompactor( + ExternalCompactionUtil.getHostPortString(compactorAddress.getAddress())) + .setExternalCompactionId(eci.toString()).build(); + return grpcClient.getCompactionJob(request); } catch (Exception e) { currentCompactionId.set(null); throw e; } finally { - ThriftUtil.returnClient(coordinatorClient, getContext()); + // TODO: Channel is currently cached and shared, look into using pooling } }); - return nextJobThriftCall.run(); + return nextJobRpcCall.run(); } /** * Get the client to the CompactionCoordinator * * @return compaction coordinator client - * @throws TTransportException when unable to get client */ - protected CompactionCoordinatorService.Client getCoordinatorClient() throws TTransportException { + protected CompactionCoordinatorServiceBlockingStub getCoordinatorClient() + throws TTransportException { var coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(getContext()); if (coordinatorHost.isEmpty()) { throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); } - LOG.trace("CompactionCoordinator address is: {}", coordinatorHost.orElseThrow()); - return ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.orElseThrow(), - getContext()); + // TODO: coordinatorHost contains the Thrift port so right now only host is used. + // we eventually need the gRPC port and will need to store than in Zk. + // GrpcUtil for now just uses the property in the context for the port + return CompactionCoordinatorServiceGrpc + .newBlockingStub(GrpcUtil.getChannel(coordinatorHost.orElseThrow(), getContext())); } /** @@ -506,7 +547,7 @@ protected CompactionCoordinatorService.Client getCoordinatorClient() throws TTra * @param err reference to error * @return Runnable compaction job */ - protected FileCompactorRunnable createCompactionJob(final TExternalCompactionJob job, + protected Compactor.FileCompactorRunnable createCompactionJob(final PExternalCompactionJob job, final LongAdder totalInputEntries, final LongAdder totalInputBytes, final CountDownLatch started, final CountDownLatch stopped, final AtomicReference err) { @@ -520,17 +561,19 @@ protected FileCompactorRunnable createCompactionJob(final TExternalCompactionJob public void initialize() throws RetriesExceededException { LOG.info("Starting up compaction runnable for job: {}", job); this.compactionStartTime = Timer.startNew(); - TCompactionStatusUpdate update = new TCompactionStatusUpdate(TCompactionState.STARTED, - "Compaction started", -1, -1, -1, getCompactionAge().toNanos()); + PCompactionStatusUpdate update = + PCompactionStatusUpdate.newBuilder().setState(PCompactionState.STARTED) + .setMessage("Compaction started").setEntriesToBeCompacted(-1).setEntriesRead(-1) + .setEntriesWritten(-1).setCompactionAgeNanos(getCompactionAge().toNanos()).build(); updateCompactionState(job, update); - final var extent = KeyExtent.fromThrift(job.getExtent()); + final var extent = KeyExtent.fromProtobuf(job.getExtent()); final AccumuloConfiguration aConfig; final TableConfiguration tConfig = getContext().getTableConfiguration(extent.tableId()); - if (!job.getOverrides().isEmpty()) { + if (!job.getOverridesMap().isEmpty()) { aConfig = new ConfigurationCopy(tConfig); - job.getOverrides().forEach(((ConfigurationCopy) aConfig)::set); - LOG.debug("Overriding table properties with {}", job.getOverrides()); + job.getOverridesMap().forEach(((ConfigurationCopy) aConfig)::set); + LOG.debug("Overriding table properties with {}", job.getOverridesMap()); } else { aConfig = tConfig; } @@ -539,7 +582,7 @@ public void initialize() throws RetriesExceededException { new ReferencedTabletFile(new Path(job.getOutputFile())); final Map files = new TreeMap<>(); - job.getFiles().forEach(f -> { + job.getFilesList().forEach(f -> { long estEntries = f.getEntries(); StoredTabletFile stf = new StoredTabletFile(f.getMetadataFileEntry()); // This happens with bulk import files @@ -553,12 +596,12 @@ public void initialize() throws RetriesExceededException { }); final List iters = new ArrayList<>(); - job.getIteratorSettings().getIterators() + job.getIteratorSettings().getIteratorsList() .forEach(tis -> iters.add(SystemIteratorUtil.toIteratorSetting(tis))); final ExtCEnv cenv = new ExtCEnv(JOB_HOLDER, getResourceGroup()); compactor.set( - new FileCompactor(getContext(), extent, files, outputFile, job.isPropagateDeletes(), + new FileCompactor(getContext(), extent, files, outputFile, job.getPropagateDeletes(), cenv, iters, aConfig, tConfig.getCryptoService(), pausedMetrics)); } @@ -582,24 +625,25 @@ public void run() { started.countDown(); org.apache.accumulo.server.compaction.CompactionStats stat = compactor.get().call(); - TCompactionStats cs = new TCompactionStats(); - cs.setEntriesRead(stat.getEntriesRead()); - cs.setEntriesWritten(stat.getEntriesWritten()); - cs.setFileSize(stat.getFileSize()); + PCompactionStats cs = PCompactionStats.newBuilder().setEntriesRead(stat.getEntriesRead()) + .setEntriesWritten(stat.getEntriesWritten()).setFileSize(stat.getFileSize()).build(); JOB_HOLDER.setStats(cs); LOG.info("Compaction completed successfully {} ", job.getExternalCompactionId()); // Update state when completed - TCompactionStatusUpdate update2 = new TCompactionStatusUpdate(TCompactionState.SUCCEEDED, - "Compaction completed successfully", -1, -1, -1, this.getCompactionAge().toNanos()); + PCompactionStatusUpdate update2 = PCompactionStatusUpdate.newBuilder() + .setState(PCompactionState.SUCCEEDED).setMessage("Compaction completed successfully") + .setEntriesToBeCompacted(-1).setEntriesRead(-1).setEntriesWritten(-1) + .setCompactionAgeNanos(this.getCompactionAge().toNanos()).build(); + updateCompactionState(job, update2); } catch (FileCompactor.CompactionCanceledException cce) { LOG.debug("Compaction canceled {}", job.getExternalCompactionId()); err.set(cce); } catch (Exception e) { - KeyExtent fromThriftExtent = KeyExtent.fromThrift(job.getExtent()); + KeyExtent fromProtobufExtent = KeyExtent.fromProtobuf(job.getExtent()); LOG.error("Compaction failed: id: {}, extent: {}", job.getExternalCompactionId(), - fromThriftExtent, e); + fromProtobufExtent, e); err.set(e); } finally { stopped.countDown(); @@ -711,7 +755,6 @@ public void run() { // mark compactor as idle while not in the compaction loop updateIdleStatus(true); - currentCompactionId.set(null); err.set(null); JOB_HOLDER.reset(); @@ -724,11 +767,11 @@ public void run() { nextSortLogsCheckTime = logSorter.sortLogsIfNeeded(); } - TExternalCompactionJob job; + PExternalCompactionJob job; try { - TNextCompactionJob next = getNextJob(getNextId()); + PNextCompactionJob next = getNextJob(getNextId()); job = next.getJob(); - if (!job.isSetExternalCompactionId()) { + if (!job.hasExternalCompactionId()) { LOG.trace("No external compactions in group {}", this.getResourceGroup()); UtilWaitThread.sleep(getWaitTimeBetweenCompactionChecks(next.getCompactorCount())); continue; @@ -789,9 +832,12 @@ public void run() { watcher.run(); try { LOG.debug("Updating coordinator with compaction progress: {}.", message); - TCompactionStatusUpdate update = new TCompactionStatusUpdate( - TCompactionState.IN_PROGRESS, message, inputEntries, entriesRead, - entriesWritten, fcr.getCompactionAge().toNanos()); + PCompactionStatusUpdate update = + PCompactionStatusUpdate.newBuilder().setState(PCompactionState.IN_PROGRESS) + .setMessage(message).setEntriesToBeCompacted(inputEntries) + .setEntriesRead(entriesRead).setEntriesWritten(entriesWritten) + .setCompactionAgeNanos(fcr.getCompactionAge().toNanos()).build(); + updateCompactionState(job, update); } catch (RetriesExceededException e) { LOG.warn("Error updating coordinator with compaction progress, error: {}", @@ -818,9 +864,11 @@ public void run() { || (err.get() != null && err.get().getClass().equals(InterruptedException.class))) { LOG.warn("Compaction thread was interrupted, sending CANCELLED state"); try { - TCompactionStatusUpdate update = - new TCompactionStatusUpdate(TCompactionState.CANCELLED, "Compaction cancelled", - -1, -1, -1, fcr.getCompactionAge().toNanos()); + PCompactionStatusUpdate update = PCompactionStatusUpdate.newBuilder() + .setState(PCompactionState.CANCELLED).setMessage("Compaction cancelled") + .setEntriesToBeCompacted(-1).setEntriesRead(-1).setEntriesWritten(-1) + .setCompactionAgeNanos(fcr.getCompactionAge().toNanos()).build(); + updateCompactionState(job, update); updateCompactionFailed(job); } catch (RetriesExceededException e) { @@ -829,18 +877,21 @@ public void run() { currentCompactionId.set(null); } } else if (err.get() != null) { - KeyExtent fromThriftExtent = KeyExtent.fromThrift(job.getExtent()); + KeyExtent fromProtobufExtent = KeyExtent.fromProtobuf(job.getExtent()); try { LOG.info("Updating coordinator with compaction failure: id: {}, extent: {}", - job.getExternalCompactionId(), fromThriftExtent); - TCompactionStatusUpdate update = new TCompactionStatusUpdate(TCompactionState.FAILED, - "Compaction failed due to: " + err.get().getMessage(), -1, -1, -1, - fcr.getCompactionAge().toNanos()); + job.getExternalCompactionId(), fromProtobufExtent); + PCompactionStatusUpdate update = + PCompactionStatusUpdate.newBuilder().setState(PCompactionState.FAILED) + .setMessage("Compaction failed due to: " + err.get().getMessage()) + .setEntriesToBeCompacted(-1).setEntriesRead(-1).setEntriesWritten(-1) + .setCompactionAgeNanos(fcr.getCompactionAge().toNanos()).build(); + updateCompactionState(job, update); updateCompactionFailed(job); } catch (RetriesExceededException e) { LOG.error("Error updating coordinator with compaction failure: id: {}, extent: {}", - job.getExternalCompactionId(), fromThriftExtent, e); + job.getExternalCompactionId(), fromProtobufExtent, e); } finally { currentCompactionId.set(null); } @@ -964,7 +1015,7 @@ public TExternalCompactionJob getRunningCompaction(TInfo tinfo, TCredentials cre // method is called by a coordinator starting up to determine what is currently running on all // compactors. - TExternalCompactionJob job = null; + PExternalCompactionJob job = null; synchronized (JOB_HOLDER) { job = JOB_HOLDER.getJob(); } @@ -972,7 +1023,7 @@ public TExternalCompactionJob getRunningCompaction(TInfo tinfo, TCredentials cre if (null == job) { return new TExternalCompactionJob(); } else { - return job; + return convert(job); } } diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java index 490e5908ba1..f08f6f5aa21 100644 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java +++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java @@ -18,6 +18,8 @@ */ package org.apache.accumulo.compactor; +import static org.apache.accumulo.core.rpc.grpc.ThriftProtobufUtil.convert; + import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.TableId; @@ -26,18 +28,19 @@ import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.spi.compaction.CompactionKind; import org.apache.accumulo.core.tabletserver.thrift.TCompactionReason; -import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.compaction.FileCompactor.CompactionEnv; import org.apache.accumulo.server.iterators.SystemIteratorEnvironment; import org.apache.accumulo.server.iterators.TabletIteratorEnvironment; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; public class ExtCEnv implements CompactionEnv { private final CompactionJobHolder jobHolder; - private TExternalCompactionJob job; + private PExternalCompactionJob job; private String groupName; public static class CompactorIterEnv extends TabletIteratorEnvironment { @@ -76,7 +79,7 @@ public IteratorScope getIteratorScope() { public SystemIteratorEnvironment createIteratorEnv(ServerContext context, AccumuloConfiguration acuTableConf, TableId tableId) { return new CompactorIterEnv(context, IteratorScope.majc, - !jobHolder.getJob().isPropagateDeletes(), acuTableConf, tableId, + !jobHolder.getJob().getPropagateDeletes(), acuTableConf, tableId, CompactionKind.valueOf(job.getKind().name()), groupName); } @@ -87,7 +90,10 @@ public SortedKeyValueIterator getMinCIterator() { @Override public TCompactionReason getReason() { - switch (job.getKind()) { + var reason = convert(job.getKind()); + Preconditions.checkState(reason != null, "Unknown compaction kind %s", job.getKind()); + + switch (reason) { case USER: return TCompactionReason.USER; case SYSTEM: diff --git a/server/manager/pom.xml b/server/manager/pom.xml index cb52ea5681f..ea3421f7fc6 100644 --- a/server/manager/pom.xml +++ b/server/manager/pom.xml @@ -48,10 +48,22 @@ com.google.guava guava + + com.google.protobuf + protobuf-java + commons-io commons-io + + io.grpc + grpc-api + + + io.grpc + grpc-stub + io.micrometer micrometer-core @@ -68,6 +80,10 @@ org.apache.accumulo accumulo-core + + org.apache.accumulo + accumulo-grpc + org.apache.accumulo accumulo-server-base 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 e3b99b1615a..6313c231ce4 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 @@ -57,6 +57,7 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.cli.ConfigOpts; @@ -133,6 +134,7 @@ import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.compaction.CompactionConfigStorage; import org.apache.accumulo.server.fs.VolumeManager; +import org.apache.accumulo.server.grpc.CompactionCoordinatorServiceServer; import org.apache.accumulo.server.manager.LiveTServerSet; import org.apache.accumulo.server.manager.LiveTServerSet.LiveTServersSnapshot; import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection; @@ -215,6 +217,8 @@ public class Manager extends AbstractServer ServiceLock managerLock = null; private TServer clientService = null; + private CompactionCoordinatorServiceServer grpcClientService = null; + protected volatile TabletBalancer tabletBalancer; private final BalancerEnvironment balancerEnvironment; private final BalancerMetrics balancerMetrics = new BalancerMetrics(); @@ -327,6 +331,7 @@ synchronized void setManagerState(final ManagerState newState) { final var future = getContext().getScheduledExecutor().scheduleWithFixedDelay(() -> { // This frees the main thread and will cause the manager to exit clientService.stop(); + grpcClientService.stop(); Manager.this.nextEvent.event("stopped event loop"); }, 100L, 1000L, MILLISECONDS); ThreadPools.watchNonCriticalScheduledTask(future); @@ -1116,8 +1121,8 @@ public void run() { HighlyAvailableServiceWrapper.service(managerClientHandler, this); ServerAddress sa; - var processor = ThriftProcessorTypes.getManagerTProcessor(fateServiceHandler, - compactionCoordinator.getThriftService(), haProxy, getContext()); + var processor = + ThriftProcessorTypes.getManagerTProcessor(fateServiceHandler, haProxy, getContext()); try { sa = TServerUtils.startServer(context, getHostname(), Property.MANAGER_CLIENTPORT, processor, @@ -1129,6 +1134,20 @@ public void run() { clientService = sa.server; log.info("Started Manager client service at {}", sa.address); + final CompactionCoordinatorServiceServer grpcService; + try { + // Start up the grpc compaction service + // TODO determine property name and default port to use for this, for now just using to + // manager.port.grpc.client + final IntStream port = getConfiguration().getPortStream(Property.MANAGER_GRPC_CLIENTPORT); + grpcService = new CompactionCoordinatorServiceServer(compactionCoordinator.getGrpcService(), + port.findFirst().orElseThrow()); + grpcService.start(); + } catch (IOException e) { + throw new IllegalStateException("Unable to start grpc server on host " + getHostname(), e); + } + grpcClientService = grpcService; + // block until we can obtain the ZK lock for the manager ServiceLockData sld; try { @@ -1315,6 +1334,7 @@ boolean canSuspendTablets() { while (clientService.isServing()) { sleepUninterruptibly(500, MILLISECONDS); } + log.info("Shutting down fate."); getFateRefs().keySet().forEach(type -> fate(type).shutdown(0, MINUTES)); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java index 9b31ca33f17..9a788248e47 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java @@ -27,6 +27,7 @@ import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.OPID; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.SELECTED; +import static org.apache.accumulo.core.rpc.grpc.ThriftProtobufUtil.convert; import java.io.FileNotFoundException; import java.io.IOException; @@ -43,6 +44,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledFuture; @@ -59,22 +61,14 @@ import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.client.admin.compaction.CompactableFile; import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode; -import org.apache.accumulo.core.clientImpl.thrift.TInfo; import org.apache.accumulo.core.clientImpl.thrift.TableOperation; import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType; import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; -import org.apache.accumulo.core.compaction.thrift.TCompactionState; -import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; -import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; -import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; -import org.apache.accumulo.core.compaction.thrift.TNextCompactionJob; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.NamespaceId; 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.FateId; import org.apache.accumulo.core.fate.FateInstanceType; @@ -96,17 +90,11 @@ import org.apache.accumulo.core.metadata.schema.TabletsMetadata; import org.apache.accumulo.core.metadata.schema.filters.HasExternalCompactionsFilter; import org.apache.accumulo.core.metrics.MetricsProducer; -import org.apache.accumulo.core.securityImpl.thrift.TCredentials; import org.apache.accumulo.core.spi.compaction.CompactionJob; import org.apache.accumulo.core.spi.compaction.CompactionKind; import org.apache.accumulo.core.spi.compaction.CompactionPlanner; import org.apache.accumulo.core.spi.compaction.CompactionServiceId; import org.apache.accumulo.core.spi.compaction.CompactorGroupId; -import org.apache.accumulo.core.tabletserver.thrift.InputFile; -import org.apache.accumulo.core.tabletserver.thrift.IteratorConfig; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; -import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.core.util.cache.Caches.CacheName; @@ -118,6 +106,25 @@ import org.apache.accumulo.core.util.threads.Threads; import org.apache.accumulo.core.util.time.SteadyTime; import org.apache.accumulo.core.volume.Volume; +import org.apache.accumulo.grpc.compaction.protobuf.CancelRequest; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCompletedRequest; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceGrpc; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionFailedRequest; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionJobRequest; +import org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest; +import org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionKind; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionState; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate; +import org.apache.accumulo.grpc.compaction.protobuf.PCredentials; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList; +import org.apache.accumulo.grpc.compaction.protobuf.PInputFile; +import org.apache.accumulo.grpc.compaction.protobuf.PIteratorConfig; +import org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo; +import org.apache.accumulo.grpc.compaction.protobuf.UpdateCompactionStatusRequest; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.compaction.coordinator.commit.CommitCompaction; import org.apache.accumulo.manager.compaction.coordinator.commit.CompactionCommitData; @@ -133,7 +140,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.thrift.TException; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -146,12 +152,15 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Sets; import com.google.common.net.HostAndPort; +import com.google.protobuf.Empty; +import io.grpc.Status; +import io.grpc.StatusRuntimeException; +import io.grpc.stub.StreamObserver; import io.micrometer.core.instrument.Gauge; import io.micrometer.core.instrument.MeterRegistry; -public class CompactionCoordinator - implements CompactionCoordinatorService.Iface, Runnable, MetricsProducer { +public class CompactionCoordinator implements Runnable, MetricsProducer { private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class); @@ -192,6 +201,7 @@ public class CompactionCoordinator private final int jobQueueInitialSize; private volatile long coordinatorStartTime; + private final GrpcCompactionCoordinatorService grpcService; public CompactionCoordinator(ServerContext ctx, SecurityOperation security, AtomicReference>> fateInstances, Manager manager) { @@ -235,6 +245,8 @@ public CompactionCoordinator(ServerContext ctx, SecurityOperation security, compactorCounts = ctx.getCaches().createNewBuilder(CacheName.COMPACTOR_COUNTS, false) .expireAfterWrite(30, TimeUnit.SECONDS).build(this::countCompactors); // At this point the manager does not have its lock so no actions should be taken yet + + grpcService = new GrpcCompactionCoordinatorService(); } protected int countCompactors(String groupName) { @@ -289,9 +301,8 @@ public void run() { } else { LOG.info("Found {} running external compactions", running.size()); running.forEach(rc -> { - TCompactionStatusUpdate update = new TCompactionStatusUpdate(); - update.setState(TCompactionState.IN_PROGRESS); - update.setMessage(RESTART_UPDATE_MSG); + PCompactionStatusUpdate update = PCompactionStatusUpdate.newBuilder() + .setState(PCompactionState.IN_PROGRESS).setMessage(RESTART_UPDATE_MSG).build(); rc.addUpdate(System.currentTimeMillis(), update); RUNNING_CACHE.put(ExternalCompactionId.of(rc.getJob().getExternalCompactionId()), rc); }); @@ -350,21 +361,12 @@ public long getNumRunningCompactions() { return RUNNING_CACHE.size(); } - /** - * Return the next compaction job from the queue to a Compactor - * - * @param groupName group - * @param compactorAddress compactor address - * @throws ThriftSecurityException when permission error - * @return compaction job - */ - @Override - public TNextCompactionJob getCompactionJob(TInfo tinfo, TCredentials credentials, - String groupName, String compactorAddress, String externalCompactionId) - throws ThriftSecurityException { + public CompletableFuture getAsyncCompactionJob(ProtoTInfo ptinfo, + PCredentials credentials, String groupName, String compactorAddress, + String externalCompactionId) throws ThriftSecurityException { // do not expect users to call this directly, expect compactors to call this method - if (!security.canPerformSystemActions(credentials)) { + if (!security.canPerformSystemActions(convert(credentials))) { throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED).asThriftException(); } @@ -372,12 +374,8 @@ public TNextCompactionJob getCompactionJob(TInfo tinfo, TCredentials credentials LOG.trace("getCompactionJob called for group {} by compactor {}", groupId, compactorAddress); TIME_COMPACTOR_LAST_CHECKED.put(groupId, System.currentTimeMillis()); - TExternalCompactionJob result = null; - - CompactionJobQueues.MetaJob metaJob = jobQueues.poll(groupId); - - while (metaJob != null) { - + return jobQueues.getAsync(groupId).thenApply(metaJob -> { + LOG.trace("Next metaJob is ready {}", metaJob.getJob()); Optional compactionConfig = getCompactionConfig(metaJob); // this method may reread the metadata, do not use the metadata in metaJob for anything after @@ -394,34 +392,26 @@ public TNextCompactionJob getCompactionJob(TInfo tinfo, TCredentials credentials ecm = reserveCompaction(metaJob, compactorAddress, cid); } + final PExternalCompactionJob result; if (ecm != null) { - result = createThriftJob(externalCompactionId, ecm, metaJob, compactionConfig); + result = createRpcJob(externalCompactionId, ecm, metaJob, compactionConfig); // It is possible that by the time this added that the the compactor that made this request // is dead. In this cases the compaction is not actually running. RUNNING_CACHE.put(ExternalCompactionId.of(result.getExternalCompactionId()), new RunningCompaction(result, compactorAddress, groupName)); TabletLogger.compacting(metaJob.getTabletMetadata(), cid, compactorAddress, metaJob.getJob()); - break; + LOG.info("Found job {}", result.getExternalCompactionId()); } else { - LOG.debug( - "Unable to reserve compaction job for {}, pulling another off the queue for group {}", - metaJob.getTabletMetadata().getExtent(), groupName); - metaJob = jobQueues.poll(CompactorGroupId.of(groupName)); + LOG.debug("Unable to reserve compaction job for {} {}, returning empty job to compactor {}", + groupName, metaJob.getTabletMetadata().getExtent(), compactorAddress); + result = PExternalCompactionJob.newBuilder().build(); } - } - - if (metaJob == null) { - LOG.debug("No jobs found in group {} ", groupName); - } - if (result == null) { - LOG.trace("No jobs found for group {}, returning empty job to compactor {}", groupName, - compactorAddress); - result = new TExternalCompactionJob(); - } + return PNextCompactionJob.newBuilder().setJob(result) + .setCompactorCount(compactorCounts.get(groupName)).build(); + }); - return new TNextCompactionJob(result, compactorCounts.get(groupName)); } @VisibleForTesting @@ -604,9 +594,8 @@ protected CompactionMetadata reserveCompaction(CompactionJobQueues.MetaJob metaJ return null; } - protected TExternalCompactionJob createThriftJob(String externalCompactionId, - CompactionMetadata ecm, CompactionJobQueues.MetaJob metaJob, - Optional compactionConfig) { + protected PExternalCompactionJob createRpcJob(String externalCompactionId, CompactionMetadata ecm, + CompactionJobQueues.MetaJob metaJob, Optional compactionConfig) { Set selectedFiles; if (metaJob.getJob().getKind() == CompactionKind.SYSTEM) { @@ -621,13 +610,14 @@ protected TExternalCompactionJob createThriftJob(String externalCompactionId, Map overrides = CompactionPluginUtils.computeOverrides(compactionConfig, ctx, metaJob.getTabletMetadata().getExtent(), metaJob.getJob().getFiles(), selectedFiles); - IteratorConfig iteratorSettings = SystemIteratorUtil + PIteratorConfig iteratorSettings = SystemIteratorUtil .toIteratorConfig(compactionConfig.map(CompactionConfig::getIterators).orElse(List.of())); var files = ecm.getJobFiles().stream().map(storedTabletFile -> { var dfv = metaJob.getTabletMetadata().getFilesMap().get(storedTabletFile); - return new InputFile(storedTabletFile.getMetadata(), dfv.getSize(), dfv.getNumEntries(), - dfv.getTime()); + return PInputFile.newBuilder().setMetadataFileEntry(storedTabletFile.getMetadata()) + .setSize(dfv.getSize()).setEntries(dfv.getNumEntries()).setTimestamp(dfv.getTime()) + .build(); }).collect(toList()); // The fateId here corresponds to the Fate transaction that is driving a user initiated @@ -639,11 +629,30 @@ protected TExternalCompactionJob createThriftJob(String externalCompactionId, fateId = metaJob.getTabletMetadata().getSelectedFiles().getFateId(); } - return new TExternalCompactionJob(externalCompactionId, - metaJob.getTabletMetadata().getExtent().toThrift(), files, iteratorSettings, - ecm.getCompactTmpName().getNormalizedPathStr(), ecm.getPropagateDeletes(), - TCompactionKind.valueOf(ecm.getKind().name()), fateId == null ? null : fateId.toThrift(), - overrides); + PExternalCompactionJob.Builder builder = + PExternalCompactionJob.newBuilder().setExternalCompactionId(externalCompactionId) + .setExtent(metaJob.getTabletMetadata().getExtent().toProtobuf()).addAllFiles(files) + .setIteratorSettings(iteratorSettings) + .setOutputFile(ecm.getCompactTmpName().getNormalizedPathStr()) + .setPropagateDeletes(ecm.getPropagateDeletes()).setKind(convertKind(ecm.getKind())) + .putAllOverrides(overrides); + + if (fateId != null) { + builder.setFateId(convert(fateId.toThrift())); + } + + return builder.build(); + } + + protected static PCompactionKind convertKind(CompactionKind kind) { + switch (kind) { + case SYSTEM: + return PCompactionKind.SYSTEM; + case USER: + return PCompactionKind.USER; + default: + throw new IllegalArgumentException("Unexpected TCompactionKind: " + kind); + } } @Override @@ -662,10 +671,6 @@ public void addJobs(TabletMetadata tabletMetadata, Collection job jobQueues.add(tabletMetadata, jobs); } - public CompactionCoordinatorService.Iface getThriftService() { - return this; - } - private Optional getCompactionConfig(CompactionJobQueues.MetaJob metaJob) { if (metaJob.getJob().getKind() == CompactionKind.USER && metaJob.getTabletMetadata().getSelectedFiles() != null) { @@ -676,111 +681,6 @@ private Optional getCompactionConfig(CompactionJobQueues.MetaJ return Optional.empty(); } - /** - * Compactors calls this method when they have finished a compaction. This method does the - * following. - * - *
    - *
  1. Reads the tablets metadata and determines if the compaction can commit. Its possible that - * things changed while the compaction was running and it can no longer commit.
  2. - *
  3. Commit the compaction using a conditional mutation. If the tablets files or location - * changed since reading the tablets metadata, then conditional mutation will fail. When this - * happens it will reread the metadata and go back to step 1 conceptually. When committing a - * compaction the compacted files are removed and scan entries are added to the tablet in case the - * files are in use, this prevents GC from deleting the files between updating tablet metadata and - * refreshing the tablet. The scan entries are only added when a tablet has a location.
  4. - *
  5. After successful commit a refresh request is sent to the tablet if it has a location. This - * will cause the tablet to start using the newly compacted files for future scans. Also the - * tablet can delete the scan entries if there are no active scans using them.
  6. - *
- * - *

- * User compactions will be refreshed as part of the fate operation. The user compaction fate - * operation will see the compaction was committed after this code updates the tablet metadata, - * however if it were to rely on this code to do the refresh it would not be able to know when the - * refresh was actually done. Therefore, user compactions will refresh as part of the fate - * operation so that it's known to be done before the fate operation returns. Since the fate - * operation will do it, there is no need to do it here for user compactions. - * - * @param tinfo trace info - * @param credentials tcredentials object - * @param externalCompactionId compaction id - * @param textent tablet extent - * @param stats compaction stats - * @throws ThriftSecurityException when permission error - */ - @Override - public void compactionCompleted(TInfo tinfo, TCredentials credentials, - String externalCompactionId, TKeyExtent textent, TCompactionStats stats) - throws ThriftSecurityException { - // do not expect users to call this directly, expect other tservers to call this method - if (!security.canPerformSystemActions(credentials)) { - throw new AccumuloSecurityException(credentials.getPrincipal(), - SecurityErrorCode.PERMISSION_DENIED).asThriftException(); - } - - // maybe fate has not started yet - var localFates = fateInstances.get(); - while (localFates == null) { - UtilWaitThread.sleep(100); - if (shutdown.getCount() == 0) { - return; - } - localFates = fateInstances.get(); - } - - var extent = KeyExtent.fromThrift(textent); - var localFate = localFates.get(FateInstanceType.fromTableId(extent.tableId())); - - LOG.info("Compaction completed, id: {}, stats: {}, extent: {}", externalCompactionId, stats, - extent); - final var ecid = ExternalCompactionId.of(externalCompactionId); - - var tabletMeta = - ctx.getAmple().readTablet(extent, ECOMP, SELECTED, LOCATION, FILES, COMPACTED, OPID); - - var tableState = manager.getContext().getTableState(extent.tableId()); - if (tableState != TableState.ONLINE) { - // Its important this check is done after the compaction id is set in the metadata table to - // avoid race conditions with the client code that waits for tables to go offline. That code - // looks for compaction ids in the metadata table after setting the table state. When that - // client code sees nothing for a tablet its important that nothing will changes the tablets - // files after that point in time which this check ensure. - LOG.debug("Not committing compaction {} for {} because of table state {}", ecid, extent, - tableState); - // cleanup metadata table and files related to the compaction - compactionsFailed(Map.of(ecid, extent)); - return; - } - - if (!CommitCompaction.canCommitCompaction(ecid, tabletMeta)) { - return; - } - - // Start a fate transaction to commit the compaction. - CompactionMetadata ecm = tabletMeta.getExternalCompactions().get(ecid); - var renameOp = new RenameCompactionFile(new CompactionCommitData(ecid, extent, ecm, stats)); - var txid = localFate.seedTransaction("COMMIT_COMPACTION", FateKey.forCompactionCommit(ecid), - renameOp, true, "Commit compaction " + ecid); - - txid.ifPresentOrElse(fateId -> LOG.debug("initiated compaction commit {} {}", ecid, fateId), - () -> LOG.debug("compaction commit already initiated for {}", ecid)); - } - - @Override - public void compactionFailed(TInfo tinfo, TCredentials credentials, String externalCompactionId, - TKeyExtent extent) throws ThriftSecurityException { - // do not expect users to call this directly, expect other tservers to call this method - if (!security.canPerformSystemActions(credentials)) { - throw new AccumuloSecurityException(credentials.getPrincipal(), - SecurityErrorCode.PERMISSION_DENIED).asThriftException(); - } - KeyExtent fromThriftExtent = KeyExtent.fromThrift(extent); - LOG.info("Compaction failed, id: {}, extent: {}", externalCompactionId, fromThriftExtent); - final var ecid = ExternalCompactionId.of(externalCompactionId); - compactionsFailed(Map.of(ecid, KeyExtent.fromThrift(extent))); - } - void compactionsFailed(Map compactions) { // Need to process each level by itself because the conditional tablet mutator does not support // mutating multiple data levels at the same time @@ -882,33 +782,6 @@ public boolean test(TabletMetadata tabletMetadata) { compactions.forEach((k, v) -> recordCompletion(k)); } - /** - * Compactor calls to update the status of the assigned compaction - * - * @param tinfo trace info - * @param credentials tcredentials object - * @param externalCompactionId compaction id - * @param update compaction status update - * @param timestamp timestamp of the message - * @throws ThriftSecurityException when permission error - */ - @Override - public void updateCompactionStatus(TInfo tinfo, TCredentials credentials, - String externalCompactionId, TCompactionStatusUpdate update, long timestamp) - throws ThriftSecurityException { - // do not expect users to call this directly, expect other tservers to call this method - if (!security.canPerformSystemActions(credentials)) { - throw new AccumuloSecurityException(credentials.getPrincipal(), - SecurityErrorCode.PERMISSION_DENIED).asThriftException(); - } - LOG.debug("Compaction status update, id: {}, timestamp: {}, update: {}", externalCompactionId, - timestamp, update); - final RunningCompaction rc = RUNNING_CACHE.get(ExternalCompactionId.of(externalCompactionId)); - if (null != rc) { - rc.addUpdate(timestamp, update); - } - } - public void recordCompletion(ExternalCompactionId ecid) { var rc = RUNNING_CACHE.remove(ecid); if (rc != null) { @@ -926,82 +799,8 @@ protected Set readExternalCompactionIds() { } /** - * Return information about running compactions - * - * @param tinfo trace info - * @param credentials tcredentials object - * @return map of ECID to TExternalCompaction objects - * @throws ThriftSecurityException permission error + * /* Method exists to be called from test */ - @Override - public TExternalCompactionList getRunningCompactions(TInfo tinfo, TCredentials credentials) - throws ThriftSecurityException { - // do not expect users to call this directly, expect other tservers to call this method - if (!security.canPerformSystemActions(credentials)) { - throw new AccumuloSecurityException(credentials.getPrincipal(), - SecurityErrorCode.PERMISSION_DENIED).asThriftException(); - } - - final TExternalCompactionList result = new TExternalCompactionList(); - RUNNING_CACHE.forEach((ecid, rc) -> { - TExternalCompaction trc = new TExternalCompaction(); - trc.setGroupName(rc.getGroupName()); - trc.setCompactor(rc.getCompactorAddress()); - trc.setUpdates(rc.getUpdates()); - trc.setJob(rc.getJob()); - result.putToCompactions(ecid.canonical(), trc); - }); - return result; - } - - /** - * Return information about recently completed compactions - * - * @param tinfo trace info - * @param credentials tcredentials object - * @return map of ECID to TExternalCompaction objects - * @throws ThriftSecurityException permission error - */ - @Override - public TExternalCompactionList getCompletedCompactions(TInfo tinfo, TCredentials credentials) - throws ThriftSecurityException { - // do not expect users to call this directly, expect other tservers to call this method - if (!security.canPerformSystemActions(credentials)) { - throw new AccumuloSecurityException(credentials.getPrincipal(), - SecurityErrorCode.PERMISSION_DENIED).asThriftException(); - } - final TExternalCompactionList result = new TExternalCompactionList(); - completed.asMap().forEach((ecid, rc) -> { - TExternalCompaction trc = new TExternalCompaction(); - trc.setGroupName(rc.getGroupName()); - trc.setCompactor(rc.getCompactorAddress()); - trc.setJob(rc.getJob()); - trc.setUpdates(rc.getUpdates()); - result.putToCompactions(ecid.canonical(), trc); - }); - return result; - } - - @Override - public void cancel(TInfo tinfo, TCredentials credentials, String externalCompactionId) - throws TException { - var runningCompaction = RUNNING_CACHE.get(ExternalCompactionId.of(externalCompactionId)); - var extent = KeyExtent.fromThrift(runningCompaction.getJob().getExtent()); - try { - NamespaceId nsId = this.ctx.getNamespaceId(extent.tableId()); - if (!security.canCompact(credentials, extent.tableId(), nsId)) { - throw new AccumuloSecurityException(credentials.getPrincipal(), - SecurityErrorCode.PERMISSION_DENIED).asThriftException(); - } - } catch (TableNotFoundException e) { - throw new ThriftTableOperationException(extent.tableId().canonical(), null, - TableOperation.COMPACT_CANCEL, TableOperationExceptionType.NOTFOUND, e.getMessage()); - } - - cancelCompactionOnCompactor(runningCompaction.getCompactorAddress(), externalCompactionId); - } - - /* Method exists to be called from test */ public CompactionJobQueues getJobQueues() { return jobQueues; } @@ -1236,4 +1035,332 @@ private static Set getFilesReservedBySelection(TabletMetadata return Set.of(); } + + public CompactionCoordinatorServiceGrpc.CompactionCoordinatorServiceImplBase getGrpcService() { + return new GrpcCompactionCoordinatorService(); + } + + private class GrpcCompactionCoordinatorService + extends CompactionCoordinatorServiceGrpc.CompactionCoordinatorServiceImplBase { + + @Override + public void getCompactionJob(CompactionJobRequest request, + StreamObserver responseObserver) { + + var credentials = request.getCredentials(); + var ptinfo = request.getPtinfo(); + + try { + LOG.debug("Received compaction job grpc {}", request.getExternalCompactionId()); + + // Get the next job as a future as we need to wait until something is available + var result = CompactionCoordinator.this.getAsyncCompactionJob(ptinfo, credentials, + request.getGroupName(), request.getCompactor(), request.getExternalCompactionId()); + + // Async send back to the compactor when a new job is ready + // Need the unused var for errorprone + var unused = result.thenAccept(ecj -> { + LOG.debug("Received next compaction job {}", ecj); + responseObserver.onNext(ecj); + responseObserver.onCompleted(); + }).exceptionally(e -> { + LOG.warn("Received exception processing compaction job {}", e.getMessage()); + LOG.debug(e.getMessage(), e); + responseObserver.onError(new StatusRuntimeException(Status.INTERNAL)); + return null; + }); + + } catch (ThriftSecurityException e) { + throw new RuntimeException(e); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + throw e; + } + } + + /** + * Compactors calls this method when they have finished a compaction. This method does the + * following. + * + *

    + *
  1. Reads the tablets metadata and determines if the compaction can commit. Its possible that + * things changed while the compaction was running and it can no longer commit.
  2. + *
  3. Commit the compaction using a conditional mutation. If the tablets files or location + * changed since reading the tablets metadata, then conditional mutation will fail. When this + * happens it will reread the metadata and go back to step 1 conceptually. When committing a + * compaction the compacted files are removed and scan entries are added to the tablet in case + * the files are in use, this prevents GC from deleting the files between updating tablet + * metadata and refreshing the tablet. The scan entries are only added when a tablet has a + * location.
  4. + *
  5. After successful commit a refresh request is sent to the tablet if it has a location. + * This will cause the tablet to start using the newly compacted files for future scans. Also + * the tablet can delete the scan entries if there are no active scans using them.
  6. + *
+ * + *

+ * User compactions will be refreshed as part of the fate operation. The user compaction fate + * operation will see the compaction was committed after this code updates the tablet metadata, + * however if it were to rely on this code to do the refresh it would not be able to know when + * the refresh was actually done. Therefore, user compactions will refresh as part of the fate + * operation so that it's known to be done before the fate operation returns. Since the fate + * operation will do it, there is no need to do it here for user compactions. + * + * @param request CompactionCompletedRequest + */ + @Override + public void compactionCompleted(CompactionCompletedRequest request, + StreamObserver responseObserver) { + + // TODO: Do we want to offload this processing to a new thread like we plan to do with + // getCompactionJob() ? + + var credentials = convert(request.getCredentials()); + + try { + // do not expect users to call this directly, expect other tservers to call this method + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + + // maybe fate has not started yet + var localFates = fateInstances.get(); + while (localFates == null) { + UtilWaitThread.sleep(100); + if (shutdown.getCount() == 0) { + return; + } + localFates = fateInstances.get(); + } + + var extent = KeyExtent.fromProtobuf(request.getExtent()); + var localFate = localFates.get(FateInstanceType.fromTableId(extent.tableId())); + + LOG.info("Compaction completed, id: {}, stats: {}, extent: {}", + request.getExternalCompactionId(), request.getStats(), extent); + final var ecid = ExternalCompactionId.of(request.getExternalCompactionId()); + + var tabletMeta = + ctx.getAmple().readTablet(extent, ECOMP, SELECTED, LOCATION, FILES, COMPACTED, OPID); + + var tableState = manager.getContext().getTableState(extent.tableId()); + if (tableState != TableState.ONLINE) { + // Its important this check is done after the compaction id is set in the metadata table + // to avoid race conditions with the client code that waits for tables to go offline. + // That codelooks for compaction ids in the metadata table after setting the table + // state. When that client code sees nothing for a tablet its important that nothing + // will change the tablets files after that point in time which this check ensure. + LOG.debug("Not committing compaction {} for {} because of table state {}", ecid, extent, + tableState); + // cleanup metadata table and files related to the compaction + compactionsFailed(Map.of(ecid, extent)); + return; + } + + if (!CommitCompaction.canCommitCompaction(ecid, tabletMeta)) { + return; + } + + // Start a fate transaction to commit the compaction. + CompactionMetadata ecm = tabletMeta.getExternalCompactions().get(ecid); + var renameOp = new RenameCompactionFile( + new CompactionCommitData(ecid, extent, ecm, request.getStats())); + var txid = localFate.seedTransaction("COMMIT_COMPACTION", FateKey.forCompactionCommit(ecid), + renameOp, true, "Commit compaction " + ecid); + + txid.ifPresentOrElse(fateId -> LOG.debug("initiated compaction commit {} {}", ecid, fateId), + () -> LOG.debug("compaction commit already initiated for {}", ecid)); + + responseObserver.onNext(Empty.getDefaultInstance()); + responseObserver.onCompleted(); + } catch (ThriftSecurityException e) { + responseObserver.onError(e); + throw new RuntimeException(e); + } catch (Exception e) { + responseObserver.onError(e); + LOG.error(e.getMessage(), e); + throw e; + } + } + + @Override + public void updateCompactionStatus(UpdateCompactionStatusRequest request, + StreamObserver responseObserver) { + + // TODO: Do we want to offload this processing to a new thread like we plan to do with + // getCompactionJob() ? + + var credentials = convert(request.getCredentials()); + + try { + // do not expect users to call this directly, expect other tservers to call this method + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + LOG.debug("Compaction status update, id: {}, timestamp: {}, update: {}", + request.getExternalCompactionId(), request.getTimestamp(), request.getStatus()); + final RunningCompaction rc = + RUNNING_CACHE.get(ExternalCompactionId.of(request.getExternalCompactionId())); + if (null != rc) { + rc.addUpdate(request.getTimestamp(), request.getStatus()); + } + responseObserver.onNext(Empty.getDefaultInstance()); + responseObserver.onCompleted(); + } catch (ThriftSecurityException e) { + responseObserver.onError(e); + throw new RuntimeException(e); + } catch (Exception e) { + responseObserver.onError(e); + LOG.error(e.getMessage(), e); + throw e; + } + } + + @Override + public void compactionFailed(CompactionFailedRequest request, + StreamObserver responseObserver) { + + // TODO: Do we want to offload this processing to a new thread like we plan to do with + // getCompactionJob() ? + + var credentials = convert(request.getCredentials()); + + try { + // do not expect users to call this directly, expect other tservers to call this method + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + KeyExtent extent = KeyExtent.fromProtobuf(request.getExtent()); + LOG.info("Compaction failed, id: {}, extent: {}", request.getExternalCompactionId(), + extent); + final var ecid = ExternalCompactionId.of(request.getExternalCompactionId()); + compactionsFailed(Map.of(ecid, extent)); + + responseObserver.onNext(Empty.getDefaultInstance()); + responseObserver.onCompleted(); + } catch (ThriftSecurityException e) { + responseObserver.onError(e); + throw new RuntimeException(e); + } catch (Exception e) { + responseObserver.onError(e); + LOG.error(e.getMessage(), e); + throw e; + } + } + + @Override + public void getRunningCompactions(GetRunningCompactionsRequest request, + StreamObserver responseObserver) { + + // TODO: Do we want to offload this processing to a new thread like we plan to do with + // getCompactionJob() ? + + var credentials = convert(request.getCredentials()); + + try { + // do not expect users to call this directly, expect other tservers to call this method + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + + final PExternalCompactionList.Builder resultBuilder = PExternalCompactionList.newBuilder(); + RUNNING_CACHE.forEach((ecid, rc) -> { + PExternalCompaction prc = PExternalCompaction.newBuilder().setGroupName(rc.getGroupName()) + .setCompactor(rc.getCompactorAddress()).setJob(rc.getJob()) + .putAllUpdates(rc.getUpdates()).build(); + + LOG.info("Running compaction: {}", prc); + resultBuilder.putCompactions(ecid.canonical(), prc); + }); + responseObserver.onNext(resultBuilder.build()); + responseObserver.onCompleted(); + } catch (ThriftSecurityException e) { + responseObserver.onError(e); + throw new RuntimeException(e); + } catch (Exception e) { + responseObserver.onError(e); + LOG.error(e.getMessage(), e); + throw e; + } + } + + /** + * Return information about recently completed compactions + * + */ + @Override + public void getCompletedCompactions(GetCompletedCompactionsRequest request, + StreamObserver responseObserver) { + + // TODO: Do we want to offload this processing to a new thread like we plan to do with + // getCompactionJob() ? + + var credentials = convert(request.getCredentials()); + + try { + // do not expect users to call this directly, expect other tservers to call this method + if (!security.canPerformSystemActions(credentials)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + final PExternalCompactionList.Builder resultBuilder = PExternalCompactionList.newBuilder(); + + completed.asMap().forEach((ecid, rc) -> { + PExternalCompaction prc = PExternalCompaction.newBuilder().setGroupName(rc.getGroupName()) + .setCompactor(rc.getCompactorAddress()).setJob(rc.getJob()) + .putAllUpdates(rc.getUpdates()).build(); + resultBuilder.putCompactions(ecid.canonical(), prc); + }); + responseObserver.onNext(resultBuilder.build()); + responseObserver.onCompleted(); + } catch (ThriftSecurityException e) { + responseObserver.onError(e); + throw new RuntimeException(e); + } catch (Exception e) { + responseObserver.onError(e); + LOG.error(e.getMessage(), e); + throw e; + } + } + + @Override + public void cancel(CancelRequest request, StreamObserver responseObserver) { + // TODO: Do we want to offload this processing to a new thread like we plan to do with + // getCompactionJob() ? + + var credentials = convert(request.getCredentials()); + + try { + var runningCompaction = + RUNNING_CACHE.get(ExternalCompactionId.of(request.getExternalCompactionId())); + var extent = KeyExtent.fromProtobuf(runningCompaction.getJob().getExtent()); + try { + NamespaceId nsId = CompactionCoordinator.this.ctx.getNamespaceId(extent.tableId()); + if (!security.canCompact(credentials, extent.tableId(), nsId)) { + throw new AccumuloSecurityException(credentials.getPrincipal(), + SecurityErrorCode.PERMISSION_DENIED).asThriftException(); + } + } catch (TableNotFoundException e) { + throw new ThriftTableOperationException(extent.tableId().canonical(), null, + TableOperation.COMPACT_CANCEL, TableOperationExceptionType.NOTFOUND, e.getMessage()); + } + + cancelCompactionOnCompactor(runningCompaction.getCompactorAddress(), + request.getExternalCompactionId()); + + responseObserver.onNext(Empty.getDefaultInstance()); + responseObserver.onCompleted(); + } catch (ThriftSecurityException | ThriftTableOperationException e) { + responseObserver.onError(e); + throw new RuntimeException(e); + } catch (Exception e) { + responseObserver.onError(e); + LOG.error(e.getMessage(), e); + throw e; + } + } + } } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CommitCompaction.java b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CommitCompaction.java index 0daa221f1f6..0e390078ec6 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CommitCompaction.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CommitCompaction.java @@ -45,8 +45,8 @@ import org.apache.accumulo.core.metadata.schema.SelectedFiles; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.spi.compaction.CompactionKind; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; import org.apache.accumulo.core.util.Retry; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.tableOps.ManagerRepo; import org.apache.accumulo.server.ServerContext; @@ -157,7 +157,7 @@ private TabletMetadata commitCompaction(ServerContext ctx, ExternalCompactionId return tablet; } - private void updateTabletForCompaction(TCompactionStats stats, ExternalCompactionId ecid, + private void updateTabletForCompaction(PCompactionStats stats, ExternalCompactionId ecid, TabletMetadata tablet, Optional newDatafile, CompactionMetadata ecm, Ample.ConditionalTabletMutator tabletMutator) { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CompactionCommitData.java b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CompactionCommitData.java index 23b293c25e9..a57c15cabd4 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CompactionCommitData.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/commit/CompactionCommitData.java @@ -30,7 +30,7 @@ import org.apache.accumulo.core.metadata.schema.CompactionMetadata; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; import org.apache.accumulo.core.spi.compaction.CompactionKind; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStats; public class CompactionCommitData implements Serializable { private static final long serialVersionUID = 1L; @@ -39,10 +39,10 @@ public class CompactionCommitData implements Serializable { final String outputTmpPath; final String ecid; final TKeyExtent textent; - final TCompactionStats stats; + final PCompactionStats stats; public CompactionCommitData(ExternalCompactionId ecid, KeyExtent extent, CompactionMetadata ecm, - TCompactionStats stats) { + PCompactionStats stats) { this.ecid = ecid.canonical(); this.textent = extent.toThrift(); this.kind = ecm.getKind(); diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java index 90c100aa729..817da9a740e 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/compaction/CompactionCoordinatorTest.java @@ -22,6 +22,7 @@ import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.OPID; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.SELECTED; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.USER_COMPACTION_REQUESTED; +import static org.apache.accumulo.core.rpc.grpc.ThriftProtobufUtil.convert; import static org.apache.accumulo.manager.compaction.coordinator.CompactionCoordinator.canReserveCompaction; import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.createMock; @@ -30,7 +31,6 @@ import static org.easymock.EasyMock.replay; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import java.net.URI; @@ -42,16 +42,13 @@ import java.util.Optional; import java.util.Set; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import org.apache.accumulo.core.client.admin.CompactionConfig; -import org.apache.accumulo.core.clientImpl.thrift.TInfo; -import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; -import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; -import org.apache.accumulo.core.compaction.thrift.TNextCompactionJob; import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.TableId; @@ -77,14 +74,16 @@ import org.apache.accumulo.core.spi.compaction.CompactionJob; import org.apache.accumulo.core.spi.compaction.CompactionKind; import org.apache.accumulo.core.spi.compaction.CompactorGroupId; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind; -import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats; -import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.cache.Caches; import org.apache.accumulo.core.util.compaction.CompactionJobImpl; import org.apache.accumulo.core.util.compaction.RunningCompaction; import org.apache.accumulo.core.util.time.SteadyTime; +import org.apache.accumulo.grpc.compaction.protobuf.PCredentials; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.PNextCompactionJob; +import org.apache.accumulo.grpc.compaction.protobuf.ProtoTInfo; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.compaction.coordinator.CompactionCoordinator; import org.apache.accumulo.manager.compaction.queue.CompactionJobPriorityQueue; @@ -157,15 +156,6 @@ protected void startInternalStateCleaner(ScheduledThreadPoolExecutor schedExecut this.shutdown.countDown(); } - @Override - public void compactionCompleted(TInfo tinfo, TCredentials credentials, - String externalCompactionId, TKeyExtent textent, TCompactionStats stats) - throws ThriftSecurityException {} - - @Override - public void compactionFailed(TInfo tinfo, TCredentials credentials, String externalCompactionId, - TKeyExtent extent) throws ThriftSecurityException {} - void setMetadataCompactionIds(Set mci) { metadataCompactionIds = mci; } @@ -219,18 +209,19 @@ protected CompactionMetadata createExternalCompactionMetadata(CompactionJob job, } @Override - protected TExternalCompactionJob createThriftJob(String externalCompactionId, + protected PExternalCompactionJob createRpcJob(String externalCompactionId, CompactionMetadata ecm, MetaJob metaJob, Optional compactionConfig) { - return new TExternalCompactionJob(externalCompactionId, - metaJob.getTabletMetadata().getExtent().toThrift(), List.of(), - SystemIteratorUtil.toIteratorConfig(List.of()), - ecm.getCompactTmpName().getNormalizedPathStr(), ecm.getPropagateDeletes(), - TCompactionKind.valueOf(ecm.getKind().name()), - FateId - .from(FateInstanceType.fromTableId(metaJob.getTabletMetadata().getExtent().tableId()), - UUID.randomUUID()) - .toThrift(), - Map.of()); + return PExternalCompactionJob.newBuilder().setExternalCompactionId(externalCompactionId) + .setExtent(metaJob.getTabletMetadata().getExtent().toProtobuf()) + .setIteratorSettings(SystemIteratorUtil.toIteratorConfig(List.of())) + .setOutputFile(ecm.getCompactTmpName().getNormalizedPathStr()) + .setPropagateDeletes(ecm.getPropagateDeletes()).setKind(convertKind(ecm.getKind())) + .setFateId( + convert(FateId + .from(FateInstanceType.fromTableId( + metaJob.getTabletMetadata().getExtent().tableId()), UUID.randomUUID()) + .toThrift())) + .build(); } @Override @@ -279,8 +270,8 @@ public void testCoordinatorRestartOneRunningCompaction() throws Exception { List runningCompactions = new ArrayList<>(); ExternalCompactionId eci = ExternalCompactionId.generate(UUID.randomUUID()); - TExternalCompactionJob job = EasyMock.createNiceMock(TExternalCompactionJob.class); - expect(job.getExternalCompactionId()).andReturn(eci.toString()).anyTimes(); + PExternalCompactionJob job = + PExternalCompactionJob.newBuilder().setExternalCompactionId(eci.canonical()).build(); TKeyExtent extent = new TKeyExtent(); extent.setTable("1".getBytes()); runningCompactions.add(new RunningCompaction(job, tserverAddr.toString(), GROUP_ID.toString())); @@ -291,7 +282,7 @@ public void testCoordinatorRestartOneRunningCompaction() throws Exception { expect(manager.getSteadyTime()).andReturn(SteadyTime.from(100000, TimeUnit.NANOSECONDS)) .anyTimes(); - EasyMock.replay(context, job, security, manager); + EasyMock.replay(context, security, manager); var coordinator = new TestCoordinator(context, security, runningCompactions, manager); coordinator.resetInternals(); @@ -309,7 +300,7 @@ public void testCoordinatorRestartOneRunningCompaction() throws Exception { assertEquals(GROUP_ID.toString(), rc.getGroupName()); assertEquals(tserverAddr.toString(), rc.getCompactorAddress()); - EasyMock.verify(context, job, security); + EasyMock.verify(context, security); } @Test @@ -328,10 +319,12 @@ public void testGetCompactionJob() throws Exception { expect(context.getMetricsInfo()).andReturn(metricsInfo).anyTimes(); TCredentials creds = EasyMock.createNiceMock(TCredentials.class); + expect(context.rpcCreds()).andReturn(creds).anyTimes(); AuditedSecurityOperation security = EasyMock.createNiceMock(AuditedSecurityOperation.class); - expect(security.canPerformSystemActions(creds)).andReturn(true).anyTimes(); + expect(security.canPerformSystemActions(anyObject(TCredentials.class))).andReturn(true) + .anyTimes(); KeyExtent ke = new KeyExtent(TableId.of("2a"), new Text("z"), new Text("b")); TabletMetadata tm = EasyMock.createNiceMock(TabletMetadata.class); @@ -364,12 +357,13 @@ public void testGetCompactionJob() throws Exception { // Get the next job ExternalCompactionId eci = ExternalCompactionId.generate(UUID.randomUUID()); - TNextCompactionJob nextJob = coordinator.getCompactionJob(new TInfo(), creds, - GROUP_ID.toString(), "localhost:10241", eci.toString()); - assertEquals(3, nextJob.getCompactorCount()); - TExternalCompactionJob createdJob = nextJob.getJob(); + CompletableFuture nextJob = coordinator.getAsyncCompactionJob( + ProtoTInfo.newBuilder().build(), PCredentials.newBuilder().build(), GROUP_ID.toString(), + "localhost:10241", eci.toString()); + assertEquals(3, nextJob.get().getCompactorCount()); + PExternalCompactionJob createdJob = nextJob.get().getJob(); assertEquals(eci.toString(), createdJob.getExternalCompactionId()); - assertEquals(ke, KeyExtent.fromThrift(createdJob.getExtent())); + assertEquals(ke, KeyExtent.fromProtobuf(createdJob.getExtent())); assertEquals(0, coordinator.getJobQueues().getQueuedJobCount()); assertEquals(1, coordinator.getRunning().size()); @@ -392,7 +386,7 @@ public void testGetCompactionJobNoJobs() throws Exception { TCredentials creds = EasyMock.createNiceMock(TCredentials.class); AuditedSecurityOperation security = EasyMock.createNiceMock(AuditedSecurityOperation.class); - expect(security.canPerformSystemActions(creds)).andReturn(true); + expect(security.canPerformSystemActions(anyObject(TCredentials.class))).andReturn(true); Manager manager = EasyMock.createNiceMock(Manager.class); expect(manager.getSteadyTime()).andReturn(SteadyTime.from(100000, TimeUnit.NANOSECONDS)) @@ -401,10 +395,12 @@ public void testGetCompactionJobNoJobs() throws Exception { EasyMock.replay(context, creds, security, manager); var coordinator = new TestCoordinator(context, security, new ArrayList<>(), manager); - TNextCompactionJob nextJob = coordinator.getCompactionJob(TraceUtil.traceInfo(), creds, - GROUP_ID.toString(), "localhost:10240", UUID.randomUUID().toString()); - assertEquals(3, nextJob.getCompactorCount()); - assertNull(nextJob.getJob().getExternalCompactionId()); + CompletableFuture nextJob = coordinator.getAsyncCompactionJob( + TraceUtil.protoTraceInfo(), PCredentials.newBuilder().build(), GROUP_ID.toString(), + "localhost:10240", UUID.randomUUID().toString()); + Thread.sleep(100); + // No jobs + assertFalse(nextJob.isDone()); EasyMock.verify(context, creds, security); } @@ -432,9 +428,12 @@ public void testCleanUpRunning() throws Exception { var ecid2 = ExternalCompactionId.generate(UUID.randomUUID()); var ecid3 = ExternalCompactionId.generate(UUID.randomUUID()); - coordinator.getRunning().put(ecid1, new RunningCompaction(new TExternalCompaction())); - coordinator.getRunning().put(ecid2, new RunningCompaction(new TExternalCompaction())); - coordinator.getRunning().put(ecid3, new RunningCompaction(new TExternalCompaction())); + coordinator.getRunning().put(ecid1, + new RunningCompaction(PExternalCompaction.newBuilder().build())); + coordinator.getRunning().put(ecid2, + new RunningCompaction(PExternalCompaction.newBuilder().build())); + coordinator.getRunning().put(ecid3, + new RunningCompaction(PExternalCompaction.newBuilder().build())); coordinator.cleanUpInternalState(); diff --git a/server/monitor/pom.xml b/server/monitor/pom.xml index 46e3a1df356..078578e0b02 100644 --- a/server/monitor/pom.xml +++ b/server/monitor/pom.xml @@ -48,6 +48,10 @@ com.google.guava guava + + io.grpc + grpc-api + jakarta.inject jakarta.inject-api @@ -72,6 +76,10 @@ org.apache.accumulo accumulo-core + + org.apache.accumulo + accumulo-grpc + org.apache.accumulo accumulo-server-base diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java index 39ac86c96c1..b8f48fc94d3 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java @@ -47,9 +47,6 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.cli.ConfigOpts; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; -import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; -import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; @@ -68,6 +65,7 @@ import org.apache.accumulo.core.metrics.MetricsInfo; import org.apache.accumulo.core.rpc.ThriftUtil; import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; +import org.apache.accumulo.core.rpc.grpc.GrpcUtil; import org.apache.accumulo.core.tabletscan.thrift.ActiveScan; import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService; import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction; @@ -77,6 +75,11 @@ import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; import org.apache.accumulo.core.util.threads.Threads; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceGrpc; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceGrpc.CompactionCoordinatorServiceBlockingStub; +import org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList; import org.apache.accumulo.monitor.rest.compactions.external.ExternalCompactionInfo; import org.apache.accumulo.monitor.util.logging.RecentLogs; import org.apache.accumulo.server.AbstractServer; @@ -175,7 +178,7 @@ public boolean add(Pair obj) { private GCStatus gcStatus; private Optional coordinatorHost = Optional.empty(); private long coordinatorCheckNanos = 0L; - private CompactionCoordinatorService.Client coordinatorClient; + private CompactionCoordinatorServiceBlockingStub coordinatorClient; private final String coordinatorMissingMsg = "Error getting the compaction coordinator. Check that it is running. It is not " + "started automatically with other cluster processes so must be started by running " @@ -393,7 +396,9 @@ public void fetchData() { } finally { if (coordinatorClient != null) { - ThriftUtil.returnClient(coordinatorClient, context); + // TODO: we may need to return the client here depending on how pooling works + // with gRPC + // ThriftUtil.returnClient(coordinatorClient, context); coordinatorClient = null; } lastRecalc.set(currentTime); @@ -610,7 +615,7 @@ public static class CompactionStats { private final Map allCompactions = new HashMap<>(); private final RecentLogs recentLogs = new RecentLogs(); private final ExternalCompactionInfo ecInfo = new ExternalCompactionInfo(); - private final Map ecRunningMap = new ConcurrentHashMap<>(); + private final Map ecRunningMap = new ConcurrentHashMap<>(); private long scansFetchedNanos = 0L; private long compactsFetchedNanos = 0L; private long ecInfoFetchedNanos = 0L; @@ -670,37 +675,41 @@ public synchronized ExternalCompactionInfo getCompactorsInfo() { * user fetches since RPC calls are going to the coordinator. This allows for fine grain updates * of external compaction progress. */ - public synchronized Map fetchRunningInfo() { + public synchronized Map fetchRunningInfo() { if (coordinatorHost.isEmpty()) { throw new IllegalStateException(coordinatorMissingMsg); } var ccHost = coordinatorHost.orElseThrow(); log.info("User initiated fetch of running External Compactions from " + ccHost); var client = getCoordinator(ccHost); - TExternalCompactionList running; + PExternalCompactionList running; try { - running = client.getRunningCompactions(TraceUtil.traceInfo(), getContext().rpcCreds()); + running = client.getRunningCompactions(GetRunningCompactionsRequest.newBuilder() + .setPtinfo(TraceUtil.protoTraceInfo()).setCredentials(getContext().gRpcCreds()).build()); } catch (Exception e) { throw new IllegalStateException("Unable to get running compactions from " + ccHost, e); } ecRunningMap.clear(); - if (running.getCompactions() != null) { - ecRunningMap.putAll(running.getCompactions()); + if (!running.getCompactionsMap().isEmpty()) { + ecRunningMap.putAll(running.getCompactionsMap()); } return ecRunningMap; } - public Map getEcRunningMap() { + public Map getEcRunningMap() { return ecRunningMap; } - private CompactionCoordinatorService.Client getCoordinator(HostAndPort address) { + private CompactionCoordinatorServiceBlockingStub getCoordinator(HostAndPort address) { if (coordinatorClient == null) { try { - coordinatorClient = - ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, address, getContext()); + // TODO: coordinatorHost contains the Thrift port so right now only host is used. + // we eventually need the gRPC port and will need to store than in Zk. + // GrpcUtil for now just uses the property in the context for the port + coordinatorClient = CompactionCoordinatorServiceGrpc + .newBlockingStub(GrpcUtil.getChannel(coordinatorHost.orElseThrow(), getContext())); } catch (Exception e) { log.error("Unable to get Compaction coordinator at {}", address); throw new IllegalStateException(coordinatorMissingMsg, e); diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/RunningCompactions.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/RunningCompactions.java index 513c60328b5..b3965d856bb 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/RunningCompactions.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/RunningCompactions.java @@ -22,14 +22,14 @@ import java.util.List; import java.util.Map; -import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; import org.apache.accumulo.core.util.compaction.RunningCompactionInfo; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction; public class RunningCompactions { public final List running = new ArrayList<>(); - public RunningCompactions(Map rMap) { + public RunningCompactions(Map rMap) { if (rMap != null) { for (var entry : rMap.entrySet()) { running.add(new RunningCompactionInfo(entry.getValue())); diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/RunningCompactorDetails.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/RunningCompactorDetails.java index 415061ece9e..a824eaf405d 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/RunningCompactorDetails.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/RunningCompactorDetails.java @@ -21,26 +21,26 @@ import java.util.ArrayList; import java.util.List; -import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; -import org.apache.accumulo.core.tabletserver.thrift.InputFile; import org.apache.accumulo.core.util.compaction.RunningCompactionInfo; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction; +import org.apache.accumulo.grpc.compaction.protobuf.PInputFile; public class RunningCompactorDetails extends RunningCompactionInfo { // Variable names become JSON keys public final List inputFiles; public final String outputFile; - public RunningCompactorDetails(TExternalCompaction ec) { + public RunningCompactorDetails(PExternalCompaction ec) { super(ec); var job = ec.getJob(); - inputFiles = convertInputFiles(job.files); - outputFile = job.outputFile; + inputFiles = convertInputFiles(job.getFilesList()); + outputFile = job.getOutputFile(); } - private List convertInputFiles(List files) { + private List convertInputFiles(List files) { List list = new ArrayList<>(); - files.forEach(f -> list - .add(new CompactionInputFile(f.metadataFileEntry, f.size, f.entries, f.timestamp))); + files.forEach(f -> list.add(new CompactionInputFile(f.getMetadataFileEntry(), f.getSize(), + f.getEntries(), f.getTimestamp()))); // sorted largest to smallest list.sort((o1, o2) -> Long.compare(o2.size, o1.size)); return list; diff --git a/test/pom.xml b/test/pom.xml index a02d7fadb99..594389cf11f 100644 --- a/test/pom.xml +++ b/test/pom.xml @@ -62,6 +62,10 @@ commons-io commons-io + + io.grpc + grpc-api + io.micrometer micrometer-core @@ -90,6 +94,10 @@ org.apache.accumulo accumulo-gc + + org.apache.accumulo + accumulo-grpc + org.apache.accumulo accumulo-manager diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction2BaseIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction2BaseIT.java index bdcd9af8170..f3830eb5680 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction2BaseIT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction2BaseIT.java @@ -49,7 +49,6 @@ import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.clientImpl.TableOperationsImpl; -import org.apache.accumulo.core.compaction.thrift.TCompactionState; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.metadata.AccumuloTable; @@ -57,6 +56,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema; import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; import org.apache.accumulo.core.metadata.schema.TabletsMetadata; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionState; import org.apache.accumulo.harness.MiniClusterConfigurationCallback; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.minicluster.ServerType; @@ -120,7 +120,7 @@ public void testSplitCancelsExternalCompaction() throws Exception { client.tableOperations().addSplits(table1, splits); confirmCompactionCompleted(getCluster().getServerContext(), ecids, - TCompactionState.CANCELLED); + PCompactionState.CANCELLED); // ensure compaction ids were deleted by split operation from metadata table try (TabletsMetadata tm = getCluster().getServerContext().getAmple().readTablets() @@ -137,7 +137,7 @@ public void testSplitCancelsExternalCompaction() throws Exception { // Verify that the tmp file are cleaned up Wait.waitFor(() -> FindCompactionTmpFiles - .findTempFiles(getCluster().getServerContext(), tid.canonical()).size() == 0, 60_000); + .findTempFiles(getCluster().getServerContext(), tid.canonical()).isEmpty(), 60_000); } } @@ -190,7 +190,7 @@ public void testUserCompactionCancellation() throws Exception { assertEquals(TableOperationsImpl.COMPACTION_CANCELED_MSG, e.getMessage()); confirmCompactionCompleted(getCluster().getServerContext(), ecids, - TCompactionState.CANCELLED); + PCompactionState.CANCELLED); // ensure the canceled compaction deletes any tablet metadata related to the compaction while (countTablets(getCluster().getServerContext(), table1, @@ -234,7 +234,7 @@ public void testDeleteTableCancelsUserExternalCompaction() throws Exception { client.tableOperations().delete(table1); confirmCompactionCompleted(getCluster().getServerContext(), ecids, - TCompactionState.CANCELLED); + PCompactionState.CANCELLED); // Ensure compaction did not write anything to metadata table after delete table try (var scanner = client.createScanner(AccumuloTable.METADATA.tableName())) { @@ -290,7 +290,7 @@ public void testDeleteTableCancelsExternalCompaction() throws Exception { LoggerFactory.getLogger(getClass()).debug("Table deleted."); - confirmCompactionCompleted(ctx, ecids, TCompactionState.CANCELLED); + confirmCompactionCompleted(ctx, ecids, PCompactionState.CANCELLED); LoggerFactory.getLogger(getClass()).debug("Confirmed compaction cancelled."); diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java index 515c67ec575..06f88e228bb 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java @@ -45,8 +45,6 @@ import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.compaction.thrift.TCompactionState; -import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; -import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.iterators.IteratorUtil; @@ -56,6 +54,8 @@ import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; import org.apache.accumulo.core.util.compaction.RunningCompactionInfo; import org.apache.accumulo.core.util.threads.Threads; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList; import org.apache.accumulo.harness.AccumuloClusterHarness; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.accumulo.server.ServerContext; @@ -328,8 +328,8 @@ private void checkRunning() throws TException { throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); } - TExternalCompactionList ecList = getRunningCompactions(ctx, coordinatorHost); - Map ecMap = ecList.getCompactions(); + PExternalCompactionList ecList = getRunningCompactions(ctx, coordinatorHost); + Map ecMap = ecList.getCompactionsMap(); if (ecMap != null) { ecMap.forEach((ecid, ec) -> { // returns null if it's a new mapping diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java index 256751b0367..573e4c958d1 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java @@ -46,10 +46,6 @@ import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.client.admin.NewTableConfiguration; import org.apache.accumulo.core.clientImpl.ClientContext; -import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService; -import org.apache.accumulo.core.compaction.thrift.TCompactionState; -import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; -import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; import org.apache.accumulo.core.conf.ClientProperty; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; @@ -60,13 +56,19 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; import org.apache.accumulo.core.metadata.schema.TabletsMetadata; -import org.apache.accumulo.core.rpc.ThriftUtil; -import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; +import org.apache.accumulo.core.rpc.grpc.GrpcUtil; import org.apache.accumulo.core.spi.compaction.RatioBasedCompactionPlanner; import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceGrpc; +import org.apache.accumulo.grpc.compaction.protobuf.CompactionCoordinatorServiceGrpc.CompactionCoordinatorServiceBlockingStub; +import org.apache.accumulo.grpc.compaction.protobuf.GetCompletedCompactionsRequest; +import org.apache.accumulo.grpc.compaction.protobuf.GetRunningCompactionsRequest; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionState; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.compaction.ExternalCompaction_1_IT.TestFilter; @@ -74,7 +76,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.Text; -import org.apache.thrift.TException; import org.apache.thrift.transport.TTransportException; import com.beust.jcommander.internal.Maps; @@ -241,37 +242,47 @@ public static void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuratio coreSite.set("fs.file.impl", RawLocalFileSystem.class.getName()); } - public static TExternalCompactionList getRunningCompactions(ClientContext context, - Optional coordinatorHost) throws TException { - CompactionCoordinatorService.Client client = - ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.orElseThrow(), context); + public static PExternalCompactionList getRunningCompactions(ClientContext context, + Optional coordinatorHost) { + // TODO: coordinatorHost contains the Thrift port so right now only host is used. + // we eventually need the gRPC port and will need to store than in Zk. + // GrpcUtil for now just uses the property in the context for the port + CompactionCoordinatorServiceBlockingStub client = CompactionCoordinatorServiceGrpc + .newBlockingStub(GrpcUtil.getChannel(coordinatorHost.orElseThrow(), context)); try { - TExternalCompactionList running = - client.getRunningCompactions(TraceUtil.traceInfo(), context.rpcCreds()); + PExternalCompactionList running = + client.getRunningCompactions(GetRunningCompactionsRequest.newBuilder() + .setPtinfo(TraceUtil.protoTraceInfo()).setCredentials(context.gRpcCreds()).build()); return running; } finally { - ThriftUtil.returnClient(client, context); + // TODO return gRpc client if needed + // ThriftUtil.returnClient(client, context); } } - private static TExternalCompactionList getCompletedCompactions(ClientContext context, - Optional coordinatorHost) throws Exception { - CompactionCoordinatorService.Client client = - ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, coordinatorHost.orElseThrow(), context); + private static PExternalCompactionList getCompletedCompactions(ClientContext context, + Optional coordinatorHost) { + // TODO: coordinatorHost contains the Thrift port so right now only host is used. + // we eventually need the gRPC port and will need to store than in Zk. + // GrpcUtil for now just uses the property in the context for the port + CompactionCoordinatorServiceBlockingStub client = CompactionCoordinatorServiceGrpc + .newBlockingStub(GrpcUtil.getChannel(coordinatorHost.orElseThrow(), context)); try { - TExternalCompactionList completed = - client.getCompletedCompactions(TraceUtil.traceInfo(), context.rpcCreds()); + PExternalCompactionList completed = + client.getCompletedCompactions(GetCompletedCompactionsRequest.newBuilder() + .setPtinfo(TraceUtil.protoTraceInfo()).setCredentials(context.gRpcCreds()).build()); return completed; } finally { - ThriftUtil.returnClient(client, context); + // TODO return gRpc client if needed + // ThriftUtil.returnClient(client, context); } } - public static TCompactionState getLastState(TExternalCompaction status) { - ArrayList timestamps = new ArrayList<>(status.getUpdates().size()); - status.getUpdates().keySet().forEach(k -> timestamps.add(k)); + public static PCompactionState getLastState(PExternalCompaction status) { + ArrayList timestamps = new ArrayList<>(status.getUpdatesMap().size()); + status.getUpdatesMap().keySet().forEach(k -> timestamps.add(k)); Collections.sort(timestamps); - return status.getUpdates().get(timestamps.get(timestamps.size() - 1)).getState(); + return status.getUpdatesMap().get(timestamps.get(timestamps.size() - 1)).getState(); } public static Set waitForCompactionStartAndReturnEcids(ServerContext ctx, @@ -320,14 +331,14 @@ public static int confirmCompactionRunning(ServerContext ctx, Set ecids, - TCompactionState expectedState) throws Exception { + PCompactionState expectedState) throws Exception { Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); if (coordinatorHost.isEmpty()) { throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper"); } // The running compaction should be removed - TExternalCompactionList running = + PExternalCompactionList running = ExternalCompactionTestUtils.getRunningCompactions(ctx, coordinatorHost); - while (running.getCompactions() != null && running.getCompactions().keySet().stream() + while (running.getCompactionsMap().keySet().stream() .anyMatch((e) -> ecids.contains(ExternalCompactionId.of(e)))) { running = ExternalCompactionTestUtils.getRunningCompactions(ctx, coordinatorHost); } // The compaction should be in the completed list with the expected state - TExternalCompactionList completed = + PExternalCompactionList completed = ExternalCompactionTestUtils.getCompletedCompactions(ctx, coordinatorHost); - while (completed.getCompactions() == null) { + while (completed.getCompactionsMap().isEmpty()) { UtilWaitThread.sleep(50); completed = ExternalCompactionTestUtils.getCompletedCompactions(ctx, coordinatorHost); } for (ExternalCompactionId e : ecids) { - TExternalCompaction tec = completed.getCompactions().get(e.canonical()); + PExternalCompaction tec = completed.getCompactionsMap().get(e.canonical()); assertNotNull(tec); assertEquals(expectedState, ExternalCompactionTestUtils.getLastState(tec)); } diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java index 6d80c970e38..6cd0c4bf735 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java @@ -28,6 +28,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.EnumSet; @@ -43,9 +44,6 @@ import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.compaction.thrift.TCompactionState; -import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; -import org.apache.accumulo.core.compaction.thrift.TExternalCompaction; -import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; @@ -56,6 +54,10 @@ import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil; import org.apache.accumulo.core.util.compaction.RunningCompactionInfo; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionState; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompaction; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionList; import org.apache.accumulo.harness.MiniClusterConfigurationCallback; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.manager.compaction.coordinator.CompactionCoordinator; @@ -67,8 +69,6 @@ import org.apache.accumulo.test.util.Wait; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; -import org.apache.thrift.TException; -import org.apache.thrift.transport.TTransportException; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -132,7 +132,7 @@ public void testMergeCancelsExternalCompaction() throws Exception { client.tableOperations().merge(table1, start, end); confirmCompactionCompleted(getCluster().getServerContext(), ecids, - TCompactionState.CANCELLED); + PCompactionState.CANCELLED); // ensure compaction ids were deleted by merge operation from metadata table try (TabletsMetadata tm = getCluster().getServerContext().getAmple().readTablets() @@ -226,28 +226,27 @@ private Map getRunningCompactionInfo final Map results = new HashMap<>(); while (results.isEmpty()) { - TExternalCompactionList running = null; - while (running == null || running.getCompactions() == null) { + PExternalCompactionList running = null; + while (running == null || running.getCompactionsMap().isEmpty()) { try { Optional coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(ctx); if (coordinatorHost.isEmpty()) { - throw new TTransportException( - "Unable to get CompactionCoordinator address from ZooKeeper"); + throw new IOException("Unable to get CompactionCoordinator address from ZooKeeper"); } running = getRunningCompactions(ctx, coordinatorHost); - } catch (TException t) { + } catch (IOException t) { running = null; Thread.sleep(2000); } } for (ExternalCompactionId ecid : ecids) { - final TExternalCompaction tec = running.getCompactions().get(ecid.canonical()); - if (tec != null && tec.getUpdatesSize() > 0) { + final PExternalCompaction tec = running.getCompactionsMap().get(ecid.canonical()); + if (tec != null && tec.getUpdatesCount() > 0) { // When the coordinator restarts it inserts a message into the updates. If this // is the last message, then don't insert this into the results. We want to get // an actual update from the Compactor. - TreeMap sorted = new TreeMap<>(tec.getUpdates()); + TreeMap sorted = new TreeMap<>(tec.getUpdatesMap()); var lastEntry = sorted.lastEntry(); if (lastEntry.getValue().getMessage().equals(CompactionCoordinator.RESTART_UPDATE_MSG)) { continue; diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java index afd4b271e25..50ec26d2f54 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java @@ -29,18 +29,18 @@ import org.apache.accumulo.compactor.Compactor; import org.apache.accumulo.core.cli.ConfigOpts; import org.apache.accumulo.core.compaction.thrift.CompactorService.Iface; -import org.apache.accumulo.core.compaction.thrift.TCompactionState; -import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.metadata.ReferencedTabletFile; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; -import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; import org.apache.accumulo.core.util.UtilWaitThread; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionState; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionStatusUpdate; +import org.apache.accumulo.grpc.compaction.protobuf.PExternalCompactionJob; import org.apache.accumulo.server.compaction.FileCompactor; import org.apache.accumulo.server.compaction.FileCompactor.CompactionCanceledException; -import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException; +import org.apache.accumulo.server.compaction.RetryableRpcCall.RetriesExceededException; import org.apache.accumulo.server.tablets.TabletNameGenerator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,7 +61,7 @@ protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor, } @Override - protected FileCompactorRunnable createCompactionJob(TExternalCompactionJob job, + protected FileCompactorRunnable createCompactionJob(PExternalCompactionJob job, LongAdder totalInputEntries, LongAdder totalInputBytes, CountDownLatch started, CountDownLatch stopped, AtomicReference err) { @@ -85,17 +85,16 @@ public Duration getCompactionAge() { public void run() { try { LOG.info("Starting up compaction runnable for job: {}", job); - TCompactionStatusUpdate update = new TCompactionStatusUpdate(); - update.setState(TCompactionState.STARTED); - update.setMessage("Compaction started"); + PCompactionStatusUpdate update = PCompactionStatusUpdate.newBuilder() + .setState(PCompactionState.STARTED).setMessage("Compaction started").build(); updateCompactionState(job, update); // Create tmp output file final TabletMetadata tm = getContext().getAmple() - .readTablet(KeyExtent.fromThrift(job.getExtent()), ColumnType.DIR); - ReferencedTabletFile newFile = - TabletNameGenerator.getNextDataFilenameForMajc(job.isPropagateDeletes(), getContext(), - tm, (dir) -> {}, ExternalCompactionId.from(job.getExternalCompactionId())); + .readTablet(KeyExtent.fromProtobuf(job.getExtent()), ColumnType.DIR); + ReferencedTabletFile newFile = TabletNameGenerator.getNextDataFilenameForMajc( + job.getPropagateDeletes(), getContext(), tm, (dir) -> {}, + ExternalCompactionId.from(job.getExternalCompactionId())); LOG.info("Creating tmp file: {}", newFile.getPath()); getContext().getVolumeManager().createNewFile(newFile.getPath()); @@ -110,9 +109,9 @@ public void run() { throw new CompactionCanceledException(); } catch (Exception e) { - KeyExtent fromThriftExtent = KeyExtent.fromThrift(job.getExtent()); + KeyExtent fromProtobufExtent = KeyExtent.fromProtobuf(job.getExtent()); LOG.error("Compaction failed: id: {}, extent: {}", job.getExternalCompactionId(), - fromThriftExtent, e); + fromProtobufExtent, e); err.set(e); } finally { stopped.countDown(); @@ -122,7 +121,7 @@ public void run() { @Override public void initialize() throws RetriesExceededException { // This isn't used, just need to create and return something - ref.set(new FileCompactor(getContext(), KeyExtent.fromThrift(job.getExtent()), null, null, + ref.set(new FileCompactor(getContext(), KeyExtent.fromProtobuf(job.getExtent()), null, null, false, null, null, null, null, null)); } diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/SplitCancelsMajCIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/SplitCancelsMajCIT.java index e7ee6a19bbe..4fdff9235e1 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/SplitCancelsMajCIT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/SplitCancelsMajCIT.java @@ -34,13 +34,13 @@ import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.BatchWriter; import org.apache.accumulo.core.client.IteratorSetting; -import org.apache.accumulo.core.compaction.thrift.TCompactionState; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.metadata.schema.ExternalCompactionId; +import org.apache.accumulo.grpc.compaction.protobuf.PCompactionState; import org.apache.accumulo.harness.MiniClusterConfigurationCallback; import org.apache.accumulo.harness.SharedMiniClusterBase; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; @@ -120,7 +120,7 @@ public void test() throws Exception { c.tableOperations().addSplits(tableName, partitionKeys); ExternalCompactionTestUtils.confirmCompactionCompleted(getCluster().getServerContext(), - compactionIds, TCompactionState.CANCELLED); + compactionIds, PCompactionState.CANCELLED); thread.join(); // wait for the restarted compaction diff --git a/test/src/main/java/org/apache/accumulo/test/util/SlowOps.java b/test/src/main/java/org/apache/accumulo/test/util/SlowOps.java index fb48e21fe32..7e0e969aee1 100644 --- a/test/src/main/java/org/apache/accumulo/test/util/SlowOps.java +++ b/test/src/main/java/org/apache/accumulo/test/util/SlowOps.java @@ -226,7 +226,7 @@ private boolean blockUntilCompactionRunning() { do { boolean tableFound = ExternalCompactionUtil.getCompactionsRunningOnCompactors((ClientContext) client).stream() - .map(rc -> KeyExtent.fromThrift(rc.getJob().getExtent()).tableId()) + .map(rc -> KeyExtent.fromProtobuf(rc.getJob().getExtent()).tableId()) .anyMatch(tableId::equals); if (tableFound) { diff --git a/test/src/main/resources/log4j2-test.properties b/test/src/main/resources/log4j2-test.properties index ac62675d852..2c5e6c6bc8c 100644 --- a/test/src/main/resources/log4j2-test.properties +++ b/test/src/main/resources/log4j2-test.properties @@ -148,5 +148,8 @@ logger.40.level = trace logger.41.name = org.apache.accumulo.server.metadata logger.41.level = trace +logger.42.name = io.grpc.netty +logger.42.level = info + rootLogger.level = debug rootLogger.appenderRef.console.ref = STDOUT