diff --git a/BUILDING.txt b/BUILDING.txt index 6f33a60f41103..03dffdd80c74c 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -6,7 +6,6 @@ Requirements: * Unix System * JDK 1.8 * Maven 3.3 or later -* ProtocolBuffer 3.7.1 * CMake 3.1 or newer (if compiling native code) * Zlib devel (if compiling native code) * Cyrus SASL devel (if compiling native code) @@ -62,16 +61,6 @@ Installing required packages for clean install of Ubuntu 14.04 LTS Desktop: $ sudo apt-get -y install maven * Native libraries $ sudo apt-get -y install build-essential autoconf automake libtool cmake zlib1g-dev pkg-config libssl-dev libsasl2-dev -* ProtocolBuffer 3.7.1 (required) - $ mkdir -p /opt/protobuf-3.7-src \ - && curl -L -s -S \ - https://github.com/protocolbuffers/protobuf/releases/download/v3.7.1/protobuf-java-3.7.1.tar.gz \ - -o /opt/protobuf-3.7.1.tar.gz \ - && tar xzf /opt/protobuf-3.7.1.tar.gz --strip-components 1 -C /opt/protobuf-3.7-src \ - && cd /opt/protobuf-3.7-src \ - && ./configure\ - && make install \ - && rm -rf /opt/protobuf-3.7-src Optional packages: @@ -310,16 +299,6 @@ level once; and then work from the submodule. Keep in mind that SNAPSHOTs time out after a while, using the Maven '-nsu' will stop Maven from trying to update SNAPSHOTs from external repos. ----------------------------------------------------------------------------------- -Protocol Buffer compiler - -The version of Protocol Buffer compiler, protoc, must match the version of the -protobuf JAR. - -If you have multiple versions of protoc in your system, you can set in your -build shell the HADOOP_PROTOC_PATH environment variable to point to the one you -want to use for the Hadoop build. If you don't define this environment variable, -protoc is looked up in the PATH. ---------------------------------------------------------------------------------- Importing projects to eclipse @@ -405,15 +384,6 @@ Installing required dependencies for clean install of macOS 10.14: * Install native libraries, only openssl is required to compile native code, you may optionally install zlib, lz4, etc. $ brew install openssl -* Protocol Buffers 3.7.1 (required) - $ wget https://github.com/protocolbuffers/protobuf/releases/download/v3.7.1/protobuf-java-3.7.1.tar.gz - $ mkdir -p protobuf-3.7 && tar zxvf protobuf-java-3.7.1.tar.gz --strip-components 1 -C protobuf-3.7 - $ cd protobuf-3.7 - $ ./configure - $ make - $ make check - $ make install - $ protoc --version Note that building Hadoop 3.1.1/3.1.2/3.2.0 native code from source is broken on macOS. For 3.1.1/3.1.2, you need to manually backport YARN-8622. For 3.2.0, @@ -439,7 +409,6 @@ Requirements: * Windows System * JDK 1.8 * Maven 3.0 or later -* ProtocolBuffer 3.7.1 * CMake 3.1 or newer * Visual Studio 2010 Professional or Higher * Windows SDK 8.1 (if building CPU rate control for the container executor) diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile index 657c223a810fe..969d8bb44e376 100644 --- a/dev-support/docker/Dockerfile +++ b/dev-support/docker/Dockerfile @@ -105,23 +105,6 @@ RUN mkdir -p /opt/cmake \ ENV CMAKE_HOME /opt/cmake ENV PATH "${PATH}:/opt/cmake/bin" -###### -# Install Google Protobuf 3.7.1 (2.6.0 ships with Xenial) -###### -# hadolint ignore=DL3003 -RUN mkdir -p /opt/protobuf-src \ - && curl -L -s -S \ - https://github.com/protocolbuffers/protobuf/releases/download/v3.7.1/protobuf-java-3.7.1.tar.gz \ - -o /opt/protobuf.tar.gz \ - && tar xzf /opt/protobuf.tar.gz --strip-components 1 -C /opt/protobuf-src \ - && cd /opt/protobuf-src \ - && ./configure --prefix=/opt/protobuf \ - && make install \ - && cd /root \ - && rm -rf /opt/protobuf-src -ENV PROTOBUF_HOME /opt/protobuf -ENV PATH "${PATH}:/opt/protobuf/bin" - ###### # Install Apache Maven 3.3.9 (3.3.9 ships with Xenial) ###### @@ -207,7 +190,7 @@ ENV MAVEN_OPTS -Xms256m -Xmx1536m ### # Hugo static website generator (for new hadoop site and Ozone docs) -RUN curl -L -o hugo.deb https://github.com/gohugoio/hugo/releases/download/v0.30.2/hugo_0.30.2_Linux-64bit.deb \ +RUN curl -L -o hugo.deb https://github.com/gohugoio/hugo/releases/download/v0.58.3/hugo_0.58.3_Linux-64bit.deb \ && dpkg --install hugo.deb \ && rm hugo.deb diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java index d234a3f40854d..7ce7ee3dd7d7f 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage; import org.apache.hadoop.hdds.scm.client.HddsClientUtils; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.security.x509.SecurityConfig; @@ -56,7 +57,6 @@ import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.rpc.RpcType; import org.apache.ratis.rpc.SupportedRpcType; -import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; @@ -219,39 +219,16 @@ private CompletableFuture sendRequestAsync( try (Scope scope = GlobalTracer.get() .buildSpan("XceiverClientRatis." + request.getCmdType().name()) .startActive(true)) { - ContainerCommandRequestProto finalPayload = - ContainerCommandRequestProto.newBuilder(request) - .setTraceID(TracingUtil.exportCurrentSpan()) - .build(); - boolean isReadOnlyRequest = HddsUtils.isReadOnly(finalPayload); - ByteString byteString = finalPayload.toByteString(); - if (LOG.isDebugEnabled()) { - LOG.debug("sendCommandAsync {} {}", isReadOnlyRequest, - sanitizeForDebug(finalPayload)); + final ContainerCommandRequestMessage message + = ContainerCommandRequestMessage.toMessage( + request, TracingUtil.exportCurrentSpan()); + if (HddsUtils.isReadOnly(request)) { + LOG.debug("sendCommandAsync ReadOnly {}", message); + return getClient().sendReadOnlyAsync(message); + } else { + LOG.debug("sendCommandAsync {}", message); + return getClient().sendAsync(message); } - return isReadOnlyRequest ? - getClient().sendReadOnlyAsync(() -> byteString) : - getClient().sendAsync(() -> byteString); - } - } - - private ContainerCommandRequestProto sanitizeForDebug( - ContainerCommandRequestProto request) { - switch (request.getCmdType()) { - case PutSmallFile: - return request.toBuilder() - .setPutSmallFile(request.getPutSmallFile().toBuilder() - .clearData() - ) - .build(); - case WriteChunk: - return request.toBuilder() - .setWriteChunk(request.getWriteChunk().toBuilder() - .clearData() - ) - .build(); - default: - return request; } } diff --git a/hadoop-hdds/common/dev-support/findbugsExcludeFile.xml b/hadoop-hdds/common/dev-support/findbugsExcludeFile.xml index c7db6794cc0e0..4441b69d8683e 100644 --- a/hadoop-hdds/common/dev-support/findbugsExcludeFile.xml +++ b/hadoop-hdds/common/dev-support/findbugsExcludeFile.xml @@ -25,4 +25,9 @@ + + + + + diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolClientSideTranslatorPB.java index d7d53a4b8cdb2..efe79a76f31dd 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolClientSideTranslatorPB.java @@ -16,22 +16,29 @@ */ package org.apache.hadoop.hdds.protocolPB; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; import java.io.Closeable; import java.io.IOException; +import java.util.function.Consumer; + +import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.OzoneManagerDetailsProto; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCACertificateRequestProto; import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto; import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertificateRequestProto; -import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertificateRequestProto.Builder; import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetDataNodeCertRequestProto; -import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMSecurityRequest; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMSecurityRequest.Builder; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMSecurityResponse; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.Type; +import org.apache.hadoop.hdds.tracing.TracingUtil; import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtocolTranslator; import org.apache.hadoop.ipc.RPC; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; import static org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetOMCertRequestProto; /** @@ -52,6 +59,28 @@ public SCMSecurityProtocolClientSideTranslatorPB( this.rpcProxy = rpcProxy; } + /** + * Helper method to wrap the request and send the message. + */ + private SCMSecurityResponse submitRequest( + SCMSecurityProtocolProtos.Type type, + Consumer builderConsumer) throws IOException { + final SCMSecurityResponse response; + try { + + Builder builder = SCMSecurityRequest.newBuilder() + .setCmdType(type) + .setTraceID(TracingUtil.exportCurrentSpan()); + builderConsumer.accept(builder); + SCMSecurityRequest wrapper = builder.build(); + + response = rpcProxy.submitRequest(NULL_RPC_CONTROLLER, wrapper); + } catch (ServiceException ex) { + throw ProtobufHelper.getRemoteException(ex); + } + return response; + } + /** * Closes this stream and releases any system resources associated * with it. If the stream is already closed then invoking this @@ -87,8 +116,8 @@ public String getDataNodeCertificate(DatanodeDetailsProto dataNodeDetails, /** * Get SCM signed certificate for OM. * - * @param omDetails - OzoneManager Details. - * @param certSignReq - Certificate signing request. + * @param omDetails - OzoneManager Details. + * @param certSignReq - Certificate signing request. * @return byte[] - SCM signed certificate. */ @Override @@ -100,64 +129,61 @@ public String getOMCertificate(OzoneManagerDetailsProto omDetails, /** * Get SCM signed certificate for OM. * - * @param omDetails - OzoneManager Details. - * @param certSignReq - Certificate signing request. + * @param omDetails - OzoneManager Details. + * @param certSignReq - Certificate signing request. * @return byte[] - SCM signed certificate. */ public SCMGetCertResponseProto getOMCertChain( OzoneManagerDetailsProto omDetails, String certSignReq) throws IOException { - SCMGetOMCertRequestProto.Builder builder = SCMGetOMCertRequestProto + SCMGetOMCertRequestProto request = SCMGetOMCertRequestProto .newBuilder() .setCSR(certSignReq) - .setOmDetails(omDetails); - try { - return rpcProxy.getOMCertificate(NULL_RPC_CONTROLLER, builder.build()); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + .setOmDetails(omDetails) + .build(); + return submitRequest(Type.GetOMCertificate, + builder -> builder.setGetOMCertRequest(request)) + .getGetCertResponseProto(); } /** * Get SCM signed certificate with given serial id. Throws exception if * certificate is not found. * - * @param certSerialId - Certificate serial id. + * @param certSerialId - Certificate serial id. * @return string - pem encoded certificate. */ @Override public String getCertificate(String certSerialId) throws IOException { - Builder builder = SCMGetCertificateRequestProto + SCMGetCertificateRequestProto request = SCMGetCertificateRequestProto .newBuilder() - .setCertSerialId(certSerialId); - try { - return rpcProxy.getCertificate(NULL_RPC_CONTROLLER, builder.build()) - .getX509Certificate(); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + .setCertSerialId(certSerialId) + .build(); + return submitRequest(Type.GetCertificate, + builder -> builder.setGetCertificateRequest(request)) + .getGetCertResponseProto() + .getX509Certificate(); } /** * Get SCM signed certificate for Datanode. * - * @param dnDetails - Datanode Details. - * @param certSignReq - Certificate signing request. + * @param dnDetails - Datanode Details. + * @param certSignReq - Certificate signing request. * @return byte[] - SCM signed certificate. */ public SCMGetCertResponseProto getDataNodeCertificateChain( DatanodeDetailsProto dnDetails, String certSignReq) throws IOException { - SCMGetDataNodeCertRequestProto.Builder builder = + + SCMGetDataNodeCertRequestProto request = SCMGetDataNodeCertRequestProto.newBuilder() .setCSR(certSignReq) - .setDatanodeDetails(dnDetails); - try { - return rpcProxy.getDataNodeCertificate(NULL_RPC_CONTROLLER, - builder.build()); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + .setDatanodeDetails(dnDetails) + .build(); + return submitRequest(Type.GetDataNodeCertificate, + builder -> builder.setGetDataNodeCertRequest(request)) + .getGetCertResponseProto(); } /** @@ -169,12 +195,10 @@ public SCMGetCertResponseProto getDataNodeCertificateChain( public String getCACertificate() throws IOException { SCMGetCACertificateRequestProto protoIns = SCMGetCACertificateRequestProto .getDefaultInstance(); - try { - return rpcProxy.getCACertificate(NULL_RPC_CONTROLLER, protoIns) - .getX509Certificate(); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + return submitRequest(Type.GetCACertificate, + builder -> builder.setGetCACertificateRequest(protoIns)) + .getGetCertResponseProto().getX509Certificate(); + } /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolServerSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolServerSideTranslatorPB.java deleted file mode 100644 index 2fd55945753b1..0000000000000 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocolPB/SCMSecurityProtocolServerSideTranslatorPB.java +++ /dev/null @@ -1,132 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.hdds.protocolPB; - -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; -import java.io.IOException; - -import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos; -import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto; -import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertificateRequestProto; -import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetDataNodeCertRequestProto; -import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto.ResponseCode; -import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol; -import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetOMCertRequestProto; - -/** - * This class is the server-side translator that forwards requests received on - * {@link SCMSecurityProtocolPB} to the {@link - * SCMSecurityProtocol} server implementation. - */ -public class SCMSecurityProtocolServerSideTranslatorPB implements - SCMSecurityProtocolPB { - - private final SCMSecurityProtocol impl; - - public SCMSecurityProtocolServerSideTranslatorPB(SCMSecurityProtocol impl) { - this.impl = impl; - } - - /** - * Get SCM signed certificate for DataNode. - * - * @param controller - * @param request - * @return SCMGetDataNodeCertResponseProto. - */ - @Override - public SCMGetCertResponseProto getDataNodeCertificate( - RpcController controller, SCMGetDataNodeCertRequestProto request) - throws ServiceException { - try { - String certificate = impl - .getDataNodeCertificate(request.getDatanodeDetails(), - request.getCSR()); - SCMGetCertResponseProto.Builder builder = - SCMGetCertResponseProto - .newBuilder() - .setResponseCode(ResponseCode.success) - .setX509Certificate(certificate) - .setX509CACertificate(impl.getCACertificate()); - - return builder.build(); - } catch (IOException e) { - throw new ServiceException(e); - } - } - - /** - * Get SCM signed certificate for OzoneManager. - * - * @param controller - * @param request - * @return SCMGetCertResponseProto. - */ - @Override - public SCMGetCertResponseProto getOMCertificate( - RpcController controller, SCMGetOMCertRequestProto request) - throws ServiceException { - try { - String certificate = impl - .getOMCertificate(request.getOmDetails(), - request.getCSR()); - SCMGetCertResponseProto.Builder builder = - SCMGetCertResponseProto - .newBuilder() - .setResponseCode(ResponseCode.success) - .setX509Certificate(certificate) - .setX509CACertificate(impl.getCACertificate()); - return builder.build(); - } catch (IOException e) { - throw new ServiceException(e); - } - } - - @Override - public SCMGetCertResponseProto getCertificate(RpcController controller, - SCMGetCertificateRequestProto request) throws ServiceException { - try { - String certificate = impl.getCertificate(request.getCertSerialId()); - SCMGetCertResponseProto.Builder builder = - SCMGetCertResponseProto - .newBuilder() - .setResponseCode(ResponseCode.success) - .setX509Certificate(certificate); - return builder.build(); - } catch (IOException e) { - throw new ServiceException(e); - } - } - - @Override - public SCMGetCertResponseProto getCACertificate(RpcController controller, - SCMSecurityProtocolProtos.SCMGetCACertificateRequestProto request) - throws ServiceException { - try { - String certificate = impl.getCACertificate(); - SCMGetCertResponseProto.Builder builder = - SCMGetCertResponseProto - .newBuilder() - .setResponseCode(ResponseCode.success) - .setX509Certificate(certificate); - return builder.build(); - } catch (IOException e) { - throw new ServiceException(e); - } - } -} \ No newline at end of file diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java new file mode 100644 index 0000000000000..07a886a0f9c0d --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdds.ratis; + +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; +import org.apache.ratis.protocol.Message; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.ratis.util.JavaUtils; + +import java.util.Objects; +import java.util.function.Supplier; + +/** + * Implementing the {@link Message} interface + * for {@link ContainerCommandRequestProto}. + */ +public final class ContainerCommandRequestMessage implements Message { + public static ContainerCommandRequestMessage toMessage( + ContainerCommandRequestProto request, String traceId) { + final ContainerCommandRequestProto.Builder b + = ContainerCommandRequestProto.newBuilder(request); + if (traceId != null) { + b.setTraceID(traceId); + } + + ByteString data = ByteString.EMPTY; + if (request.getCmdType() == Type.WriteChunk) { + final WriteChunkRequestProto w = request.getWriteChunk(); + data = w.getData(); + b.setWriteChunk(w.toBuilder().clearData()); + } else if (request.getCmdType() == Type.PutSmallFile) { + final PutSmallFileRequestProto p = request.getPutSmallFile(); + data = p.getData(); + b.setPutSmallFile(p.toBuilder().setData(ByteString.EMPTY)); + } + return new ContainerCommandRequestMessage(b.build(), data); + } + + public static ContainerCommandRequestProto toProto( + ByteString bytes, RaftGroupId groupId) + throws InvalidProtocolBufferException { + final int i = 4 + bytes.asReadOnlyByteBuffer().getInt(); + final ContainerCommandRequestProto header + = ContainerCommandRequestProto.parseFrom(bytes.substring(4, i)); + // TODO: setting pipeline id can be avoided if the client is sending it. + // In such case, just have to validate the pipeline id. + final ContainerCommandRequestProto.Builder b = header.toBuilder(); + if (groupId != null) { + b.setPipelineID(groupId.getUuid().toString()); + } + final ByteString data = bytes.substring(i); + if (header.getCmdType() == Type.WriteChunk) { + b.setWriteChunk(b.getWriteChunkBuilder().setData(data)); + } else if (header.getCmdType() == Type.PutSmallFile) { + b.setPutSmallFile(b.getPutSmallFileBuilder().setData(data)); + } + return b.build(); + } + + private final ContainerCommandRequestProto header; + private final ByteString data; + private final Supplier contentSupplier + = JavaUtils.memoize(this::buildContent); + + private ContainerCommandRequestMessage( + ContainerCommandRequestProto header, ByteString data) { + this.header = Objects.requireNonNull(header, "header == null"); + this.data = Objects.requireNonNull(data, "data == null"); + } + + private ByteString buildContent() { + final ByteString headerBytes = header.toByteString(); + return RatisHelper.int2ByteString(headerBytes.size()) + .concat(headerBytes) + .concat(data); + } + + @Override + public ByteString getContent() { + return contentSupplier.get(); + } + + @Override + public String toString() { + return header + ", data.size=" + data.size(); + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java index 3ad4e2e7a2b43..fc0effa9e671c 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.ratis; +import java.io.DataOutputStream; import java.io.IOException; import java.security.cert.CertificateException; import java.security.cert.X509Certificate; @@ -272,4 +273,15 @@ static Long getMinReplicatedIndex( return commitInfos.stream().map(RaftProtos.CommitInfoProto::getCommitIndex) .min(Long::compareTo).orElse(null); } + + static ByteString int2ByteString(int n) { + final ByteString.Output out = ByteString.newOutput(); + try(DataOutputStream dataOut = new DataOutputStream(out)) { + dataOut.writeInt(n); + } catch (IOException e) { + throw new IllegalStateException( + "Failed to write integer n = " + n + " to a ByteString.", e); + } + return out.toByteString(); + } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java index fe479ba43829c..5c58e92d3c5d9 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java @@ -54,7 +54,7 @@ public class ContainerInfo implements Comparator, mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); mapper .setVisibility(PropertyAccessor.GETTER, JsonAutoDetect.Visibility.NONE); - WRITER = mapper.writer(); + WRITER = mapper.writerWithDefaultPrettyPrinter(); } private HddsProtos.LifeCycleState state; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java index ab3fcd185ff9a..01db597dfae1a 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java @@ -16,64 +16,57 @@ */ package org.apache.hadoop.hdds.scm.protocolPB; -import com.google.common.base.Preconditions; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.GetScmInfoResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ActivatePipelineRequestProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DeactivatePipelineRequestProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ListPipelineRequestProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ListPipelineResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ClosePipelineRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DeactivatePipelineRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ForceExitSafeModeRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ForceExitSafeModeResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineRequestProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.InSafeModeRequestProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.InSafeModeResponseProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartReplicationManagerRequestProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StopReplicationManagerRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ListPipelineRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ListPipelineResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.NodeQueryRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.NodeQueryResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.PipelineRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.PipelineResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ReplicationManagerStatusRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ReplicationManagerStatusResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMDeleteContainerRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ScmContainerLocationRequest; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ScmContainerLocationRequest.Builder; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ScmContainerLocationResponse; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartReplicationManagerRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StopReplicationManagerRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type; import org.apache.hadoop.hdds.scm.ScmInfo; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.ContainerRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.ContainerResponseProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.GetContainerRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.GetContainerResponseProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.NodeQueryRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.NodeQueryResponseProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.PipelineRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.PipelineResponseProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.SCMDeleteContainerRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.SCMListContainerRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerLocationProtocolProtos.SCMListContainerResponseProto; import org.apache.hadoop.hdds.tracing.TracingUtil; import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtocolTranslator; import org.apache.hadoop.ipc.RPC; -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; +import com.google.common.base.Preconditions; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; /** * This class is the client-side translator to translate the requests made on @@ -101,14 +94,35 @@ public StorageContainerLocationProtocolClientSideTranslatorPB( this.rpcProxy = rpcProxy; } + /** + * Helper method to wrap the request and send the message. + */ + private ScmContainerLocationResponse submitRequest( + StorageContainerLocationProtocolProtos.Type type, + Consumer builderConsumer) throws IOException { + final ScmContainerLocationResponse response; + try { + + Builder builder = ScmContainerLocationRequest.newBuilder() + .setCmdType(type) + .setTraceID(TracingUtil.exportCurrentSpan()); + builderConsumer.accept(builder); + ScmContainerLocationRequest wrapper = builder.build(); + + response = rpcProxy.submitRequest(NULL_RPC_CONTROLLER, wrapper); + } catch (ServiceException ex) { + throw ProtobufHelper.getRemoteException(ex); + } + return response; + } + /** * Asks SCM where a container should be allocated. SCM responds with the set * of datanodes that should be used creating this container. Ozone/SCM only * supports replication factor of either 1 or 3. - * @param type - Replication Type + * + * @param type - Replication Type * @param factor - Replication Count - * @return - * @throws IOException */ @Override public ContainerWithPipeline allocateContainer( @@ -122,12 +136,11 @@ public ContainerWithPipeline allocateContainer( .setOwner(owner) .build(); - final ContainerResponseProto response; - try { - response = rpcProxy.allocateContainer(NULL_RPC_CONTROLLER, request); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + ContainerResponseProto response = + submitRequest(Type.AllocateContainer, + builder -> builder.setContainerRequest(request)) + .getContainerResponse(); + //TODO should be migrated to use the top level status structure. if (response.getErrorCode() != ContainerResponseProto.Error.success) { throw new IOException(response.hasErrorMessage() ? response.getErrorMessage() : "Allocate container failed."); @@ -144,13 +157,12 @@ public ContainerInfo getContainer(long containerID) throws IOException { .setContainerID(containerID) .setTraceID(TracingUtil.exportCurrentSpan()) .build(); - try { - GetContainerResponseProto response = - rpcProxy.getContainer(NULL_RPC_CONTROLLER, request); - return ContainerInfo.fromProtobuf(response.getContainerInfo()); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + ScmContainerLocationResponse response = + submitRequest(Type.GetContainer, + (builder) -> builder.setGetContainerRequest(request)); + return ContainerInfo + .fromProtobuf(response.getGetContainerResponse().getContainerInfo()); + } /** @@ -164,14 +176,15 @@ public ContainerWithPipeline getContainerWithPipeline(long containerID) GetContainerWithPipelineRequestProto.newBuilder() .setTraceID(TracingUtil.exportCurrentSpan()) .setContainerID(containerID).build(); - try { - GetContainerWithPipelineResponseProto response = - rpcProxy.getContainerWithPipeline(NULL_RPC_CONTROLLER, request); - return ContainerWithPipeline.fromProtobuf( - response.getContainerWithPipeline()); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + + ScmContainerLocationResponse response = + submitRequest(Type.GetContainerWithPipeline, + (builder) -> builder.setGetContainerWithPipelineRequest(request)); + + return ContainerWithPipeline.fromProtobuf( + response.getGetContainerWithPipelineResponse() + .getContainerWithPipeline()); + } /** @@ -191,26 +204,22 @@ public List listContainer(long startContainerID, int count) builder.setTraceID(TracingUtil.exportCurrentSpan()); SCMListContainerRequestProto request = builder.build(); - try { - SCMListContainerResponseProto response = - rpcProxy.listContainer(NULL_RPC_CONTROLLER, request); - List containerList = new ArrayList<>(); - for (HddsProtos.ContainerInfoProto containerInfoProto : response - .getContainersList()) { - containerList.add(ContainerInfo.fromProtobuf(containerInfoProto)); - } - return containerList; - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); + SCMListContainerResponseProto response = + submitRequest(Type.ListContainer, + builder1 -> builder1.setScmListContainerRequest(request)) + .getScmListContainerResponse(); + List containerList = new ArrayList<>(); + for (HddsProtos.ContainerInfoProto containerInfoProto : response + .getContainersList()) { + containerList.add(ContainerInfo.fromProtobuf(containerInfoProto)); } + return containerList; + } /** * Ask SCM to delete a container by name. SCM will remove * the container mapping in its database. - * - * @param containerID - * @throws IOException */ @Override public void deleteContainer(long containerID) @@ -222,18 +231,13 @@ public void deleteContainer(long containerID) .setTraceID(TracingUtil.exportCurrentSpan()) .setContainerID(containerID) .build(); - try { - rpcProxy.deleteContainer(NULL_RPC_CONTROLLER, request); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + submitRequest(Type.DeleteContainer, + builder -> builder.setScmDeleteContainerRequest(request)); + } /** * Queries a list of Node Statuses. - * - * @param nodeStatuses - * @return List of Datanodes. */ @Override public List queryNode(HddsProtos.NodeState @@ -246,21 +250,18 @@ public List queryNode(HddsProtos.NodeState .setState(nodeStatuses) .setTraceID(TracingUtil.exportCurrentSpan()) .setScope(queryScope).setPoolName(poolName).build(); - try { - NodeQueryResponseProto response = - rpcProxy.queryNode(NULL_RPC_CONTROLLER, request); - return response.getDatanodesList(); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + NodeQueryResponseProto response = submitRequest(Type.QueryNode, + builder -> builder.setNodeQueryRequest(request)).getNodeQueryResponse(); + return response.getDatanodesList(); } /** * Notify from client that creates object on datanodes. - * @param type object type - * @param id object id - * @param op operation type (e.g., create, close, delete) + * + * @param type object type + * @param id object id + * @param op operation type (e.g., create, close, delete) * @param stage object creation stage : begin/complete */ @Override @@ -278,20 +279,17 @@ public void notifyObjectStageChange( .setOp(op) .setStage(stage) .build(); - try { - rpcProxy.notifyObjectStageChange(NULL_RPC_CONTROLLER, request); - } catch(ServiceException e){ - throw ProtobufHelper.getRemoteException(e); - } + submitRequest(Type.NotifyObjectStageChange, + builder -> builder.setObjectStageChangeRequest(request)); + } /** * Creates a replication pipeline of a specified type. * * @param replicationType - replication type - * @param factor - factor 1 or 3 - * @param nodePool - optional machine list to build a pipeline. - * @throws IOException + * @param factor - factor 1 or 3 + * @param nodePool - optional machine list to build a pipeline. */ @Override public Pipeline createReplicationPipeline(HddsProtos.ReplicationType @@ -303,87 +301,82 @@ public Pipeline createReplicationPipeline(HddsProtos.ReplicationType .setReplicationFactor(factor) .setReplicationType(replicationType) .build(); - try { - PipelineResponseProto response = - rpcProxy.allocatePipeline(NULL_RPC_CONTROLLER, request); - if (response.getErrorCode() == - PipelineResponseProto.Error.success) { - Preconditions.checkState(response.hasPipeline(), "With success, " + - "must come a pipeline"); - return Pipeline.getFromProtobuf(response.getPipeline()); - } else { - String errorMessage = String.format("create replication pipeline " + - "failed. code : %s Message: %s", response.getErrorCode(), - response.hasErrorMessage() ? response.getErrorMessage() : ""); - throw new IOException(errorMessage); - } - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); + + PipelineResponseProto response = + submitRequest(Type.AllocatePipeline, + builder -> builder.setPipelineRequest(request)) + .getPipelineResponse(); + if (response.getErrorCode() == + PipelineResponseProto.Error.success) { + Preconditions.checkState(response.hasPipeline(), "With success, " + + "must come a pipeline"); + return Pipeline.getFromProtobuf(response.getPipeline()); + } else { + String errorMessage = String.format("create replication pipeline " + + "failed. code : %s Message: %s", response.getErrorCode(), + response.hasErrorMessage() ? response.getErrorMessage() : ""); + throw new IOException(errorMessage); } + } @Override public List listPipelines() throws IOException { - try { - ListPipelineRequestProto request = ListPipelineRequestProto - .newBuilder().setTraceID(TracingUtil.exportCurrentSpan()) - .build(); - ListPipelineResponseProto response = rpcProxy.listPipelines( - NULL_RPC_CONTROLLER, request); - List list = new ArrayList<>(); - for (HddsProtos.Pipeline pipeline : response.getPipelinesList()) { - Pipeline fromProtobuf = Pipeline.getFromProtobuf(pipeline); - list.add(fromProtobuf); - } - return list; - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); + ListPipelineRequestProto request = ListPipelineRequestProto + .newBuilder().setTraceID(TracingUtil.exportCurrentSpan()) + .build(); + + ListPipelineResponseProto response = submitRequest(Type.ListPipelines, + builder -> builder.setListPipelineRequest(request)) + .getListPipelineResponse(); + + List list = new ArrayList<>(); + for (HddsProtos.Pipeline pipeline : response.getPipelinesList()) { + Pipeline fromProtobuf = Pipeline.getFromProtobuf(pipeline); + list.add(fromProtobuf); } + return list; + } @Override public void activatePipeline(HddsProtos.PipelineID pipelineID) throws IOException { - try { - ActivatePipelineRequestProto request = - ActivatePipelineRequestProto.newBuilder() - .setTraceID(TracingUtil.exportCurrentSpan()) - .setPipelineID(pipelineID) - .build(); - rpcProxy.activatePipeline(NULL_RPC_CONTROLLER, request); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + ActivatePipelineRequestProto request = + ActivatePipelineRequestProto.newBuilder() + .setTraceID(TracingUtil.exportCurrentSpan()) + .setPipelineID(pipelineID) + .build(); + submitRequest(Type.ActivatePipeline, + builder -> builder.setActivatePipelineRequest(request)); + } @Override public void deactivatePipeline(HddsProtos.PipelineID pipelineID) throws IOException { - try { - DeactivatePipelineRequestProto request = - DeactivatePipelineRequestProto.newBuilder() - .setTraceID(TracingUtil.exportCurrentSpan()) - .setPipelineID(pipelineID) - .build(); - rpcProxy.deactivatePipeline(NULL_RPC_CONTROLLER, request); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + + DeactivatePipelineRequestProto request = + DeactivatePipelineRequestProto.newBuilder() + .setTraceID(TracingUtil.exportCurrentSpan()) + .setPipelineID(pipelineID) + .build(); + submitRequest(Type.DeactivatePipeline, + builder -> builder.setDeactivatePipelineRequest(request)); } @Override public void closePipeline(HddsProtos.PipelineID pipelineID) throws IOException { - try { - ClosePipelineRequestProto request = - ClosePipelineRequestProto.newBuilder() - .setTraceID(TracingUtil.exportCurrentSpan()) - .setPipelineID(pipelineID) - .build(); - rpcProxy.closePipeline(NULL_RPC_CONTROLLER, request); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + + ClosePipelineRequestProto request = + ClosePipelineRequestProto.newBuilder() + .setTraceID(TracingUtil.exportCurrentSpan()) + .setPipelineID(pipelineID) + .build(); + submitRequest(Type.ClosePipeline, + builder -> builder.setClosePipelineRequest(request)); + } @Override @@ -392,16 +385,14 @@ public ScmInfo getScmInfo() throws IOException { HddsProtos.GetScmInfoRequestProto.newBuilder() .setTraceID(TracingUtil.exportCurrentSpan()) .build(); - try { - HddsProtos.GetScmInfoResponseProto resp = rpcProxy.getScmInfo( - NULL_RPC_CONTROLLER, request); - ScmInfo.Builder builder = new ScmInfo.Builder() - .setClusterId(resp.getClusterId()) - .setScmId(resp.getScmId()); - return builder.build(); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + + GetScmInfoResponseProto resp = submitRequest(Type.GetScmInfo, + builder -> builder.setGetScmInfoRequest(request)) + .getGetScmInfoResponse(); + ScmInfo.Builder builder = new ScmInfo.Builder() + .setClusterId(resp.getClusterId()) + .setScmId(resp.getScmId()); + return builder.build(); } @@ -409,73 +400,67 @@ public ScmInfo getScmInfo() throws IOException { * Check if SCM is in safe mode. * * @return Returns true if SCM is in safe mode else returns false. - * @throws IOException */ @Override public boolean inSafeMode() throws IOException { InSafeModeRequestProto request = InSafeModeRequestProto.getDefaultInstance(); - try { - InSafeModeResponseProto resp = rpcProxy.inSafeMode( - NULL_RPC_CONTROLLER, request); - return resp.getInSafeMode(); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + + return submitRequest(Type.InSafeMode, + builder -> builder.setInSafeModeRequest(request)) + .getInSafeModeResponse().getInSafeMode(); + } /** * Force SCM out of Safe mode. * * @return returns true if operation is successful. - * @throws IOException */ @Override public boolean forceExitSafeMode() throws IOException { ForceExitSafeModeRequestProto request = ForceExitSafeModeRequestProto.getDefaultInstance(); - try { - ForceExitSafeModeResponseProto resp = rpcProxy - .forceExitSafeMode(NULL_RPC_CONTROLLER, request); - return resp.getExitedSafeMode(); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + ForceExitSafeModeResponseProto resp = + submitRequest(Type.ForceExitSafeMode, + builder -> builder.setForceExitSafeModeRequest(request)) + .getForceExitSafeModeResponse(); + + return resp.getExitedSafeMode(); + } @Override public void startReplicationManager() throws IOException { - try { - StartReplicationManagerRequestProto request = - StartReplicationManagerRequestProto.getDefaultInstance(); - rpcProxy.startReplicationManager(NULL_RPC_CONTROLLER, request); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + + StartReplicationManagerRequestProto request = + StartReplicationManagerRequestProto.getDefaultInstance(); + submitRequest(Type.StartReplicationManager, + builder -> builder.setStartReplicationManagerRequest(request)); + } @Override public void stopReplicationManager() throws IOException { - try { - StopReplicationManagerRequestProto request = - StopReplicationManagerRequestProto.getDefaultInstance(); - rpcProxy.stopReplicationManager(NULL_RPC_CONTROLLER, request); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + + StopReplicationManagerRequestProto request = + StopReplicationManagerRequestProto.getDefaultInstance(); + submitRequest(Type.StopReplicationManager, + builder -> builder.setStopReplicationManagerRequest(request)); + } @Override public boolean getReplicationManagerStatus() throws IOException { - try { - ReplicationManagerStatusRequestProto request = - ReplicationManagerStatusRequestProto.getDefaultInstance(); - ReplicationManagerStatusResponseProto response = - rpcProxy.getReplicationManagerStatus(NULL_RPC_CONTROLLER, request); - return response.getIsRunning(); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } + + ReplicationManagerStatusRequestProto request = + ReplicationManagerStatusRequestProto.getDefaultInstance(); + ReplicationManagerStatusResponseProto response = + submitRequest(Type.GetReplicationManagerStatus, + builder -> builder.setSeplicationManagerStatusRequest(request)) + .getReplicationManagerStatusResponse(); + return response.getIsRunning(); + } @Override diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java index 4d5ecab68b52b..263864fede890 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java @@ -22,11 +22,13 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdfs.DFSUtil; import org.eclipse.jetty.util.StringUtil; import org.rocksdb.ColumnFamilyDescriptor; import org.rocksdb.ColumnFamilyOptions; import org.rocksdb.DBOptions; +import org.rocksdb.InfoLogLevel; import org.rocksdb.RocksDB; import org.rocksdb.Statistics; import org.rocksdb.StatsLevel; @@ -54,6 +56,8 @@ public final class DBStoreBuilder { private static final Logger LOG = LoggerFactory.getLogger(DBStoreBuilder.class); + public static final Logger ROCKS_DB_LOGGER = + LoggerFactory.getLogger(RocksDB.class); private Set tables; private DBProfile dbProfile; private DBOptions rocksDBOption; @@ -63,8 +67,9 @@ public final class DBStoreBuilder { private Configuration configuration; private CodecRegistry registry; private String rocksDbStat; + private RocksDBConfiguration rocksDBConfiguration; - private DBStoreBuilder(Configuration configuration) { + private DBStoreBuilder(OzoneConfiguration configuration) { tables = new HashSet<>(); tableNames = new LinkedList<>(); this.configuration = configuration; @@ -72,9 +77,11 @@ private DBStoreBuilder(Configuration configuration) { this.rocksDbStat = configuration.getTrimmed( OZONE_METADATA_STORE_ROCKSDB_STATISTICS, OZONE_METADATA_STORE_ROCKSDB_STATISTICS_DEFAULT); + this.rocksDBConfiguration = + configuration.getObject(RocksDBConfiguration.class); } - public static DBStoreBuilder newBuilder(Configuration configuration) { + public static DBStoreBuilder newBuilder(OzoneConfiguration configuration) { return new DBStoreBuilder(configuration); } @@ -199,6 +206,19 @@ private DBOptions getDbProfile() { option = dbProfile.getDBOptions(); } + if (rocksDBConfiguration.isRocksdbLoggingEnabled()) { + org.rocksdb.Logger logger = new org.rocksdb.Logger(option) { + @Override + protected void log(InfoLogLevel infoLogLevel, String s) { + ROCKS_DB_LOGGER.info(s); + } + }; + InfoLogLevel level = InfoLogLevel.valueOf(rocksDBConfiguration + .getRocksdbLogLevel() + "_LEVEL"); + logger.setInfoLogLevel(level); + option.setLogger(logger); + } + if (!rocksDbStat.equals(OZONE_METADATA_STORE_ROCKSDB_STATISTICS_OFF)) { Statistics statistics = new Statistics(); statistics.setStatsLevel(StatsLevel.valueOf(rocksDbStat)); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RDBCheckpointManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RDBCheckpointManager.java index 53aeab7c3abba..42b9b77d2d874 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RDBCheckpointManager.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RDBCheckpointManager.java @@ -83,7 +83,7 @@ public RocksDBCheckpoint createCheckpoint(String parentDir) { Instant end = Instant.now(); long duration = Duration.between(start, end).toMillis(); - LOG.debug("Created checkpoint at " + checkpointPath.toString() + " in " + LOG.info("Created checkpoint at " + checkpointPath.toString() + " in " + duration + " milliseconds"); return new RocksDBCheckpoint( diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBCheckpoint.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBCheckpoint.java index 5e8843a91bef5..149743816c203 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBCheckpoint.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBCheckpoint.java @@ -76,7 +76,8 @@ public long checkpointCreationTimeTaken() { @Override public void cleanupCheckpoint() throws IOException { - LOG.debug("Cleaning up checkpoint at " + checkpointLocation.toString()); + LOG.info("Cleaning up RocksDB checkpoint at " + + checkpointLocation.toString()); FileUtils.deleteDirectory(checkpointLocation.toFile()); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBConfiguration.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBConfiguration.java new file mode 100644 index 0000000000000..1a8c846a3eb21 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBConfiguration.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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.utils.db; + +import org.apache.hadoop.hdds.conf.Config; +import org.apache.hadoop.hdds.conf.ConfigGroup; +import org.apache.hadoop.hdds.conf.ConfigTag; +import org.apache.hadoop.hdds.conf.ConfigType; + +/** + * Holds configuration items for OM RocksDB. + */ +@ConfigGroup(prefix = "hadoop.hdds.db") +public class RocksDBConfiguration { + + private boolean rocksdbLogEnabled; + + @Config(key = "rocksdb.logging.enabled", + type = ConfigType.BOOLEAN, + defaultValue = "false", + tags = {ConfigTag.OM}, + description = "Enable/Disable RocksDB logging for OM.") + public void setRocksdbLoggingEnabled(boolean enabled) { + this.rocksdbLogEnabled = enabled; + } + + public boolean isRocksdbLoggingEnabled() { + return rocksdbLogEnabled; + } + + private String rocksdbLogLevel; + + @Config(key = "rocksdb.logging.level", + type = ConfigType.STRING, + defaultValue = "INFO", + tags = {ConfigTag.OM}, + description = "OM RocksDB logging level (INFO/DEBUG/WARN/ERROR/FATAL)") + public void setRocksdbLogLevel(String level) { + this.rocksdbLogLevel = level; + } + + public String getRocksdbLogLevel() { + return rocksdbLogLevel; + } + +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index c5f23bbf9da84..597eff1f658fe 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -104,7 +104,7 @@ public TypedTable( // We should build cache after OM restart when clean up policy is // NEVER. Setting epoch value -1, so that when it is marked for // delete, this will be considered for cleanup. - cache.put(new CacheKey<>(kv.getKey()), + cache.loadInitial(new CacheKey<>(kv.getKey()), new CacheValue<>(Optional.of(kv.getValue()), EPOCH_DEFAULT)); } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/cache/TableCache.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/cache/TableCache.java index 1f16969a3ba1a..de5a07978f51e 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/cache/TableCache.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/cache/TableCache.java @@ -43,6 +43,15 @@ public interface TableCache>> 8) ^ getTable()[(((crc ^ b) << 24) >>> 24)]; + } + + @Override + public final void update(ByteBuffer b) { + crc = update(crc, b, getTable()); + } + + private static int update(int crc, ByteBuffer b, int[] table) { + for(; b.remaining() > 7;) { + final int c0 = (b.get() ^ crc) & 0xff; + final int c1 = (b.get() ^ (crc >>>= 8)) & 0xff; + final int c2 = (b.get() ^ (crc >>>= 8)) & 0xff; + final int c3 = (b.get() ^ (crc >>> 8)) & 0xff; + crc = (table[0x700 + c0] ^ table[0x600 + c1]) + ^ (table[0x500 + c2] ^ table[0x400 + c3]); + + final int c4 = b.get() & 0xff; + final int c5 = b.get() & 0xff; + final int c6 = b.get() & 0xff; + final int c7 = b.get() & 0xff; + + crc ^= (table[0x300 + c4] ^ table[0x200 + c5]) + ^ (table[0x100 + c6] ^ table[c7]); + } + + // loop unroll - duff's device style + switch (b.remaining()) { + case 7: + crc = (crc >>> 8) ^ table[((crc ^ b.get()) & 0xff)]; + case 6: + crc = (crc >>> 8) ^ table[((crc ^ b.get()) & 0xff)]; + case 5: + crc = (crc >>> 8) ^ table[((crc ^ b.get()) & 0xff)]; + case 4: + crc = (crc >>> 8) ^ table[((crc ^ b.get()) & 0xff)]; + case 3: + crc = (crc >>> 8) ^ table[((crc ^ b.get()) & 0xff)]; + case 2: + crc = (crc >>> 8) ^ table[((crc ^ b.get()) & 0xff)]; + case 1: + crc = (crc >>> 8) ^ table[((crc ^ b.get()) & 0xff)]; + default: // noop + } + + return crc; + } + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/PureJavaCrc32ByteBuffer.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/PureJavaCrc32ByteBuffer.java new file mode 100644 index 0000000000000..0d1f6307501a3 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/PureJavaCrc32ByteBuffer.java @@ -0,0 +1,556 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.ozone.common; + +/** + * Similar to {@link org.apache.hadoop.util.PureJavaCrc32} + * except that this class implement {@link ChecksumByteBuffer}. + */ +final class PureJavaCrc32ByteBuffer extends ChecksumByteBuffer.CrcIntTable { + @Override + int[] getTable() { + return T; + } + + /** + * CRC-32 lookup table generated by the polynomial 0xEDB88320. + * See also org.apache.hadoop.util.TestPureJavaCrc32.Table. + */ + private static final int[] T = { + /* T8_0 */ + 0x00000000, 0x77073096, 0xEE0E612C, 0x990951BA, + 0x076DC419, 0x706AF48F, 0xE963A535, 0x9E6495A3, + 0x0EDB8832, 0x79DCB8A4, 0xE0D5E91E, 0x97D2D988, + 0x09B64C2B, 0x7EB17CBD, 0xE7B82D07, 0x90BF1D91, + 0x1DB71064, 0x6AB020F2, 0xF3B97148, 0x84BE41DE, + 0x1ADAD47D, 0x6DDDE4EB, 0xF4D4B551, 0x83D385C7, + 0x136C9856, 0x646BA8C0, 0xFD62F97A, 0x8A65C9EC, + 0x14015C4F, 0x63066CD9, 0xFA0F3D63, 0x8D080DF5, + 0x3B6E20C8, 0x4C69105E, 0xD56041E4, 0xA2677172, + 0x3C03E4D1, 0x4B04D447, 0xD20D85FD, 0xA50AB56B, + 0x35B5A8FA, 0x42B2986C, 0xDBBBC9D6, 0xACBCF940, + 0x32D86CE3, 0x45DF5C75, 0xDCD60DCF, 0xABD13D59, + 0x26D930AC, 0x51DE003A, 0xC8D75180, 0xBFD06116, + 0x21B4F4B5, 0x56B3C423, 0xCFBA9599, 0xB8BDA50F, + 0x2802B89E, 0x5F058808, 0xC60CD9B2, 0xB10BE924, + 0x2F6F7C87, 0x58684C11, 0xC1611DAB, 0xB6662D3D, + 0x76DC4190, 0x01DB7106, 0x98D220BC, 0xEFD5102A, + 0x71B18589, 0x06B6B51F, 0x9FBFE4A5, 0xE8B8D433, + 0x7807C9A2, 0x0F00F934, 0x9609A88E, 0xE10E9818, + 0x7F6A0DBB, 0x086D3D2D, 0x91646C97, 0xE6635C01, + 0x6B6B51F4, 0x1C6C6162, 0x856530D8, 0xF262004E, + 0x6C0695ED, 0x1B01A57B, 0x8208F4C1, 0xF50FC457, + 0x65B0D9C6, 0x12B7E950, 0x8BBEB8EA, 0xFCB9887C, + 0x62DD1DDF, 0x15DA2D49, 0x8CD37CF3, 0xFBD44C65, + 0x4DB26158, 0x3AB551CE, 0xA3BC0074, 0xD4BB30E2, + 0x4ADFA541, 0x3DD895D7, 0xA4D1C46D, 0xD3D6F4FB, + 0x4369E96A, 0x346ED9FC, 0xAD678846, 0xDA60B8D0, + 0x44042D73, 0x33031DE5, 0xAA0A4C5F, 0xDD0D7CC9, + 0x5005713C, 0x270241AA, 0xBE0B1010, 0xC90C2086, + 0x5768B525, 0x206F85B3, 0xB966D409, 0xCE61E49F, + 0x5EDEF90E, 0x29D9C998, 0xB0D09822, 0xC7D7A8B4, + 0x59B33D17, 0x2EB40D81, 0xB7BD5C3B, 0xC0BA6CAD, + 0xEDB88320, 0x9ABFB3B6, 0x03B6E20C, 0x74B1D29A, + 0xEAD54739, 0x9DD277AF, 0x04DB2615, 0x73DC1683, + 0xE3630B12, 0x94643B84, 0x0D6D6A3E, 0x7A6A5AA8, + 0xE40ECF0B, 0x9309FF9D, 0x0A00AE27, 0x7D079EB1, + 0xF00F9344, 0x8708A3D2, 0x1E01F268, 0x6906C2FE, + 0xF762575D, 0x806567CB, 0x196C3671, 0x6E6B06E7, + 0xFED41B76, 0x89D32BE0, 0x10DA7A5A, 0x67DD4ACC, + 0xF9B9DF6F, 0x8EBEEFF9, 0x17B7BE43, 0x60B08ED5, + 0xD6D6A3E8, 0xA1D1937E, 0x38D8C2C4, 0x4FDFF252, + 0xD1BB67F1, 0xA6BC5767, 0x3FB506DD, 0x48B2364B, + 0xD80D2BDA, 0xAF0A1B4C, 0x36034AF6, 0x41047A60, + 0xDF60EFC3, 0xA867DF55, 0x316E8EEF, 0x4669BE79, + 0xCB61B38C, 0xBC66831A, 0x256FD2A0, 0x5268E236, + 0xCC0C7795, 0xBB0B4703, 0x220216B9, 0x5505262F, + 0xC5BA3BBE, 0xB2BD0B28, 0x2BB45A92, 0x5CB36A04, + 0xC2D7FFA7, 0xB5D0CF31, 0x2CD99E8B, 0x5BDEAE1D, + 0x9B64C2B0, 0xEC63F226, 0x756AA39C, 0x026D930A, + 0x9C0906A9, 0xEB0E363F, 0x72076785, 0x05005713, + 0x95BF4A82, 0xE2B87A14, 0x7BB12BAE, 0x0CB61B38, + 0x92D28E9B, 0xE5D5BE0D, 0x7CDCEFB7, 0x0BDBDF21, + 0x86D3D2D4, 0xF1D4E242, 0x68DDB3F8, 0x1FDA836E, + 0x81BE16CD, 0xF6B9265B, 0x6FB077E1, 0x18B74777, + 0x88085AE6, 0xFF0F6A70, 0x66063BCA, 0x11010B5C, + 0x8F659EFF, 0xF862AE69, 0x616BFFD3, 0x166CCF45, + 0xA00AE278, 0xD70DD2EE, 0x4E048354, 0x3903B3C2, + 0xA7672661, 0xD06016F7, 0x4969474D, 0x3E6E77DB, + 0xAED16A4A, 0xD9D65ADC, 0x40DF0B66, 0x37D83BF0, + 0xA9BCAE53, 0xDEBB9EC5, 0x47B2CF7F, 0x30B5FFE9, + 0xBDBDF21C, 0xCABAC28A, 0x53B39330, 0x24B4A3A6, + 0xBAD03605, 0xCDD70693, 0x54DE5729, 0x23D967BF, + 0xB3667A2E, 0xC4614AB8, 0x5D681B02, 0x2A6F2B94, + 0xB40BBE37, 0xC30C8EA1, 0x5A05DF1B, 0x2D02EF8D, + /* T8_1 */ + 0x00000000, 0x191B3141, 0x32366282, 0x2B2D53C3, + 0x646CC504, 0x7D77F445, 0x565AA786, 0x4F4196C7, + 0xC8D98A08, 0xD1C2BB49, 0xFAEFE88A, 0xE3F4D9CB, + 0xACB54F0C, 0xB5AE7E4D, 0x9E832D8E, 0x87981CCF, + 0x4AC21251, 0x53D92310, 0x78F470D3, 0x61EF4192, + 0x2EAED755, 0x37B5E614, 0x1C98B5D7, 0x05838496, + 0x821B9859, 0x9B00A918, 0xB02DFADB, 0xA936CB9A, + 0xE6775D5D, 0xFF6C6C1C, 0xD4413FDF, 0xCD5A0E9E, + 0x958424A2, 0x8C9F15E3, 0xA7B24620, 0xBEA97761, + 0xF1E8E1A6, 0xE8F3D0E7, 0xC3DE8324, 0xDAC5B265, + 0x5D5DAEAA, 0x44469FEB, 0x6F6BCC28, 0x7670FD69, + 0x39316BAE, 0x202A5AEF, 0x0B07092C, 0x121C386D, + 0xDF4636F3, 0xC65D07B2, 0xED705471, 0xF46B6530, + 0xBB2AF3F7, 0xA231C2B6, 0x891C9175, 0x9007A034, + 0x179FBCFB, 0x0E848DBA, 0x25A9DE79, 0x3CB2EF38, + 0x73F379FF, 0x6AE848BE, 0x41C51B7D, 0x58DE2A3C, + 0xF0794F05, 0xE9627E44, 0xC24F2D87, 0xDB541CC6, + 0x94158A01, 0x8D0EBB40, 0xA623E883, 0xBF38D9C2, + 0x38A0C50D, 0x21BBF44C, 0x0A96A78F, 0x138D96CE, + 0x5CCC0009, 0x45D73148, 0x6EFA628B, 0x77E153CA, + 0xBABB5D54, 0xA3A06C15, 0x888D3FD6, 0x91960E97, + 0xDED79850, 0xC7CCA911, 0xECE1FAD2, 0xF5FACB93, + 0x7262D75C, 0x6B79E61D, 0x4054B5DE, 0x594F849F, + 0x160E1258, 0x0F152319, 0x243870DA, 0x3D23419B, + 0x65FD6BA7, 0x7CE65AE6, 0x57CB0925, 0x4ED03864, + 0x0191AEA3, 0x188A9FE2, 0x33A7CC21, 0x2ABCFD60, + 0xAD24E1AF, 0xB43FD0EE, 0x9F12832D, 0x8609B26C, + 0xC94824AB, 0xD05315EA, 0xFB7E4629, 0xE2657768, + 0x2F3F79F6, 0x362448B7, 0x1D091B74, 0x04122A35, + 0x4B53BCF2, 0x52488DB3, 0x7965DE70, 0x607EEF31, + 0xE7E6F3FE, 0xFEFDC2BF, 0xD5D0917C, 0xCCCBA03D, + 0x838A36FA, 0x9A9107BB, 0xB1BC5478, 0xA8A76539, + 0x3B83984B, 0x2298A90A, 0x09B5FAC9, 0x10AECB88, + 0x5FEF5D4F, 0x46F46C0E, 0x6DD93FCD, 0x74C20E8C, + 0xF35A1243, 0xEA412302, 0xC16C70C1, 0xD8774180, + 0x9736D747, 0x8E2DE606, 0xA500B5C5, 0xBC1B8484, + 0x71418A1A, 0x685ABB5B, 0x4377E898, 0x5A6CD9D9, + 0x152D4F1E, 0x0C367E5F, 0x271B2D9C, 0x3E001CDD, + 0xB9980012, 0xA0833153, 0x8BAE6290, 0x92B553D1, + 0xDDF4C516, 0xC4EFF457, 0xEFC2A794, 0xF6D996D5, + 0xAE07BCE9, 0xB71C8DA8, 0x9C31DE6B, 0x852AEF2A, + 0xCA6B79ED, 0xD37048AC, 0xF85D1B6F, 0xE1462A2E, + 0x66DE36E1, 0x7FC507A0, 0x54E85463, 0x4DF36522, + 0x02B2F3E5, 0x1BA9C2A4, 0x30849167, 0x299FA026, + 0xE4C5AEB8, 0xFDDE9FF9, 0xD6F3CC3A, 0xCFE8FD7B, + 0x80A96BBC, 0x99B25AFD, 0xB29F093E, 0xAB84387F, + 0x2C1C24B0, 0x350715F1, 0x1E2A4632, 0x07317773, + 0x4870E1B4, 0x516BD0F5, 0x7A468336, 0x635DB277, + 0xCBFAD74E, 0xD2E1E60F, 0xF9CCB5CC, 0xE0D7848D, + 0xAF96124A, 0xB68D230B, 0x9DA070C8, 0x84BB4189, + 0x03235D46, 0x1A386C07, 0x31153FC4, 0x280E0E85, + 0x674F9842, 0x7E54A903, 0x5579FAC0, 0x4C62CB81, + 0x8138C51F, 0x9823F45E, 0xB30EA79D, 0xAA1596DC, + 0xE554001B, 0xFC4F315A, 0xD7626299, 0xCE7953D8, + 0x49E14F17, 0x50FA7E56, 0x7BD72D95, 0x62CC1CD4, + 0x2D8D8A13, 0x3496BB52, 0x1FBBE891, 0x06A0D9D0, + 0x5E7EF3EC, 0x4765C2AD, 0x6C48916E, 0x7553A02F, + 0x3A1236E8, 0x230907A9, 0x0824546A, 0x113F652B, + 0x96A779E4, 0x8FBC48A5, 0xA4911B66, 0xBD8A2A27, + 0xF2CBBCE0, 0xEBD08DA1, 0xC0FDDE62, 0xD9E6EF23, + 0x14BCE1BD, 0x0DA7D0FC, 0x268A833F, 0x3F91B27E, + 0x70D024B9, 0x69CB15F8, 0x42E6463B, 0x5BFD777A, + 0xDC656BB5, 0xC57E5AF4, 0xEE530937, 0xF7483876, + 0xB809AEB1, 0xA1129FF0, 0x8A3FCC33, 0x9324FD72, + /* T8_2 */ + 0x00000000, 0x01C26A37, 0x0384D46E, 0x0246BE59, + 0x0709A8DC, 0x06CBC2EB, 0x048D7CB2, 0x054F1685, + 0x0E1351B8, 0x0FD13B8F, 0x0D9785D6, 0x0C55EFE1, + 0x091AF964, 0x08D89353, 0x0A9E2D0A, 0x0B5C473D, + 0x1C26A370, 0x1DE4C947, 0x1FA2771E, 0x1E601D29, + 0x1B2F0BAC, 0x1AED619B, 0x18ABDFC2, 0x1969B5F5, + 0x1235F2C8, 0x13F798FF, 0x11B126A6, 0x10734C91, + 0x153C5A14, 0x14FE3023, 0x16B88E7A, 0x177AE44D, + 0x384D46E0, 0x398F2CD7, 0x3BC9928E, 0x3A0BF8B9, + 0x3F44EE3C, 0x3E86840B, 0x3CC03A52, 0x3D025065, + 0x365E1758, 0x379C7D6F, 0x35DAC336, 0x3418A901, + 0x3157BF84, 0x3095D5B3, 0x32D36BEA, 0x331101DD, + 0x246BE590, 0x25A98FA7, 0x27EF31FE, 0x262D5BC9, + 0x23624D4C, 0x22A0277B, 0x20E69922, 0x2124F315, + 0x2A78B428, 0x2BBADE1F, 0x29FC6046, 0x283E0A71, + 0x2D711CF4, 0x2CB376C3, 0x2EF5C89A, 0x2F37A2AD, + 0x709A8DC0, 0x7158E7F7, 0x731E59AE, 0x72DC3399, + 0x7793251C, 0x76514F2B, 0x7417F172, 0x75D59B45, + 0x7E89DC78, 0x7F4BB64F, 0x7D0D0816, 0x7CCF6221, + 0x798074A4, 0x78421E93, 0x7A04A0CA, 0x7BC6CAFD, + 0x6CBC2EB0, 0x6D7E4487, 0x6F38FADE, 0x6EFA90E9, + 0x6BB5866C, 0x6A77EC5B, 0x68315202, 0x69F33835, + 0x62AF7F08, 0x636D153F, 0x612BAB66, 0x60E9C151, + 0x65A6D7D4, 0x6464BDE3, 0x662203BA, 0x67E0698D, + 0x48D7CB20, 0x4915A117, 0x4B531F4E, 0x4A917579, + 0x4FDE63FC, 0x4E1C09CB, 0x4C5AB792, 0x4D98DDA5, + 0x46C49A98, 0x4706F0AF, 0x45404EF6, 0x448224C1, + 0x41CD3244, 0x400F5873, 0x4249E62A, 0x438B8C1D, + 0x54F16850, 0x55330267, 0x5775BC3E, 0x56B7D609, + 0x53F8C08C, 0x523AAABB, 0x507C14E2, 0x51BE7ED5, + 0x5AE239E8, 0x5B2053DF, 0x5966ED86, 0x58A487B1, + 0x5DEB9134, 0x5C29FB03, 0x5E6F455A, 0x5FAD2F6D, + 0xE1351B80, 0xE0F771B7, 0xE2B1CFEE, 0xE373A5D9, + 0xE63CB35C, 0xE7FED96B, 0xE5B86732, 0xE47A0D05, + 0xEF264A38, 0xEEE4200F, 0xECA29E56, 0xED60F461, + 0xE82FE2E4, 0xE9ED88D3, 0xEBAB368A, 0xEA695CBD, + 0xFD13B8F0, 0xFCD1D2C7, 0xFE976C9E, 0xFF5506A9, + 0xFA1A102C, 0xFBD87A1B, 0xF99EC442, 0xF85CAE75, + 0xF300E948, 0xF2C2837F, 0xF0843D26, 0xF1465711, + 0xF4094194, 0xF5CB2BA3, 0xF78D95FA, 0xF64FFFCD, + 0xD9785D60, 0xD8BA3757, 0xDAFC890E, 0xDB3EE339, + 0xDE71F5BC, 0xDFB39F8B, 0xDDF521D2, 0xDC374BE5, + 0xD76B0CD8, 0xD6A966EF, 0xD4EFD8B6, 0xD52DB281, + 0xD062A404, 0xD1A0CE33, 0xD3E6706A, 0xD2241A5D, + 0xC55EFE10, 0xC49C9427, 0xC6DA2A7E, 0xC7184049, + 0xC25756CC, 0xC3953CFB, 0xC1D382A2, 0xC011E895, + 0xCB4DAFA8, 0xCA8FC59F, 0xC8C97BC6, 0xC90B11F1, + 0xCC440774, 0xCD866D43, 0xCFC0D31A, 0xCE02B92D, + 0x91AF9640, 0x906DFC77, 0x922B422E, 0x93E92819, + 0x96A63E9C, 0x976454AB, 0x9522EAF2, 0x94E080C5, + 0x9FBCC7F8, 0x9E7EADCF, 0x9C381396, 0x9DFA79A1, + 0x98B56F24, 0x99770513, 0x9B31BB4A, 0x9AF3D17D, + 0x8D893530, 0x8C4B5F07, 0x8E0DE15E, 0x8FCF8B69, + 0x8A809DEC, 0x8B42F7DB, 0x89044982, 0x88C623B5, + 0x839A6488, 0x82580EBF, 0x801EB0E6, 0x81DCDAD1, + 0x8493CC54, 0x8551A663, 0x8717183A, 0x86D5720D, + 0xA9E2D0A0, 0xA820BA97, 0xAA6604CE, 0xABA46EF9, + 0xAEEB787C, 0xAF29124B, 0xAD6FAC12, 0xACADC625, + 0xA7F18118, 0xA633EB2F, 0xA4755576, 0xA5B73F41, + 0xA0F829C4, 0xA13A43F3, 0xA37CFDAA, 0xA2BE979D, + 0xB5C473D0, 0xB40619E7, 0xB640A7BE, 0xB782CD89, + 0xB2CDDB0C, 0xB30FB13B, 0xB1490F62, 0xB08B6555, + 0xBBD72268, 0xBA15485F, 0xB853F606, 0xB9919C31, + 0xBCDE8AB4, 0xBD1CE083, 0xBF5A5EDA, 0xBE9834ED, + /* T8_3 */ + 0x00000000, 0xB8BC6765, 0xAA09C88B, 0x12B5AFEE, + 0x8F629757, 0x37DEF032, 0x256B5FDC, 0x9DD738B9, + 0xC5B428EF, 0x7D084F8A, 0x6FBDE064, 0xD7018701, + 0x4AD6BFB8, 0xF26AD8DD, 0xE0DF7733, 0x58631056, + 0x5019579F, 0xE8A530FA, 0xFA109F14, 0x42ACF871, + 0xDF7BC0C8, 0x67C7A7AD, 0x75720843, 0xCDCE6F26, + 0x95AD7F70, 0x2D111815, 0x3FA4B7FB, 0x8718D09E, + 0x1ACFE827, 0xA2738F42, 0xB0C620AC, 0x087A47C9, + 0xA032AF3E, 0x188EC85B, 0x0A3B67B5, 0xB28700D0, + 0x2F503869, 0x97EC5F0C, 0x8559F0E2, 0x3DE59787, + 0x658687D1, 0xDD3AE0B4, 0xCF8F4F5A, 0x7733283F, + 0xEAE41086, 0x525877E3, 0x40EDD80D, 0xF851BF68, + 0xF02BF8A1, 0x48979FC4, 0x5A22302A, 0xE29E574F, + 0x7F496FF6, 0xC7F50893, 0xD540A77D, 0x6DFCC018, + 0x359FD04E, 0x8D23B72B, 0x9F9618C5, 0x272A7FA0, + 0xBAFD4719, 0x0241207C, 0x10F48F92, 0xA848E8F7, + 0x9B14583D, 0x23A83F58, 0x311D90B6, 0x89A1F7D3, + 0x1476CF6A, 0xACCAA80F, 0xBE7F07E1, 0x06C36084, + 0x5EA070D2, 0xE61C17B7, 0xF4A9B859, 0x4C15DF3C, + 0xD1C2E785, 0x697E80E0, 0x7BCB2F0E, 0xC377486B, + 0xCB0D0FA2, 0x73B168C7, 0x6104C729, 0xD9B8A04C, + 0x446F98F5, 0xFCD3FF90, 0xEE66507E, 0x56DA371B, + 0x0EB9274D, 0xB6054028, 0xA4B0EFC6, 0x1C0C88A3, + 0x81DBB01A, 0x3967D77F, 0x2BD27891, 0x936E1FF4, + 0x3B26F703, 0x839A9066, 0x912F3F88, 0x299358ED, + 0xB4446054, 0x0CF80731, 0x1E4DA8DF, 0xA6F1CFBA, + 0xFE92DFEC, 0x462EB889, 0x549B1767, 0xEC277002, + 0x71F048BB, 0xC94C2FDE, 0xDBF98030, 0x6345E755, + 0x6B3FA09C, 0xD383C7F9, 0xC1366817, 0x798A0F72, + 0xE45D37CB, 0x5CE150AE, 0x4E54FF40, 0xF6E89825, + 0xAE8B8873, 0x1637EF16, 0x048240F8, 0xBC3E279D, + 0x21E91F24, 0x99557841, 0x8BE0D7AF, 0x335CB0CA, + 0xED59B63B, 0x55E5D15E, 0x47507EB0, 0xFFEC19D5, + 0x623B216C, 0xDA874609, 0xC832E9E7, 0x708E8E82, + 0x28ED9ED4, 0x9051F9B1, 0x82E4565F, 0x3A58313A, + 0xA78F0983, 0x1F336EE6, 0x0D86C108, 0xB53AA66D, + 0xBD40E1A4, 0x05FC86C1, 0x1749292F, 0xAFF54E4A, + 0x322276F3, 0x8A9E1196, 0x982BBE78, 0x2097D91D, + 0x78F4C94B, 0xC048AE2E, 0xD2FD01C0, 0x6A4166A5, + 0xF7965E1C, 0x4F2A3979, 0x5D9F9697, 0xE523F1F2, + 0x4D6B1905, 0xF5D77E60, 0xE762D18E, 0x5FDEB6EB, + 0xC2098E52, 0x7AB5E937, 0x680046D9, 0xD0BC21BC, + 0x88DF31EA, 0x3063568F, 0x22D6F961, 0x9A6A9E04, + 0x07BDA6BD, 0xBF01C1D8, 0xADB46E36, 0x15080953, + 0x1D724E9A, 0xA5CE29FF, 0xB77B8611, 0x0FC7E174, + 0x9210D9CD, 0x2AACBEA8, 0x38191146, 0x80A57623, + 0xD8C66675, 0x607A0110, 0x72CFAEFE, 0xCA73C99B, + 0x57A4F122, 0xEF189647, 0xFDAD39A9, 0x45115ECC, + 0x764DEE06, 0xCEF18963, 0xDC44268D, 0x64F841E8, + 0xF92F7951, 0x41931E34, 0x5326B1DA, 0xEB9AD6BF, + 0xB3F9C6E9, 0x0B45A18C, 0x19F00E62, 0xA14C6907, + 0x3C9B51BE, 0x842736DB, 0x96929935, 0x2E2EFE50, + 0x2654B999, 0x9EE8DEFC, 0x8C5D7112, 0x34E11677, + 0xA9362ECE, 0x118A49AB, 0x033FE645, 0xBB838120, + 0xE3E09176, 0x5B5CF613, 0x49E959FD, 0xF1553E98, + 0x6C820621, 0xD43E6144, 0xC68BCEAA, 0x7E37A9CF, + 0xD67F4138, 0x6EC3265D, 0x7C7689B3, 0xC4CAEED6, + 0x591DD66F, 0xE1A1B10A, 0xF3141EE4, 0x4BA87981, + 0x13CB69D7, 0xAB770EB2, 0xB9C2A15C, 0x017EC639, + 0x9CA9FE80, 0x241599E5, 0x36A0360B, 0x8E1C516E, + 0x866616A7, 0x3EDA71C2, 0x2C6FDE2C, 0x94D3B949, + 0x090481F0, 0xB1B8E695, 0xA30D497B, 0x1BB12E1E, + 0x43D23E48, 0xFB6E592D, 0xE9DBF6C3, 0x516791A6, + 0xCCB0A91F, 0x740CCE7A, 0x66B96194, 0xDE0506F1, + /* T8_4 */ + 0x00000000, 0x3D6029B0, 0x7AC05360, 0x47A07AD0, + 0xF580A6C0, 0xC8E08F70, 0x8F40F5A0, 0xB220DC10, + 0x30704BC1, 0x0D106271, 0x4AB018A1, 0x77D03111, + 0xC5F0ED01, 0xF890C4B1, 0xBF30BE61, 0x825097D1, + 0x60E09782, 0x5D80BE32, 0x1A20C4E2, 0x2740ED52, + 0x95603142, 0xA80018F2, 0xEFA06222, 0xD2C04B92, + 0x5090DC43, 0x6DF0F5F3, 0x2A508F23, 0x1730A693, + 0xA5107A83, 0x98705333, 0xDFD029E3, 0xE2B00053, + 0xC1C12F04, 0xFCA106B4, 0xBB017C64, 0x866155D4, + 0x344189C4, 0x0921A074, 0x4E81DAA4, 0x73E1F314, + 0xF1B164C5, 0xCCD14D75, 0x8B7137A5, 0xB6111E15, + 0x0431C205, 0x3951EBB5, 0x7EF19165, 0x4391B8D5, + 0xA121B886, 0x9C419136, 0xDBE1EBE6, 0xE681C256, + 0x54A11E46, 0x69C137F6, 0x2E614D26, 0x13016496, + 0x9151F347, 0xAC31DAF7, 0xEB91A027, 0xD6F18997, + 0x64D15587, 0x59B17C37, 0x1E1106E7, 0x23712F57, + 0x58F35849, 0x659371F9, 0x22330B29, 0x1F532299, + 0xAD73FE89, 0x9013D739, 0xD7B3ADE9, 0xEAD38459, + 0x68831388, 0x55E33A38, 0x124340E8, 0x2F236958, + 0x9D03B548, 0xA0639CF8, 0xE7C3E628, 0xDAA3CF98, + 0x3813CFCB, 0x0573E67B, 0x42D39CAB, 0x7FB3B51B, + 0xCD93690B, 0xF0F340BB, 0xB7533A6B, 0x8A3313DB, + 0x0863840A, 0x3503ADBA, 0x72A3D76A, 0x4FC3FEDA, + 0xFDE322CA, 0xC0830B7A, 0x872371AA, 0xBA43581A, + 0x9932774D, 0xA4525EFD, 0xE3F2242D, 0xDE920D9D, + 0x6CB2D18D, 0x51D2F83D, 0x167282ED, 0x2B12AB5D, + 0xA9423C8C, 0x9422153C, 0xD3826FEC, 0xEEE2465C, + 0x5CC29A4C, 0x61A2B3FC, 0x2602C92C, 0x1B62E09C, + 0xF9D2E0CF, 0xC4B2C97F, 0x8312B3AF, 0xBE729A1F, + 0x0C52460F, 0x31326FBF, 0x7692156F, 0x4BF23CDF, + 0xC9A2AB0E, 0xF4C282BE, 0xB362F86E, 0x8E02D1DE, + 0x3C220DCE, 0x0142247E, 0x46E25EAE, 0x7B82771E, + 0xB1E6B092, 0x8C869922, 0xCB26E3F2, 0xF646CA42, + 0x44661652, 0x79063FE2, 0x3EA64532, 0x03C66C82, + 0x8196FB53, 0xBCF6D2E3, 0xFB56A833, 0xC6368183, + 0x74165D93, 0x49767423, 0x0ED60EF3, 0x33B62743, + 0xD1062710, 0xEC660EA0, 0xABC67470, 0x96A65DC0, + 0x248681D0, 0x19E6A860, 0x5E46D2B0, 0x6326FB00, + 0xE1766CD1, 0xDC164561, 0x9BB63FB1, 0xA6D61601, + 0x14F6CA11, 0x2996E3A1, 0x6E369971, 0x5356B0C1, + 0x70279F96, 0x4D47B626, 0x0AE7CCF6, 0x3787E546, + 0x85A73956, 0xB8C710E6, 0xFF676A36, 0xC2074386, + 0x4057D457, 0x7D37FDE7, 0x3A978737, 0x07F7AE87, + 0xB5D77297, 0x88B75B27, 0xCF1721F7, 0xF2770847, + 0x10C70814, 0x2DA721A4, 0x6A075B74, 0x576772C4, + 0xE547AED4, 0xD8278764, 0x9F87FDB4, 0xA2E7D404, + 0x20B743D5, 0x1DD76A65, 0x5A7710B5, 0x67173905, + 0xD537E515, 0xE857CCA5, 0xAFF7B675, 0x92979FC5, + 0xE915E8DB, 0xD475C16B, 0x93D5BBBB, 0xAEB5920B, + 0x1C954E1B, 0x21F567AB, 0x66551D7B, 0x5B3534CB, + 0xD965A31A, 0xE4058AAA, 0xA3A5F07A, 0x9EC5D9CA, + 0x2CE505DA, 0x11852C6A, 0x562556BA, 0x6B457F0A, + 0x89F57F59, 0xB49556E9, 0xF3352C39, 0xCE550589, + 0x7C75D999, 0x4115F029, 0x06B58AF9, 0x3BD5A349, + 0xB9853498, 0x84E51D28, 0xC34567F8, 0xFE254E48, + 0x4C059258, 0x7165BBE8, 0x36C5C138, 0x0BA5E888, + 0x28D4C7DF, 0x15B4EE6F, 0x521494BF, 0x6F74BD0F, + 0xDD54611F, 0xE03448AF, 0xA794327F, 0x9AF41BCF, + 0x18A48C1E, 0x25C4A5AE, 0x6264DF7E, 0x5F04F6CE, + 0xED242ADE, 0xD044036E, 0x97E479BE, 0xAA84500E, + 0x4834505D, 0x755479ED, 0x32F4033D, 0x0F942A8D, + 0xBDB4F69D, 0x80D4DF2D, 0xC774A5FD, 0xFA148C4D, + 0x78441B9C, 0x4524322C, 0x028448FC, 0x3FE4614C, + 0x8DC4BD5C, 0xB0A494EC, 0xF704EE3C, 0xCA64C78C, + /* T8_5 */ + 0x00000000, 0xCB5CD3A5, 0x4DC8A10B, 0x869472AE, + 0x9B914216, 0x50CD91B3, 0xD659E31D, 0x1D0530B8, + 0xEC53826D, 0x270F51C8, 0xA19B2366, 0x6AC7F0C3, + 0x77C2C07B, 0xBC9E13DE, 0x3A0A6170, 0xF156B2D5, + 0x03D6029B, 0xC88AD13E, 0x4E1EA390, 0x85427035, + 0x9847408D, 0x531B9328, 0xD58FE186, 0x1ED33223, + 0xEF8580F6, 0x24D95353, 0xA24D21FD, 0x6911F258, + 0x7414C2E0, 0xBF481145, 0x39DC63EB, 0xF280B04E, + 0x07AC0536, 0xCCF0D693, 0x4A64A43D, 0x81387798, + 0x9C3D4720, 0x57619485, 0xD1F5E62B, 0x1AA9358E, + 0xEBFF875B, 0x20A354FE, 0xA6372650, 0x6D6BF5F5, + 0x706EC54D, 0xBB3216E8, 0x3DA66446, 0xF6FAB7E3, + 0x047A07AD, 0xCF26D408, 0x49B2A6A6, 0x82EE7503, + 0x9FEB45BB, 0x54B7961E, 0xD223E4B0, 0x197F3715, + 0xE82985C0, 0x23755665, 0xA5E124CB, 0x6EBDF76E, + 0x73B8C7D6, 0xB8E41473, 0x3E7066DD, 0xF52CB578, + 0x0F580A6C, 0xC404D9C9, 0x4290AB67, 0x89CC78C2, + 0x94C9487A, 0x5F959BDF, 0xD901E971, 0x125D3AD4, + 0xE30B8801, 0x28575BA4, 0xAEC3290A, 0x659FFAAF, + 0x789ACA17, 0xB3C619B2, 0x35526B1C, 0xFE0EB8B9, + 0x0C8E08F7, 0xC7D2DB52, 0x4146A9FC, 0x8A1A7A59, + 0x971F4AE1, 0x5C439944, 0xDAD7EBEA, 0x118B384F, + 0xE0DD8A9A, 0x2B81593F, 0xAD152B91, 0x6649F834, + 0x7B4CC88C, 0xB0101B29, 0x36846987, 0xFDD8BA22, + 0x08F40F5A, 0xC3A8DCFF, 0x453CAE51, 0x8E607DF4, + 0x93654D4C, 0x58399EE9, 0xDEADEC47, 0x15F13FE2, + 0xE4A78D37, 0x2FFB5E92, 0xA96F2C3C, 0x6233FF99, + 0x7F36CF21, 0xB46A1C84, 0x32FE6E2A, 0xF9A2BD8F, + 0x0B220DC1, 0xC07EDE64, 0x46EAACCA, 0x8DB67F6F, + 0x90B34FD7, 0x5BEF9C72, 0xDD7BEEDC, 0x16273D79, + 0xE7718FAC, 0x2C2D5C09, 0xAAB92EA7, 0x61E5FD02, + 0x7CE0CDBA, 0xB7BC1E1F, 0x31286CB1, 0xFA74BF14, + 0x1EB014D8, 0xD5ECC77D, 0x5378B5D3, 0x98246676, + 0x852156CE, 0x4E7D856B, 0xC8E9F7C5, 0x03B52460, + 0xF2E396B5, 0x39BF4510, 0xBF2B37BE, 0x7477E41B, + 0x6972D4A3, 0xA22E0706, 0x24BA75A8, 0xEFE6A60D, + 0x1D661643, 0xD63AC5E6, 0x50AEB748, 0x9BF264ED, + 0x86F75455, 0x4DAB87F0, 0xCB3FF55E, 0x006326FB, + 0xF135942E, 0x3A69478B, 0xBCFD3525, 0x77A1E680, + 0x6AA4D638, 0xA1F8059D, 0x276C7733, 0xEC30A496, + 0x191C11EE, 0xD240C24B, 0x54D4B0E5, 0x9F886340, + 0x828D53F8, 0x49D1805D, 0xCF45F2F3, 0x04192156, + 0xF54F9383, 0x3E134026, 0xB8873288, 0x73DBE12D, + 0x6EDED195, 0xA5820230, 0x2316709E, 0xE84AA33B, + 0x1ACA1375, 0xD196C0D0, 0x5702B27E, 0x9C5E61DB, + 0x815B5163, 0x4A0782C6, 0xCC93F068, 0x07CF23CD, + 0xF6999118, 0x3DC542BD, 0xBB513013, 0x700DE3B6, + 0x6D08D30E, 0xA65400AB, 0x20C07205, 0xEB9CA1A0, + 0x11E81EB4, 0xDAB4CD11, 0x5C20BFBF, 0x977C6C1A, + 0x8A795CA2, 0x41258F07, 0xC7B1FDA9, 0x0CED2E0C, + 0xFDBB9CD9, 0x36E74F7C, 0xB0733DD2, 0x7B2FEE77, + 0x662ADECF, 0xAD760D6A, 0x2BE27FC4, 0xE0BEAC61, + 0x123E1C2F, 0xD962CF8A, 0x5FF6BD24, 0x94AA6E81, + 0x89AF5E39, 0x42F38D9C, 0xC467FF32, 0x0F3B2C97, + 0xFE6D9E42, 0x35314DE7, 0xB3A53F49, 0x78F9ECEC, + 0x65FCDC54, 0xAEA00FF1, 0x28347D5F, 0xE368AEFA, + 0x16441B82, 0xDD18C827, 0x5B8CBA89, 0x90D0692C, + 0x8DD55994, 0x46898A31, 0xC01DF89F, 0x0B412B3A, + 0xFA1799EF, 0x314B4A4A, 0xB7DF38E4, 0x7C83EB41, + 0x6186DBF9, 0xAADA085C, 0x2C4E7AF2, 0xE712A957, + 0x15921919, 0xDECECABC, 0x585AB812, 0x93066BB7, + 0x8E035B0F, 0x455F88AA, 0xC3CBFA04, 0x089729A1, + 0xF9C19B74, 0x329D48D1, 0xB4093A7F, 0x7F55E9DA, + 0x6250D962, 0xA90C0AC7, 0x2F987869, 0xE4C4ABCC, + /* T8_6 */ + 0x00000000, 0xA6770BB4, 0x979F1129, 0x31E81A9D, + 0xF44F2413, 0x52382FA7, 0x63D0353A, 0xC5A73E8E, + 0x33EF4E67, 0x959845D3, 0xA4705F4E, 0x020754FA, + 0xC7A06A74, 0x61D761C0, 0x503F7B5D, 0xF64870E9, + 0x67DE9CCE, 0xC1A9977A, 0xF0418DE7, 0x56368653, + 0x9391B8DD, 0x35E6B369, 0x040EA9F4, 0xA279A240, + 0x5431D2A9, 0xF246D91D, 0xC3AEC380, 0x65D9C834, + 0xA07EF6BA, 0x0609FD0E, 0x37E1E793, 0x9196EC27, + 0xCFBD399C, 0x69CA3228, 0x582228B5, 0xFE552301, + 0x3BF21D8F, 0x9D85163B, 0xAC6D0CA6, 0x0A1A0712, + 0xFC5277FB, 0x5A257C4F, 0x6BCD66D2, 0xCDBA6D66, + 0x081D53E8, 0xAE6A585C, 0x9F8242C1, 0x39F54975, + 0xA863A552, 0x0E14AEE6, 0x3FFCB47B, 0x998BBFCF, + 0x5C2C8141, 0xFA5B8AF5, 0xCBB39068, 0x6DC49BDC, + 0x9B8CEB35, 0x3DFBE081, 0x0C13FA1C, 0xAA64F1A8, + 0x6FC3CF26, 0xC9B4C492, 0xF85CDE0F, 0x5E2BD5BB, + 0x440B7579, 0xE27C7ECD, 0xD3946450, 0x75E36FE4, + 0xB044516A, 0x16335ADE, 0x27DB4043, 0x81AC4BF7, + 0x77E43B1E, 0xD19330AA, 0xE07B2A37, 0x460C2183, + 0x83AB1F0D, 0x25DC14B9, 0x14340E24, 0xB2430590, + 0x23D5E9B7, 0x85A2E203, 0xB44AF89E, 0x123DF32A, + 0xD79ACDA4, 0x71EDC610, 0x4005DC8D, 0xE672D739, + 0x103AA7D0, 0xB64DAC64, 0x87A5B6F9, 0x21D2BD4D, + 0xE47583C3, 0x42028877, 0x73EA92EA, 0xD59D995E, + 0x8BB64CE5, 0x2DC14751, 0x1C295DCC, 0xBA5E5678, + 0x7FF968F6, 0xD98E6342, 0xE86679DF, 0x4E11726B, + 0xB8590282, 0x1E2E0936, 0x2FC613AB, 0x89B1181F, + 0x4C162691, 0xEA612D25, 0xDB8937B8, 0x7DFE3C0C, + 0xEC68D02B, 0x4A1FDB9F, 0x7BF7C102, 0xDD80CAB6, + 0x1827F438, 0xBE50FF8C, 0x8FB8E511, 0x29CFEEA5, + 0xDF879E4C, 0x79F095F8, 0x48188F65, 0xEE6F84D1, + 0x2BC8BA5F, 0x8DBFB1EB, 0xBC57AB76, 0x1A20A0C2, + 0x8816EAF2, 0x2E61E146, 0x1F89FBDB, 0xB9FEF06F, + 0x7C59CEE1, 0xDA2EC555, 0xEBC6DFC8, 0x4DB1D47C, + 0xBBF9A495, 0x1D8EAF21, 0x2C66B5BC, 0x8A11BE08, + 0x4FB68086, 0xE9C18B32, 0xD82991AF, 0x7E5E9A1B, + 0xEFC8763C, 0x49BF7D88, 0x78576715, 0xDE206CA1, + 0x1B87522F, 0xBDF0599B, 0x8C184306, 0x2A6F48B2, + 0xDC27385B, 0x7A5033EF, 0x4BB82972, 0xEDCF22C6, + 0x28681C48, 0x8E1F17FC, 0xBFF70D61, 0x198006D5, + 0x47ABD36E, 0xE1DCD8DA, 0xD034C247, 0x7643C9F3, + 0xB3E4F77D, 0x1593FCC9, 0x247BE654, 0x820CEDE0, + 0x74449D09, 0xD23396BD, 0xE3DB8C20, 0x45AC8794, + 0x800BB91A, 0x267CB2AE, 0x1794A833, 0xB1E3A387, + 0x20754FA0, 0x86024414, 0xB7EA5E89, 0x119D553D, + 0xD43A6BB3, 0x724D6007, 0x43A57A9A, 0xE5D2712E, + 0x139A01C7, 0xB5ED0A73, 0x840510EE, 0x22721B5A, + 0xE7D525D4, 0x41A22E60, 0x704A34FD, 0xD63D3F49, + 0xCC1D9F8B, 0x6A6A943F, 0x5B828EA2, 0xFDF58516, + 0x3852BB98, 0x9E25B02C, 0xAFCDAAB1, 0x09BAA105, + 0xFFF2D1EC, 0x5985DA58, 0x686DC0C5, 0xCE1ACB71, + 0x0BBDF5FF, 0xADCAFE4B, 0x9C22E4D6, 0x3A55EF62, + 0xABC30345, 0x0DB408F1, 0x3C5C126C, 0x9A2B19D8, + 0x5F8C2756, 0xF9FB2CE2, 0xC813367F, 0x6E643DCB, + 0x982C4D22, 0x3E5B4696, 0x0FB35C0B, 0xA9C457BF, + 0x6C636931, 0xCA146285, 0xFBFC7818, 0x5D8B73AC, + 0x03A0A617, 0xA5D7ADA3, 0x943FB73E, 0x3248BC8A, + 0xF7EF8204, 0x519889B0, 0x6070932D, 0xC6079899, + 0x304FE870, 0x9638E3C4, 0xA7D0F959, 0x01A7F2ED, + 0xC400CC63, 0x6277C7D7, 0x539FDD4A, 0xF5E8D6FE, + 0x647E3AD9, 0xC209316D, 0xF3E12BF0, 0x55962044, + 0x90311ECA, 0x3646157E, 0x07AE0FE3, 0xA1D90457, + 0x579174BE, 0xF1E67F0A, 0xC00E6597, 0x66796E23, + 0xA3DE50AD, 0x05A95B19, 0x34414184, 0x92364A30, + /* T8_7 */ + 0x00000000, 0xCCAA009E, 0x4225077D, 0x8E8F07E3, + 0x844A0EFA, 0x48E00E64, 0xC66F0987, 0x0AC50919, + 0xD3E51BB5, 0x1F4F1B2B, 0x91C01CC8, 0x5D6A1C56, + 0x57AF154F, 0x9B0515D1, 0x158A1232, 0xD92012AC, + 0x7CBB312B, 0xB01131B5, 0x3E9E3656, 0xF23436C8, + 0xF8F13FD1, 0x345B3F4F, 0xBAD438AC, 0x767E3832, + 0xAF5E2A9E, 0x63F42A00, 0xED7B2DE3, 0x21D12D7D, + 0x2B142464, 0xE7BE24FA, 0x69312319, 0xA59B2387, + 0xF9766256, 0x35DC62C8, 0xBB53652B, 0x77F965B5, + 0x7D3C6CAC, 0xB1966C32, 0x3F196BD1, 0xF3B36B4F, + 0x2A9379E3, 0xE639797D, 0x68B67E9E, 0xA41C7E00, + 0xAED97719, 0x62737787, 0xECFC7064, 0x205670FA, + 0x85CD537D, 0x496753E3, 0xC7E85400, 0x0B42549E, + 0x01875D87, 0xCD2D5D19, 0x43A25AFA, 0x8F085A64, + 0x562848C8, 0x9A824856, 0x140D4FB5, 0xD8A74F2B, + 0xD2624632, 0x1EC846AC, 0x9047414F, 0x5CED41D1, + 0x299DC2ED, 0xE537C273, 0x6BB8C590, 0xA712C50E, + 0xADD7CC17, 0x617DCC89, 0xEFF2CB6A, 0x2358CBF4, + 0xFA78D958, 0x36D2D9C6, 0xB85DDE25, 0x74F7DEBB, + 0x7E32D7A2, 0xB298D73C, 0x3C17D0DF, 0xF0BDD041, + 0x5526F3C6, 0x998CF358, 0x1703F4BB, 0xDBA9F425, + 0xD16CFD3C, 0x1DC6FDA2, 0x9349FA41, 0x5FE3FADF, + 0x86C3E873, 0x4A69E8ED, 0xC4E6EF0E, 0x084CEF90, + 0x0289E689, 0xCE23E617, 0x40ACE1F4, 0x8C06E16A, + 0xD0EBA0BB, 0x1C41A025, 0x92CEA7C6, 0x5E64A758, + 0x54A1AE41, 0x980BAEDF, 0x1684A93C, 0xDA2EA9A2, + 0x030EBB0E, 0xCFA4BB90, 0x412BBC73, 0x8D81BCED, + 0x8744B5F4, 0x4BEEB56A, 0xC561B289, 0x09CBB217, + 0xAC509190, 0x60FA910E, 0xEE7596ED, 0x22DF9673, + 0x281A9F6A, 0xE4B09FF4, 0x6A3F9817, 0xA6959889, + 0x7FB58A25, 0xB31F8ABB, 0x3D908D58, 0xF13A8DC6, + 0xFBFF84DF, 0x37558441, 0xB9DA83A2, 0x7570833C, + 0x533B85DA, 0x9F918544, 0x111E82A7, 0xDDB48239, + 0xD7718B20, 0x1BDB8BBE, 0x95548C5D, 0x59FE8CC3, + 0x80DE9E6F, 0x4C749EF1, 0xC2FB9912, 0x0E51998C, + 0x04949095, 0xC83E900B, 0x46B197E8, 0x8A1B9776, + 0x2F80B4F1, 0xE32AB46F, 0x6DA5B38C, 0xA10FB312, + 0xABCABA0B, 0x6760BA95, 0xE9EFBD76, 0x2545BDE8, + 0xFC65AF44, 0x30CFAFDA, 0xBE40A839, 0x72EAA8A7, + 0x782FA1BE, 0xB485A120, 0x3A0AA6C3, 0xF6A0A65D, + 0xAA4DE78C, 0x66E7E712, 0xE868E0F1, 0x24C2E06F, + 0x2E07E976, 0xE2ADE9E8, 0x6C22EE0B, 0xA088EE95, + 0x79A8FC39, 0xB502FCA7, 0x3B8DFB44, 0xF727FBDA, + 0xFDE2F2C3, 0x3148F25D, 0xBFC7F5BE, 0x736DF520, + 0xD6F6D6A7, 0x1A5CD639, 0x94D3D1DA, 0x5879D144, + 0x52BCD85D, 0x9E16D8C3, 0x1099DF20, 0xDC33DFBE, + 0x0513CD12, 0xC9B9CD8C, 0x4736CA6F, 0x8B9CCAF1, + 0x8159C3E8, 0x4DF3C376, 0xC37CC495, 0x0FD6C40B, + 0x7AA64737, 0xB60C47A9, 0x3883404A, 0xF42940D4, + 0xFEEC49CD, 0x32464953, 0xBCC94EB0, 0x70634E2E, + 0xA9435C82, 0x65E95C1C, 0xEB665BFF, 0x27CC5B61, + 0x2D095278, 0xE1A352E6, 0x6F2C5505, 0xA386559B, + 0x061D761C, 0xCAB77682, 0x44387161, 0x889271FF, + 0x825778E6, 0x4EFD7878, 0xC0727F9B, 0x0CD87F05, + 0xD5F86DA9, 0x19526D37, 0x97DD6AD4, 0x5B776A4A, + 0x51B26353, 0x9D1863CD, 0x1397642E, 0xDF3D64B0, + 0x83D02561, 0x4F7A25FF, 0xC1F5221C, 0x0D5F2282, + 0x079A2B9B, 0xCB302B05, 0x45BF2CE6, 0x89152C78, + 0x50353ED4, 0x9C9F3E4A, 0x121039A9, 0xDEBA3937, + 0xD47F302E, 0x18D530B0, 0x965A3753, 0x5AF037CD, + 0xFF6B144A, 0x33C114D4, 0xBD4E1337, 0x71E413A9, + 0x7B211AB0, 0xB78B1A2E, 0x39041DCD, 0xF5AE1D53, + 0x2C8E0FFF, 0xE0240F61, 0x6EAB0882, 0xA201081C, + 0xA8C40105, 0x646E019B, 0xEAE10678, 0x264B06E6 + }; +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/PureJavaCrc32CByteBuffer.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/PureJavaCrc32CByteBuffer.java new file mode 100644 index 0000000000000..1c443575f8179 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/PureJavaCrc32CByteBuffer.java @@ -0,0 +1,559 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * Some portions of this file Copyright (c) 2004-2006 Intel Corportation + * and licensed under the BSD license. + */ +package org.apache.hadoop.ozone.common; + +/** + * Similar to {@link org.apache.hadoop.util.PureJavaCrc32C} + * except that this class implement {@link ChecksumByteBuffer}. + */ +final class PureJavaCrc32CByteBuffer extends ChecksumByteBuffer.CrcIntTable { + @Override + int[] getTable() { + return T; + } + + /** + * CRC-32C lookup table generated by the polynomial 0x82F63B78. + * See also org.apache.hadoop.util.TestPureJavaCrc32.Table. + */ + private static final int[] T = { + /* T8_0 */ + 0x00000000, 0xF26B8303, 0xE13B70F7, 0x1350F3F4, + 0xC79A971F, 0x35F1141C, 0x26A1E7E8, 0xD4CA64EB, + 0x8AD958CF, 0x78B2DBCC, 0x6BE22838, 0x9989AB3B, + 0x4D43CFD0, 0xBF284CD3, 0xAC78BF27, 0x5E133C24, + 0x105EC76F, 0xE235446C, 0xF165B798, 0x030E349B, + 0xD7C45070, 0x25AFD373, 0x36FF2087, 0xC494A384, + 0x9A879FA0, 0x68EC1CA3, 0x7BBCEF57, 0x89D76C54, + 0x5D1D08BF, 0xAF768BBC, 0xBC267848, 0x4E4DFB4B, + 0x20BD8EDE, 0xD2D60DDD, 0xC186FE29, 0x33ED7D2A, + 0xE72719C1, 0x154C9AC2, 0x061C6936, 0xF477EA35, + 0xAA64D611, 0x580F5512, 0x4B5FA6E6, 0xB93425E5, + 0x6DFE410E, 0x9F95C20D, 0x8CC531F9, 0x7EAEB2FA, + 0x30E349B1, 0xC288CAB2, 0xD1D83946, 0x23B3BA45, + 0xF779DEAE, 0x05125DAD, 0x1642AE59, 0xE4292D5A, + 0xBA3A117E, 0x4851927D, 0x5B016189, 0xA96AE28A, + 0x7DA08661, 0x8FCB0562, 0x9C9BF696, 0x6EF07595, + 0x417B1DBC, 0xB3109EBF, 0xA0406D4B, 0x522BEE48, + 0x86E18AA3, 0x748A09A0, 0x67DAFA54, 0x95B17957, + 0xCBA24573, 0x39C9C670, 0x2A993584, 0xD8F2B687, + 0x0C38D26C, 0xFE53516F, 0xED03A29B, 0x1F682198, + 0x5125DAD3, 0xA34E59D0, 0xB01EAA24, 0x42752927, + 0x96BF4DCC, 0x64D4CECF, 0x77843D3B, 0x85EFBE38, + 0xDBFC821C, 0x2997011F, 0x3AC7F2EB, 0xC8AC71E8, + 0x1C661503, 0xEE0D9600, 0xFD5D65F4, 0x0F36E6F7, + 0x61C69362, 0x93AD1061, 0x80FDE395, 0x72966096, + 0xA65C047D, 0x5437877E, 0x4767748A, 0xB50CF789, + 0xEB1FCBAD, 0x197448AE, 0x0A24BB5A, 0xF84F3859, + 0x2C855CB2, 0xDEEEDFB1, 0xCDBE2C45, 0x3FD5AF46, + 0x7198540D, 0x83F3D70E, 0x90A324FA, 0x62C8A7F9, + 0xB602C312, 0x44694011, 0x5739B3E5, 0xA55230E6, + 0xFB410CC2, 0x092A8FC1, 0x1A7A7C35, 0xE811FF36, + 0x3CDB9BDD, 0xCEB018DE, 0xDDE0EB2A, 0x2F8B6829, + 0x82F63B78, 0x709DB87B, 0x63CD4B8F, 0x91A6C88C, + 0x456CAC67, 0xB7072F64, 0xA457DC90, 0x563C5F93, + 0x082F63B7, 0xFA44E0B4, 0xE9141340, 0x1B7F9043, + 0xCFB5F4A8, 0x3DDE77AB, 0x2E8E845F, 0xDCE5075C, + 0x92A8FC17, 0x60C37F14, 0x73938CE0, 0x81F80FE3, + 0x55326B08, 0xA759E80B, 0xB4091BFF, 0x466298FC, + 0x1871A4D8, 0xEA1A27DB, 0xF94AD42F, 0x0B21572C, + 0xDFEB33C7, 0x2D80B0C4, 0x3ED04330, 0xCCBBC033, + 0xA24BB5A6, 0x502036A5, 0x4370C551, 0xB11B4652, + 0x65D122B9, 0x97BAA1BA, 0x84EA524E, 0x7681D14D, + 0x2892ED69, 0xDAF96E6A, 0xC9A99D9E, 0x3BC21E9D, + 0xEF087A76, 0x1D63F975, 0x0E330A81, 0xFC588982, + 0xB21572C9, 0x407EF1CA, 0x532E023E, 0xA145813D, + 0x758FE5D6, 0x87E466D5, 0x94B49521, 0x66DF1622, + 0x38CC2A06, 0xCAA7A905, 0xD9F75AF1, 0x2B9CD9F2, + 0xFF56BD19, 0x0D3D3E1A, 0x1E6DCDEE, 0xEC064EED, + 0xC38D26C4, 0x31E6A5C7, 0x22B65633, 0xD0DDD530, + 0x0417B1DB, 0xF67C32D8, 0xE52CC12C, 0x1747422F, + 0x49547E0B, 0xBB3FFD08, 0xA86F0EFC, 0x5A048DFF, + 0x8ECEE914, 0x7CA56A17, 0x6FF599E3, 0x9D9E1AE0, + 0xD3D3E1AB, 0x21B862A8, 0x32E8915C, 0xC083125F, + 0x144976B4, 0xE622F5B7, 0xF5720643, 0x07198540, + 0x590AB964, 0xAB613A67, 0xB831C993, 0x4A5A4A90, + 0x9E902E7B, 0x6CFBAD78, 0x7FAB5E8C, 0x8DC0DD8F, + 0xE330A81A, 0x115B2B19, 0x020BD8ED, 0xF0605BEE, + 0x24AA3F05, 0xD6C1BC06, 0xC5914FF2, 0x37FACCF1, + 0x69E9F0D5, 0x9B8273D6, 0x88D28022, 0x7AB90321, + 0xAE7367CA, 0x5C18E4C9, 0x4F48173D, 0xBD23943E, + 0xF36E6F75, 0x0105EC76, 0x12551F82, 0xE03E9C81, + 0x34F4F86A, 0xC69F7B69, 0xD5CF889D, 0x27A40B9E, + 0x79B737BA, 0x8BDCB4B9, 0x988C474D, 0x6AE7C44E, + 0xBE2DA0A5, 0x4C4623A6, 0x5F16D052, 0xAD7D5351, + /* T8_1 */ + 0x00000000, 0x13A29877, 0x274530EE, 0x34E7A899, + 0x4E8A61DC, 0x5D28F9AB, 0x69CF5132, 0x7A6DC945, + 0x9D14C3B8, 0x8EB65BCF, 0xBA51F356, 0xA9F36B21, + 0xD39EA264, 0xC03C3A13, 0xF4DB928A, 0xE7790AFD, + 0x3FC5F181, 0x2C6769F6, 0x1880C16F, 0x0B225918, + 0x714F905D, 0x62ED082A, 0x560AA0B3, 0x45A838C4, + 0xA2D13239, 0xB173AA4E, 0x859402D7, 0x96369AA0, + 0xEC5B53E5, 0xFFF9CB92, 0xCB1E630B, 0xD8BCFB7C, + 0x7F8BE302, 0x6C297B75, 0x58CED3EC, 0x4B6C4B9B, + 0x310182DE, 0x22A31AA9, 0x1644B230, 0x05E62A47, + 0xE29F20BA, 0xF13DB8CD, 0xC5DA1054, 0xD6788823, + 0xAC154166, 0xBFB7D911, 0x8B507188, 0x98F2E9FF, + 0x404E1283, 0x53EC8AF4, 0x670B226D, 0x74A9BA1A, + 0x0EC4735F, 0x1D66EB28, 0x298143B1, 0x3A23DBC6, + 0xDD5AD13B, 0xCEF8494C, 0xFA1FE1D5, 0xE9BD79A2, + 0x93D0B0E7, 0x80722890, 0xB4958009, 0xA737187E, + 0xFF17C604, 0xECB55E73, 0xD852F6EA, 0xCBF06E9D, + 0xB19DA7D8, 0xA23F3FAF, 0x96D89736, 0x857A0F41, + 0x620305BC, 0x71A19DCB, 0x45463552, 0x56E4AD25, + 0x2C896460, 0x3F2BFC17, 0x0BCC548E, 0x186ECCF9, + 0xC0D23785, 0xD370AFF2, 0xE797076B, 0xF4359F1C, + 0x8E585659, 0x9DFACE2E, 0xA91D66B7, 0xBABFFEC0, + 0x5DC6F43D, 0x4E646C4A, 0x7A83C4D3, 0x69215CA4, + 0x134C95E1, 0x00EE0D96, 0x3409A50F, 0x27AB3D78, + 0x809C2506, 0x933EBD71, 0xA7D915E8, 0xB47B8D9F, + 0xCE1644DA, 0xDDB4DCAD, 0xE9537434, 0xFAF1EC43, + 0x1D88E6BE, 0x0E2A7EC9, 0x3ACDD650, 0x296F4E27, + 0x53028762, 0x40A01F15, 0x7447B78C, 0x67E52FFB, + 0xBF59D487, 0xACFB4CF0, 0x981CE469, 0x8BBE7C1E, + 0xF1D3B55B, 0xE2712D2C, 0xD69685B5, 0xC5341DC2, + 0x224D173F, 0x31EF8F48, 0x050827D1, 0x16AABFA6, + 0x6CC776E3, 0x7F65EE94, 0x4B82460D, 0x5820DE7A, + 0xFBC3FAF9, 0xE861628E, 0xDC86CA17, 0xCF245260, + 0xB5499B25, 0xA6EB0352, 0x920CABCB, 0x81AE33BC, + 0x66D73941, 0x7575A136, 0x419209AF, 0x523091D8, + 0x285D589D, 0x3BFFC0EA, 0x0F186873, 0x1CBAF004, + 0xC4060B78, 0xD7A4930F, 0xE3433B96, 0xF0E1A3E1, + 0x8A8C6AA4, 0x992EF2D3, 0xADC95A4A, 0xBE6BC23D, + 0x5912C8C0, 0x4AB050B7, 0x7E57F82E, 0x6DF56059, + 0x1798A91C, 0x043A316B, 0x30DD99F2, 0x237F0185, + 0x844819FB, 0x97EA818C, 0xA30D2915, 0xB0AFB162, + 0xCAC27827, 0xD960E050, 0xED8748C9, 0xFE25D0BE, + 0x195CDA43, 0x0AFE4234, 0x3E19EAAD, 0x2DBB72DA, + 0x57D6BB9F, 0x447423E8, 0x70938B71, 0x63311306, + 0xBB8DE87A, 0xA82F700D, 0x9CC8D894, 0x8F6A40E3, + 0xF50789A6, 0xE6A511D1, 0xD242B948, 0xC1E0213F, + 0x26992BC2, 0x353BB3B5, 0x01DC1B2C, 0x127E835B, + 0x68134A1E, 0x7BB1D269, 0x4F567AF0, 0x5CF4E287, + 0x04D43CFD, 0x1776A48A, 0x23910C13, 0x30339464, + 0x4A5E5D21, 0x59FCC556, 0x6D1B6DCF, 0x7EB9F5B8, + 0x99C0FF45, 0x8A626732, 0xBE85CFAB, 0xAD2757DC, + 0xD74A9E99, 0xC4E806EE, 0xF00FAE77, 0xE3AD3600, + 0x3B11CD7C, 0x28B3550B, 0x1C54FD92, 0x0FF665E5, + 0x759BACA0, 0x663934D7, 0x52DE9C4E, 0x417C0439, + 0xA6050EC4, 0xB5A796B3, 0x81403E2A, 0x92E2A65D, + 0xE88F6F18, 0xFB2DF76F, 0xCFCA5FF6, 0xDC68C781, + 0x7B5FDFFF, 0x68FD4788, 0x5C1AEF11, 0x4FB87766, + 0x35D5BE23, 0x26772654, 0x12908ECD, 0x013216BA, + 0xE64B1C47, 0xF5E98430, 0xC10E2CA9, 0xD2ACB4DE, + 0xA8C17D9B, 0xBB63E5EC, 0x8F844D75, 0x9C26D502, + 0x449A2E7E, 0x5738B609, 0x63DF1E90, 0x707D86E7, + 0x0A104FA2, 0x19B2D7D5, 0x2D557F4C, 0x3EF7E73B, + 0xD98EEDC6, 0xCA2C75B1, 0xFECBDD28, 0xED69455F, + 0x97048C1A, 0x84A6146D, 0xB041BCF4, 0xA3E32483, + /* T8_2 */ + 0x00000000, 0xA541927E, 0x4F6F520D, 0xEA2EC073, + 0x9EDEA41A, 0x3B9F3664, 0xD1B1F617, 0x74F06469, + 0x38513EC5, 0x9D10ACBB, 0x773E6CC8, 0xD27FFEB6, + 0xA68F9ADF, 0x03CE08A1, 0xE9E0C8D2, 0x4CA15AAC, + 0x70A27D8A, 0xD5E3EFF4, 0x3FCD2F87, 0x9A8CBDF9, + 0xEE7CD990, 0x4B3D4BEE, 0xA1138B9D, 0x045219E3, + 0x48F3434F, 0xEDB2D131, 0x079C1142, 0xA2DD833C, + 0xD62DE755, 0x736C752B, 0x9942B558, 0x3C032726, + 0xE144FB14, 0x4405696A, 0xAE2BA919, 0x0B6A3B67, + 0x7F9A5F0E, 0xDADBCD70, 0x30F50D03, 0x95B49F7D, + 0xD915C5D1, 0x7C5457AF, 0x967A97DC, 0x333B05A2, + 0x47CB61CB, 0xE28AF3B5, 0x08A433C6, 0xADE5A1B8, + 0x91E6869E, 0x34A714E0, 0xDE89D493, 0x7BC846ED, + 0x0F382284, 0xAA79B0FA, 0x40577089, 0xE516E2F7, + 0xA9B7B85B, 0x0CF62A25, 0xE6D8EA56, 0x43997828, + 0x37691C41, 0x92288E3F, 0x78064E4C, 0xDD47DC32, + 0xC76580D9, 0x622412A7, 0x880AD2D4, 0x2D4B40AA, + 0x59BB24C3, 0xFCFAB6BD, 0x16D476CE, 0xB395E4B0, + 0xFF34BE1C, 0x5A752C62, 0xB05BEC11, 0x151A7E6F, + 0x61EA1A06, 0xC4AB8878, 0x2E85480B, 0x8BC4DA75, + 0xB7C7FD53, 0x12866F2D, 0xF8A8AF5E, 0x5DE93D20, + 0x29195949, 0x8C58CB37, 0x66760B44, 0xC337993A, + 0x8F96C396, 0x2AD751E8, 0xC0F9919B, 0x65B803E5, + 0x1148678C, 0xB409F5F2, 0x5E273581, 0xFB66A7FF, + 0x26217BCD, 0x8360E9B3, 0x694E29C0, 0xCC0FBBBE, + 0xB8FFDFD7, 0x1DBE4DA9, 0xF7908DDA, 0x52D11FA4, + 0x1E704508, 0xBB31D776, 0x511F1705, 0xF45E857B, + 0x80AEE112, 0x25EF736C, 0xCFC1B31F, 0x6A802161, + 0x56830647, 0xF3C29439, 0x19EC544A, 0xBCADC634, + 0xC85DA25D, 0x6D1C3023, 0x8732F050, 0x2273622E, + 0x6ED23882, 0xCB93AAFC, 0x21BD6A8F, 0x84FCF8F1, + 0xF00C9C98, 0x554D0EE6, 0xBF63CE95, 0x1A225CEB, + 0x8B277743, 0x2E66E53D, 0xC448254E, 0x6109B730, + 0x15F9D359, 0xB0B84127, 0x5A968154, 0xFFD7132A, + 0xB3764986, 0x1637DBF8, 0xFC191B8B, 0x595889F5, + 0x2DA8ED9C, 0x88E97FE2, 0x62C7BF91, 0xC7862DEF, + 0xFB850AC9, 0x5EC498B7, 0xB4EA58C4, 0x11ABCABA, + 0x655BAED3, 0xC01A3CAD, 0x2A34FCDE, 0x8F756EA0, + 0xC3D4340C, 0x6695A672, 0x8CBB6601, 0x29FAF47F, + 0x5D0A9016, 0xF84B0268, 0x1265C21B, 0xB7245065, + 0x6A638C57, 0xCF221E29, 0x250CDE5A, 0x804D4C24, + 0xF4BD284D, 0x51FCBA33, 0xBBD27A40, 0x1E93E83E, + 0x5232B292, 0xF77320EC, 0x1D5DE09F, 0xB81C72E1, + 0xCCEC1688, 0x69AD84F6, 0x83834485, 0x26C2D6FB, + 0x1AC1F1DD, 0xBF8063A3, 0x55AEA3D0, 0xF0EF31AE, + 0x841F55C7, 0x215EC7B9, 0xCB7007CA, 0x6E3195B4, + 0x2290CF18, 0x87D15D66, 0x6DFF9D15, 0xC8BE0F6B, + 0xBC4E6B02, 0x190FF97C, 0xF321390F, 0x5660AB71, + 0x4C42F79A, 0xE90365E4, 0x032DA597, 0xA66C37E9, + 0xD29C5380, 0x77DDC1FE, 0x9DF3018D, 0x38B293F3, + 0x7413C95F, 0xD1525B21, 0x3B7C9B52, 0x9E3D092C, + 0xEACD6D45, 0x4F8CFF3B, 0xA5A23F48, 0x00E3AD36, + 0x3CE08A10, 0x99A1186E, 0x738FD81D, 0xD6CE4A63, + 0xA23E2E0A, 0x077FBC74, 0xED517C07, 0x4810EE79, + 0x04B1B4D5, 0xA1F026AB, 0x4BDEE6D8, 0xEE9F74A6, + 0x9A6F10CF, 0x3F2E82B1, 0xD50042C2, 0x7041D0BC, + 0xAD060C8E, 0x08479EF0, 0xE2695E83, 0x4728CCFD, + 0x33D8A894, 0x96993AEA, 0x7CB7FA99, 0xD9F668E7, + 0x9557324B, 0x3016A035, 0xDA386046, 0x7F79F238, + 0x0B899651, 0xAEC8042F, 0x44E6C45C, 0xE1A75622, + 0xDDA47104, 0x78E5E37A, 0x92CB2309, 0x378AB177, + 0x437AD51E, 0xE63B4760, 0x0C158713, 0xA954156D, + 0xE5F54FC1, 0x40B4DDBF, 0xAA9A1DCC, 0x0FDB8FB2, + 0x7B2BEBDB, 0xDE6A79A5, 0x3444B9D6, 0x91052BA8, + /* T8_3 */ + 0x00000000, 0xDD45AAB8, 0xBF672381, 0x62228939, + 0x7B2231F3, 0xA6679B4B, 0xC4451272, 0x1900B8CA, + 0xF64463E6, 0x2B01C95E, 0x49234067, 0x9466EADF, + 0x8D665215, 0x5023F8AD, 0x32017194, 0xEF44DB2C, + 0xE964B13D, 0x34211B85, 0x560392BC, 0x8B463804, + 0x924680CE, 0x4F032A76, 0x2D21A34F, 0xF06409F7, + 0x1F20D2DB, 0xC2657863, 0xA047F15A, 0x7D025BE2, + 0x6402E328, 0xB9474990, 0xDB65C0A9, 0x06206A11, + 0xD725148B, 0x0A60BE33, 0x6842370A, 0xB5079DB2, + 0xAC072578, 0x71428FC0, 0x136006F9, 0xCE25AC41, + 0x2161776D, 0xFC24DDD5, 0x9E0654EC, 0x4343FE54, + 0x5A43469E, 0x8706EC26, 0xE524651F, 0x3861CFA7, + 0x3E41A5B6, 0xE3040F0E, 0x81268637, 0x5C632C8F, + 0x45639445, 0x98263EFD, 0xFA04B7C4, 0x27411D7C, + 0xC805C650, 0x15406CE8, 0x7762E5D1, 0xAA274F69, + 0xB327F7A3, 0x6E625D1B, 0x0C40D422, 0xD1057E9A, + 0xABA65FE7, 0x76E3F55F, 0x14C17C66, 0xC984D6DE, + 0xD0846E14, 0x0DC1C4AC, 0x6FE34D95, 0xB2A6E72D, + 0x5DE23C01, 0x80A796B9, 0xE2851F80, 0x3FC0B538, + 0x26C00DF2, 0xFB85A74A, 0x99A72E73, 0x44E284CB, + 0x42C2EEDA, 0x9F874462, 0xFDA5CD5B, 0x20E067E3, + 0x39E0DF29, 0xE4A57591, 0x8687FCA8, 0x5BC25610, + 0xB4868D3C, 0x69C32784, 0x0BE1AEBD, 0xD6A40405, + 0xCFA4BCCF, 0x12E11677, 0x70C39F4E, 0xAD8635F6, + 0x7C834B6C, 0xA1C6E1D4, 0xC3E468ED, 0x1EA1C255, + 0x07A17A9F, 0xDAE4D027, 0xB8C6591E, 0x6583F3A6, + 0x8AC7288A, 0x57828232, 0x35A00B0B, 0xE8E5A1B3, + 0xF1E51979, 0x2CA0B3C1, 0x4E823AF8, 0x93C79040, + 0x95E7FA51, 0x48A250E9, 0x2A80D9D0, 0xF7C57368, + 0xEEC5CBA2, 0x3380611A, 0x51A2E823, 0x8CE7429B, + 0x63A399B7, 0xBEE6330F, 0xDCC4BA36, 0x0181108E, + 0x1881A844, 0xC5C402FC, 0xA7E68BC5, 0x7AA3217D, + 0x52A0C93F, 0x8FE56387, 0xEDC7EABE, 0x30824006, + 0x2982F8CC, 0xF4C75274, 0x96E5DB4D, 0x4BA071F5, + 0xA4E4AAD9, 0x79A10061, 0x1B838958, 0xC6C623E0, + 0xDFC69B2A, 0x02833192, 0x60A1B8AB, 0xBDE41213, + 0xBBC47802, 0x6681D2BA, 0x04A35B83, 0xD9E6F13B, + 0xC0E649F1, 0x1DA3E349, 0x7F816A70, 0xA2C4C0C8, + 0x4D801BE4, 0x90C5B15C, 0xF2E73865, 0x2FA292DD, + 0x36A22A17, 0xEBE780AF, 0x89C50996, 0x5480A32E, + 0x8585DDB4, 0x58C0770C, 0x3AE2FE35, 0xE7A7548D, + 0xFEA7EC47, 0x23E246FF, 0x41C0CFC6, 0x9C85657E, + 0x73C1BE52, 0xAE8414EA, 0xCCA69DD3, 0x11E3376B, + 0x08E38FA1, 0xD5A62519, 0xB784AC20, 0x6AC10698, + 0x6CE16C89, 0xB1A4C631, 0xD3864F08, 0x0EC3E5B0, + 0x17C35D7A, 0xCA86F7C2, 0xA8A47EFB, 0x75E1D443, + 0x9AA50F6F, 0x47E0A5D7, 0x25C22CEE, 0xF8878656, + 0xE1873E9C, 0x3CC29424, 0x5EE01D1D, 0x83A5B7A5, + 0xF90696D8, 0x24433C60, 0x4661B559, 0x9B241FE1, + 0x8224A72B, 0x5F610D93, 0x3D4384AA, 0xE0062E12, + 0x0F42F53E, 0xD2075F86, 0xB025D6BF, 0x6D607C07, + 0x7460C4CD, 0xA9256E75, 0xCB07E74C, 0x16424DF4, + 0x106227E5, 0xCD278D5D, 0xAF050464, 0x7240AEDC, + 0x6B401616, 0xB605BCAE, 0xD4273597, 0x09629F2F, + 0xE6264403, 0x3B63EEBB, 0x59416782, 0x8404CD3A, + 0x9D0475F0, 0x4041DF48, 0x22635671, 0xFF26FCC9, + 0x2E238253, 0xF36628EB, 0x9144A1D2, 0x4C010B6A, + 0x5501B3A0, 0x88441918, 0xEA669021, 0x37233A99, + 0xD867E1B5, 0x05224B0D, 0x6700C234, 0xBA45688C, + 0xA345D046, 0x7E007AFE, 0x1C22F3C7, 0xC167597F, + 0xC747336E, 0x1A0299D6, 0x782010EF, 0xA565BA57, + 0xBC65029D, 0x6120A825, 0x0302211C, 0xDE478BA4, + 0x31035088, 0xEC46FA30, 0x8E647309, 0x5321D9B1, + 0x4A21617B, 0x9764CBC3, 0xF54642FA, 0x2803E842, + /* T8_4 */ + 0x00000000, 0x38116FAC, 0x7022DF58, 0x4833B0F4, + 0xE045BEB0, 0xD854D11C, 0x906761E8, 0xA8760E44, + 0xC5670B91, 0xFD76643D, 0xB545D4C9, 0x8D54BB65, + 0x2522B521, 0x1D33DA8D, 0x55006A79, 0x6D1105D5, + 0x8F2261D3, 0xB7330E7F, 0xFF00BE8B, 0xC711D127, + 0x6F67DF63, 0x5776B0CF, 0x1F45003B, 0x27546F97, + 0x4A456A42, 0x725405EE, 0x3A67B51A, 0x0276DAB6, + 0xAA00D4F2, 0x9211BB5E, 0xDA220BAA, 0xE2336406, + 0x1BA8B557, 0x23B9DAFB, 0x6B8A6A0F, 0x539B05A3, + 0xFBED0BE7, 0xC3FC644B, 0x8BCFD4BF, 0xB3DEBB13, + 0xDECFBEC6, 0xE6DED16A, 0xAEED619E, 0x96FC0E32, + 0x3E8A0076, 0x069B6FDA, 0x4EA8DF2E, 0x76B9B082, + 0x948AD484, 0xAC9BBB28, 0xE4A80BDC, 0xDCB96470, + 0x74CF6A34, 0x4CDE0598, 0x04EDB56C, 0x3CFCDAC0, + 0x51EDDF15, 0x69FCB0B9, 0x21CF004D, 0x19DE6FE1, + 0xB1A861A5, 0x89B90E09, 0xC18ABEFD, 0xF99BD151, + 0x37516AAE, 0x0F400502, 0x4773B5F6, 0x7F62DA5A, + 0xD714D41E, 0xEF05BBB2, 0xA7360B46, 0x9F2764EA, + 0xF236613F, 0xCA270E93, 0x8214BE67, 0xBA05D1CB, + 0x1273DF8F, 0x2A62B023, 0x625100D7, 0x5A406F7B, + 0xB8730B7D, 0x806264D1, 0xC851D425, 0xF040BB89, + 0x5836B5CD, 0x6027DA61, 0x28146A95, 0x10050539, + 0x7D1400EC, 0x45056F40, 0x0D36DFB4, 0x3527B018, + 0x9D51BE5C, 0xA540D1F0, 0xED736104, 0xD5620EA8, + 0x2CF9DFF9, 0x14E8B055, 0x5CDB00A1, 0x64CA6F0D, + 0xCCBC6149, 0xF4AD0EE5, 0xBC9EBE11, 0x848FD1BD, + 0xE99ED468, 0xD18FBBC4, 0x99BC0B30, 0xA1AD649C, + 0x09DB6AD8, 0x31CA0574, 0x79F9B580, 0x41E8DA2C, + 0xA3DBBE2A, 0x9BCAD186, 0xD3F96172, 0xEBE80EDE, + 0x439E009A, 0x7B8F6F36, 0x33BCDFC2, 0x0BADB06E, + 0x66BCB5BB, 0x5EADDA17, 0x169E6AE3, 0x2E8F054F, + 0x86F90B0B, 0xBEE864A7, 0xF6DBD453, 0xCECABBFF, + 0x6EA2D55C, 0x56B3BAF0, 0x1E800A04, 0x269165A8, + 0x8EE76BEC, 0xB6F60440, 0xFEC5B4B4, 0xC6D4DB18, + 0xABC5DECD, 0x93D4B161, 0xDBE70195, 0xE3F66E39, + 0x4B80607D, 0x73910FD1, 0x3BA2BF25, 0x03B3D089, + 0xE180B48F, 0xD991DB23, 0x91A26BD7, 0xA9B3047B, + 0x01C50A3F, 0x39D46593, 0x71E7D567, 0x49F6BACB, + 0x24E7BF1E, 0x1CF6D0B2, 0x54C56046, 0x6CD40FEA, + 0xC4A201AE, 0xFCB36E02, 0xB480DEF6, 0x8C91B15A, + 0x750A600B, 0x4D1B0FA7, 0x0528BF53, 0x3D39D0FF, + 0x954FDEBB, 0xAD5EB117, 0xE56D01E3, 0xDD7C6E4F, + 0xB06D6B9A, 0x887C0436, 0xC04FB4C2, 0xF85EDB6E, + 0x5028D52A, 0x6839BA86, 0x200A0A72, 0x181B65DE, + 0xFA2801D8, 0xC2396E74, 0x8A0ADE80, 0xB21BB12C, + 0x1A6DBF68, 0x227CD0C4, 0x6A4F6030, 0x525E0F9C, + 0x3F4F0A49, 0x075E65E5, 0x4F6DD511, 0x777CBABD, + 0xDF0AB4F9, 0xE71BDB55, 0xAF286BA1, 0x9739040D, + 0x59F3BFF2, 0x61E2D05E, 0x29D160AA, 0x11C00F06, + 0xB9B60142, 0x81A76EEE, 0xC994DE1A, 0xF185B1B6, + 0x9C94B463, 0xA485DBCF, 0xECB66B3B, 0xD4A70497, + 0x7CD10AD3, 0x44C0657F, 0x0CF3D58B, 0x34E2BA27, + 0xD6D1DE21, 0xEEC0B18D, 0xA6F30179, 0x9EE26ED5, + 0x36946091, 0x0E850F3D, 0x46B6BFC9, 0x7EA7D065, + 0x13B6D5B0, 0x2BA7BA1C, 0x63940AE8, 0x5B856544, + 0xF3F36B00, 0xCBE204AC, 0x83D1B458, 0xBBC0DBF4, + 0x425B0AA5, 0x7A4A6509, 0x3279D5FD, 0x0A68BA51, + 0xA21EB415, 0x9A0FDBB9, 0xD23C6B4D, 0xEA2D04E1, + 0x873C0134, 0xBF2D6E98, 0xF71EDE6C, 0xCF0FB1C0, + 0x6779BF84, 0x5F68D028, 0x175B60DC, 0x2F4A0F70, + 0xCD796B76, 0xF56804DA, 0xBD5BB42E, 0x854ADB82, + 0x2D3CD5C6, 0x152DBA6A, 0x5D1E0A9E, 0x650F6532, + 0x081E60E7, 0x300F0F4B, 0x783CBFBF, 0x402DD013, + 0xE85BDE57, 0xD04AB1FB, 0x9879010F, 0xA0686EA3, + /* T8_5 */ + 0x00000000, 0xEF306B19, 0xDB8CA0C3, 0x34BCCBDA, + 0xB2F53777, 0x5DC55C6E, 0x697997B4, 0x8649FCAD, + 0x6006181F, 0x8F367306, 0xBB8AB8DC, 0x54BAD3C5, + 0xD2F32F68, 0x3DC34471, 0x097F8FAB, 0xE64FE4B2, + 0xC00C303E, 0x2F3C5B27, 0x1B8090FD, 0xF4B0FBE4, + 0x72F90749, 0x9DC96C50, 0xA975A78A, 0x4645CC93, + 0xA00A2821, 0x4F3A4338, 0x7B8688E2, 0x94B6E3FB, + 0x12FF1F56, 0xFDCF744F, 0xC973BF95, 0x2643D48C, + 0x85F4168D, 0x6AC47D94, 0x5E78B64E, 0xB148DD57, + 0x370121FA, 0xD8314AE3, 0xEC8D8139, 0x03BDEA20, + 0xE5F20E92, 0x0AC2658B, 0x3E7EAE51, 0xD14EC548, + 0x570739E5, 0xB83752FC, 0x8C8B9926, 0x63BBF23F, + 0x45F826B3, 0xAAC84DAA, 0x9E748670, 0x7144ED69, + 0xF70D11C4, 0x183D7ADD, 0x2C81B107, 0xC3B1DA1E, + 0x25FE3EAC, 0xCACE55B5, 0xFE729E6F, 0x1142F576, + 0x970B09DB, 0x783B62C2, 0x4C87A918, 0xA3B7C201, + 0x0E045BEB, 0xE13430F2, 0xD588FB28, 0x3AB89031, + 0xBCF16C9C, 0x53C10785, 0x677DCC5F, 0x884DA746, + 0x6E0243F4, 0x813228ED, 0xB58EE337, 0x5ABE882E, + 0xDCF77483, 0x33C71F9A, 0x077BD440, 0xE84BBF59, + 0xCE086BD5, 0x213800CC, 0x1584CB16, 0xFAB4A00F, + 0x7CFD5CA2, 0x93CD37BB, 0xA771FC61, 0x48419778, + 0xAE0E73CA, 0x413E18D3, 0x7582D309, 0x9AB2B810, + 0x1CFB44BD, 0xF3CB2FA4, 0xC777E47E, 0x28478F67, + 0x8BF04D66, 0x64C0267F, 0x507CEDA5, 0xBF4C86BC, + 0x39057A11, 0xD6351108, 0xE289DAD2, 0x0DB9B1CB, + 0xEBF65579, 0x04C63E60, 0x307AF5BA, 0xDF4A9EA3, + 0x5903620E, 0xB6330917, 0x828FC2CD, 0x6DBFA9D4, + 0x4BFC7D58, 0xA4CC1641, 0x9070DD9B, 0x7F40B682, + 0xF9094A2F, 0x16392136, 0x2285EAEC, 0xCDB581F5, + 0x2BFA6547, 0xC4CA0E5E, 0xF076C584, 0x1F46AE9D, + 0x990F5230, 0x763F3929, 0x4283F2F3, 0xADB399EA, + 0x1C08B7D6, 0xF338DCCF, 0xC7841715, 0x28B47C0C, + 0xAEFD80A1, 0x41CDEBB8, 0x75712062, 0x9A414B7B, + 0x7C0EAFC9, 0x933EC4D0, 0xA7820F0A, 0x48B26413, + 0xCEFB98BE, 0x21CBF3A7, 0x1577387D, 0xFA475364, + 0xDC0487E8, 0x3334ECF1, 0x0788272B, 0xE8B84C32, + 0x6EF1B09F, 0x81C1DB86, 0xB57D105C, 0x5A4D7B45, + 0xBC029FF7, 0x5332F4EE, 0x678E3F34, 0x88BE542D, + 0x0EF7A880, 0xE1C7C399, 0xD57B0843, 0x3A4B635A, + 0x99FCA15B, 0x76CCCA42, 0x42700198, 0xAD406A81, + 0x2B09962C, 0xC439FD35, 0xF08536EF, 0x1FB55DF6, + 0xF9FAB944, 0x16CAD25D, 0x22761987, 0xCD46729E, + 0x4B0F8E33, 0xA43FE52A, 0x90832EF0, 0x7FB345E9, + 0x59F09165, 0xB6C0FA7C, 0x827C31A6, 0x6D4C5ABF, + 0xEB05A612, 0x0435CD0B, 0x308906D1, 0xDFB96DC8, + 0x39F6897A, 0xD6C6E263, 0xE27A29B9, 0x0D4A42A0, + 0x8B03BE0D, 0x6433D514, 0x508F1ECE, 0xBFBF75D7, + 0x120CEC3D, 0xFD3C8724, 0xC9804CFE, 0x26B027E7, + 0xA0F9DB4A, 0x4FC9B053, 0x7B757B89, 0x94451090, + 0x720AF422, 0x9D3A9F3B, 0xA98654E1, 0x46B63FF8, + 0xC0FFC355, 0x2FCFA84C, 0x1B736396, 0xF443088F, + 0xD200DC03, 0x3D30B71A, 0x098C7CC0, 0xE6BC17D9, + 0x60F5EB74, 0x8FC5806D, 0xBB794BB7, 0x544920AE, + 0xB206C41C, 0x5D36AF05, 0x698A64DF, 0x86BA0FC6, + 0x00F3F36B, 0xEFC39872, 0xDB7F53A8, 0x344F38B1, + 0x97F8FAB0, 0x78C891A9, 0x4C745A73, 0xA344316A, + 0x250DCDC7, 0xCA3DA6DE, 0xFE816D04, 0x11B1061D, + 0xF7FEE2AF, 0x18CE89B6, 0x2C72426C, 0xC3422975, + 0x450BD5D8, 0xAA3BBEC1, 0x9E87751B, 0x71B71E02, + 0x57F4CA8E, 0xB8C4A197, 0x8C786A4D, 0x63480154, + 0xE501FDF9, 0x0A3196E0, 0x3E8D5D3A, 0xD1BD3623, + 0x37F2D291, 0xD8C2B988, 0xEC7E7252, 0x034E194B, + 0x8507E5E6, 0x6A378EFF, 0x5E8B4525, 0xB1BB2E3C, + /* T8_6 */ + 0x00000000, 0x68032CC8, 0xD0065990, 0xB8057558, + 0xA5E0C5D1, 0xCDE3E919, 0x75E69C41, 0x1DE5B089, + 0x4E2DFD53, 0x262ED19B, 0x9E2BA4C3, 0xF628880B, + 0xEBCD3882, 0x83CE144A, 0x3BCB6112, 0x53C84DDA, + 0x9C5BFAA6, 0xF458D66E, 0x4C5DA336, 0x245E8FFE, + 0x39BB3F77, 0x51B813BF, 0xE9BD66E7, 0x81BE4A2F, + 0xD27607F5, 0xBA752B3D, 0x02705E65, 0x6A7372AD, + 0x7796C224, 0x1F95EEEC, 0xA7909BB4, 0xCF93B77C, + 0x3D5B83BD, 0x5558AF75, 0xED5DDA2D, 0x855EF6E5, + 0x98BB466C, 0xF0B86AA4, 0x48BD1FFC, 0x20BE3334, + 0x73767EEE, 0x1B755226, 0xA370277E, 0xCB730BB6, + 0xD696BB3F, 0xBE9597F7, 0x0690E2AF, 0x6E93CE67, + 0xA100791B, 0xC90355D3, 0x7106208B, 0x19050C43, + 0x04E0BCCA, 0x6CE39002, 0xD4E6E55A, 0xBCE5C992, + 0xEF2D8448, 0x872EA880, 0x3F2BDDD8, 0x5728F110, + 0x4ACD4199, 0x22CE6D51, 0x9ACB1809, 0xF2C834C1, + 0x7AB7077A, 0x12B42BB2, 0xAAB15EEA, 0xC2B27222, + 0xDF57C2AB, 0xB754EE63, 0x0F519B3B, 0x6752B7F3, + 0x349AFA29, 0x5C99D6E1, 0xE49CA3B9, 0x8C9F8F71, + 0x917A3FF8, 0xF9791330, 0x417C6668, 0x297F4AA0, + 0xE6ECFDDC, 0x8EEFD114, 0x36EAA44C, 0x5EE98884, + 0x430C380D, 0x2B0F14C5, 0x930A619D, 0xFB094D55, + 0xA8C1008F, 0xC0C22C47, 0x78C7591F, 0x10C475D7, + 0x0D21C55E, 0x6522E996, 0xDD279CCE, 0xB524B006, + 0x47EC84C7, 0x2FEFA80F, 0x97EADD57, 0xFFE9F19F, + 0xE20C4116, 0x8A0F6DDE, 0x320A1886, 0x5A09344E, + 0x09C17994, 0x61C2555C, 0xD9C72004, 0xB1C40CCC, + 0xAC21BC45, 0xC422908D, 0x7C27E5D5, 0x1424C91D, + 0xDBB77E61, 0xB3B452A9, 0x0BB127F1, 0x63B20B39, + 0x7E57BBB0, 0x16549778, 0xAE51E220, 0xC652CEE8, + 0x959A8332, 0xFD99AFFA, 0x459CDAA2, 0x2D9FF66A, + 0x307A46E3, 0x58796A2B, 0xE07C1F73, 0x887F33BB, + 0xF56E0EF4, 0x9D6D223C, 0x25685764, 0x4D6B7BAC, + 0x508ECB25, 0x388DE7ED, 0x808892B5, 0xE88BBE7D, + 0xBB43F3A7, 0xD340DF6F, 0x6B45AA37, 0x034686FF, + 0x1EA33676, 0x76A01ABE, 0xCEA56FE6, 0xA6A6432E, + 0x6935F452, 0x0136D89A, 0xB933ADC2, 0xD130810A, + 0xCCD53183, 0xA4D61D4B, 0x1CD36813, 0x74D044DB, + 0x27180901, 0x4F1B25C9, 0xF71E5091, 0x9F1D7C59, + 0x82F8CCD0, 0xEAFBE018, 0x52FE9540, 0x3AFDB988, + 0xC8358D49, 0xA036A181, 0x1833D4D9, 0x7030F811, + 0x6DD54898, 0x05D66450, 0xBDD31108, 0xD5D03DC0, + 0x8618701A, 0xEE1B5CD2, 0x561E298A, 0x3E1D0542, + 0x23F8B5CB, 0x4BFB9903, 0xF3FEEC5B, 0x9BFDC093, + 0x546E77EF, 0x3C6D5B27, 0x84682E7F, 0xEC6B02B7, + 0xF18EB23E, 0x998D9EF6, 0x2188EBAE, 0x498BC766, + 0x1A438ABC, 0x7240A674, 0xCA45D32C, 0xA246FFE4, + 0xBFA34F6D, 0xD7A063A5, 0x6FA516FD, 0x07A63A35, + 0x8FD9098E, 0xE7DA2546, 0x5FDF501E, 0x37DC7CD6, + 0x2A39CC5F, 0x423AE097, 0xFA3F95CF, 0x923CB907, + 0xC1F4F4DD, 0xA9F7D815, 0x11F2AD4D, 0x79F18185, + 0x6414310C, 0x0C171DC4, 0xB412689C, 0xDC114454, + 0x1382F328, 0x7B81DFE0, 0xC384AAB8, 0xAB878670, + 0xB66236F9, 0xDE611A31, 0x66646F69, 0x0E6743A1, + 0x5DAF0E7B, 0x35AC22B3, 0x8DA957EB, 0xE5AA7B23, + 0xF84FCBAA, 0x904CE762, 0x2849923A, 0x404ABEF2, + 0xB2828A33, 0xDA81A6FB, 0x6284D3A3, 0x0A87FF6B, + 0x17624FE2, 0x7F61632A, 0xC7641672, 0xAF673ABA, + 0xFCAF7760, 0x94AC5BA8, 0x2CA92EF0, 0x44AA0238, + 0x594FB2B1, 0x314C9E79, 0x8949EB21, 0xE14AC7E9, + 0x2ED97095, 0x46DA5C5D, 0xFEDF2905, 0x96DC05CD, + 0x8B39B544, 0xE33A998C, 0x5B3FECD4, 0x333CC01C, + 0x60F48DC6, 0x08F7A10E, 0xB0F2D456, 0xD8F1F89E, + 0xC5144817, 0xAD1764DF, 0x15121187, 0x7D113D4F, + /* T8_7 */ + 0x00000000, 0x493C7D27, 0x9278FA4E, 0xDB448769, + 0x211D826D, 0x6821FF4A, 0xB3657823, 0xFA590504, + 0x423B04DA, 0x0B0779FD, 0xD043FE94, 0x997F83B3, + 0x632686B7, 0x2A1AFB90, 0xF15E7CF9, 0xB86201DE, + 0x847609B4, 0xCD4A7493, 0x160EF3FA, 0x5F328EDD, + 0xA56B8BD9, 0xEC57F6FE, 0x37137197, 0x7E2F0CB0, + 0xC64D0D6E, 0x8F717049, 0x5435F720, 0x1D098A07, + 0xE7508F03, 0xAE6CF224, 0x7528754D, 0x3C14086A, + 0x0D006599, 0x443C18BE, 0x9F789FD7, 0xD644E2F0, + 0x2C1DE7F4, 0x65219AD3, 0xBE651DBA, 0xF759609D, + 0x4F3B6143, 0x06071C64, 0xDD439B0D, 0x947FE62A, + 0x6E26E32E, 0x271A9E09, 0xFC5E1960, 0xB5626447, + 0x89766C2D, 0xC04A110A, 0x1B0E9663, 0x5232EB44, + 0xA86BEE40, 0xE1579367, 0x3A13140E, 0x732F6929, + 0xCB4D68F7, 0x827115D0, 0x593592B9, 0x1009EF9E, + 0xEA50EA9A, 0xA36C97BD, 0x782810D4, 0x31146DF3, + 0x1A00CB32, 0x533CB615, 0x8878317C, 0xC1444C5B, + 0x3B1D495F, 0x72213478, 0xA965B311, 0xE059CE36, + 0x583BCFE8, 0x1107B2CF, 0xCA4335A6, 0x837F4881, + 0x79264D85, 0x301A30A2, 0xEB5EB7CB, 0xA262CAEC, + 0x9E76C286, 0xD74ABFA1, 0x0C0E38C8, 0x453245EF, + 0xBF6B40EB, 0xF6573DCC, 0x2D13BAA5, 0x642FC782, + 0xDC4DC65C, 0x9571BB7B, 0x4E353C12, 0x07094135, + 0xFD504431, 0xB46C3916, 0x6F28BE7F, 0x2614C358, + 0x1700AEAB, 0x5E3CD38C, 0x857854E5, 0xCC4429C2, + 0x361D2CC6, 0x7F2151E1, 0xA465D688, 0xED59ABAF, + 0x553BAA71, 0x1C07D756, 0xC743503F, 0x8E7F2D18, + 0x7426281C, 0x3D1A553B, 0xE65ED252, 0xAF62AF75, + 0x9376A71F, 0xDA4ADA38, 0x010E5D51, 0x48322076, + 0xB26B2572, 0xFB575855, 0x2013DF3C, 0x692FA21B, + 0xD14DA3C5, 0x9871DEE2, 0x4335598B, 0x0A0924AC, + 0xF05021A8, 0xB96C5C8F, 0x6228DBE6, 0x2B14A6C1, + 0x34019664, 0x7D3DEB43, 0xA6796C2A, 0xEF45110D, + 0x151C1409, 0x5C20692E, 0x8764EE47, 0xCE589360, + 0x763A92BE, 0x3F06EF99, 0xE44268F0, 0xAD7E15D7, + 0x572710D3, 0x1E1B6DF4, 0xC55FEA9D, 0x8C6397BA, + 0xB0779FD0, 0xF94BE2F7, 0x220F659E, 0x6B3318B9, + 0x916A1DBD, 0xD856609A, 0x0312E7F3, 0x4A2E9AD4, + 0xF24C9B0A, 0xBB70E62D, 0x60346144, 0x29081C63, + 0xD3511967, 0x9A6D6440, 0x4129E329, 0x08159E0E, + 0x3901F3FD, 0x703D8EDA, 0xAB7909B3, 0xE2457494, + 0x181C7190, 0x51200CB7, 0x8A648BDE, 0xC358F6F9, + 0x7B3AF727, 0x32068A00, 0xE9420D69, 0xA07E704E, + 0x5A27754A, 0x131B086D, 0xC85F8F04, 0x8163F223, + 0xBD77FA49, 0xF44B876E, 0x2F0F0007, 0x66337D20, + 0x9C6A7824, 0xD5560503, 0x0E12826A, 0x472EFF4D, + 0xFF4CFE93, 0xB67083B4, 0x6D3404DD, 0x240879FA, + 0xDE517CFE, 0x976D01D9, 0x4C2986B0, 0x0515FB97, + 0x2E015D56, 0x673D2071, 0xBC79A718, 0xF545DA3F, + 0x0F1CDF3B, 0x4620A21C, 0x9D642575, 0xD4585852, + 0x6C3A598C, 0x250624AB, 0xFE42A3C2, 0xB77EDEE5, + 0x4D27DBE1, 0x041BA6C6, 0xDF5F21AF, 0x96635C88, + 0xAA7754E2, 0xE34B29C5, 0x380FAEAC, 0x7133D38B, + 0x8B6AD68F, 0xC256ABA8, 0x19122CC1, 0x502E51E6, + 0xE84C5038, 0xA1702D1F, 0x7A34AA76, 0x3308D751, + 0xC951D255, 0x806DAF72, 0x5B29281B, 0x1215553C, + 0x230138CF, 0x6A3D45E8, 0xB179C281, 0xF845BFA6, + 0x021CBAA2, 0x4B20C785, 0x906440EC, 0xD9583DCB, + 0x613A3C15, 0x28064132, 0xF342C65B, 0xBA7EBB7C, + 0x4027BE78, 0x091BC35F, 0xD25F4436, 0x9B633911, + 0xA777317B, 0xEE4B4C5C, 0x350FCB35, 0x7C33B612, + 0x866AB316, 0xCF56CE31, 0x14124958, 0x5D2E347F, + 0xE54C35A1, 0xAC704886, 0x7734CFEF, 0x3E08B2C8, + 0xC451B7CC, 0x8D6DCAEB, 0x56294D82, 0x1F1530A5 + }; +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java index c3020844927c8..49efad05feb5a 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java @@ -18,22 +18,22 @@ package org.apache.hadoop.ozone.lock; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; /** * Lock implementation which also maintains counter. */ public final class ActiveLock { - private Lock lock; + private ReadWriteLock lock; private AtomicInteger count; /** * Use ActiveLock#newInstance to create instance. */ private ActiveLock() { - this.lock = new ReentrantLock(); + this.lock = new ReentrantReadWriteLock(); this.count = new AtomicInteger(0); } @@ -47,21 +47,58 @@ public static ActiveLock newInstance() { } /** - * Acquires the lock. + * Acquires read lock. * - *

If the lock is not available then the current thread becomes - * disabled for thread scheduling purposes and lies dormant until the - * lock has been acquired. + *

Acquires the read lock if the write lock is not held by + * another thread and returns immediately. + * + *

If the write lock is held by another thread then + * the current thread becomes disabled for thread scheduling + * purposes and lies dormant until the read lock has been acquired. + */ + void readLock() { + lock.readLock().lock(); + } + + /** + * Attempts to release the read lock. + * + *

If the number of readers is now zero then the lock + * is made available for write lock attempts. + */ + void readUnlock() { + lock.readLock().unlock(); + } + + /** + * Acquires write lock. + * + *

Acquires the write lock if neither the read nor write lock + * are held by another thread + * and returns immediately, setting the write lock hold count to + * one. + * + *

If the current thread already holds the write lock then the + * hold count is incremented by one and the method returns + * immediately. + * + *

If the lock is held by another thread then the current + * thread becomes disabled for thread scheduling purposes and + * lies dormant until the write lock has been acquired. */ - public void lock() { - lock.lock(); + void writeLock() { + lock.writeLock().lock(); } /** - * Releases the lock. + * Attempts to release the write lock. + * + *

If the current thread is the holder of this lock then + * the hold count is decremented. If the hold count is now + * zero then the lock is released. */ - public void unlock() { - lock.unlock(); + void writeUnlock() { + lock.writeLock().unlock(); } /** diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/LockManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/LockManager.java index 5f76bd6263ae9..670d4d16378bd 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/LockManager.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/LockManager.java @@ -25,42 +25,156 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; /** * Manages the locks on a given resource. A new lock is created for each * and every unique resource. Uniqueness of resource depends on the * {@code equals} implementation of it. */ -public class LockManager { +public class LockManager { private static final Logger LOG = LoggerFactory.getLogger(LockManager.class); - private final Map activeLocks = new ConcurrentHashMap<>(); + private final Map activeLocks = new ConcurrentHashMap<>(); private final GenericObjectPool lockPool = new GenericObjectPool<>(new PooledLockFactory()); /** - * Creates new LockManager instance. + * Creates new LockManager instance with the given Configuration. * * @param conf Configuration object */ - public LockManager(Configuration conf) { - int maxPoolSize = conf.getInt(HddsConfigKeys.HDDS_LOCK_MAX_CONCURRENCY, + public LockManager(final Configuration conf) { + final int maxPoolSize = conf.getInt( + HddsConfigKeys.HDDS_LOCK_MAX_CONCURRENCY, HddsConfigKeys.HDDS_LOCK_MAX_CONCURRENCY_DEFAULT); lockPool.setMaxTotal(maxPoolSize); } - /** * Acquires the lock on given resource. * *

If the lock is not available then the current thread becomes * disabled for thread scheduling purposes and lies dormant until the * lock has been acquired. + * + * @param resource on which the lock has to be acquired + * @deprecated Use {@link LockManager#writeLock} instead + */ + public void lock(final R resource) { + writeLock(resource); + } + + /** + * Releases the lock on given resource. + * + * @param resource for which the lock has to be released + * @deprecated Use {@link LockManager#writeUnlock} instead + */ + public void unlock(final R resource) { + writeUnlock(resource); + } + + /** + * Acquires the read lock on given resource. + * + *

Acquires the read lock on resource if the write lock is not held by + * another thread and returns immediately. + * + *

If the write lock on resource is held by another thread then + * the current thread becomes disabled for thread scheduling + * purposes and lies dormant until the read lock has been acquired. + * + * @param resource on which the read lock has to be acquired + */ + public void readLock(final R resource) { + acquire(resource, ActiveLock::readLock); + } + + /** + * Releases the read lock on given resource. + * + * @param resource for which the read lock has to be released + * @throws IllegalMonitorStateException if the current thread does not + * hold this lock + */ + public void readUnlock(final R resource) throws IllegalMonitorStateException { + release(resource, ActiveLock::readUnlock); + } + + /** + * Acquires the write lock on given resource. + * + *

Acquires the write lock on resource if neither the read nor write lock + * are held by another thread and returns immediately. + * + *

If the current thread already holds the write lock then the + * hold count is incremented by one and the method returns + * immediately. + * + *

If the lock is held by another thread then the current + * thread becomes disabled for thread scheduling purposes and + * lies dormant until the write lock has been acquired. + * + * @param resource on which the lock has to be acquired */ - public void lock(T resource) { - activeLocks.compute(resource, (k, v) -> { - ActiveLock lock; + public void writeLock(final R resource) { + acquire(resource, ActiveLock::writeLock); + } + + /** + * Releases the write lock on given resource. + * + * @param resource for which the lock has to be released + * @throws IllegalMonitorStateException if the current thread does not + * hold this lock + */ + public void writeUnlock(final R resource) + throws IllegalMonitorStateException { + release(resource, ActiveLock::writeUnlock); + } + + /** + * Acquires the lock on given resource using the provided lock function. + * + * @param resource on which the lock has to be acquired + * @param lockFn function to acquire the lock + */ + private void acquire(final R resource, final Consumer lockFn) { + lockFn.accept(getLockForLocking(resource)); + } + + /** + * Releases the lock on given resource using the provided release function. + * + * @param resource for which the lock has to be released + * @param releaseFn function to release the lock + */ + private void release(final R resource, final Consumer releaseFn) { + final ActiveLock lock = getLockForReleasing(resource); + releaseFn.accept(lock); + decrementActiveLockCount(resource); + } + + /** + * Returns {@link ActiveLock} instance for the given resource, + * on which the lock can be acquired. + * + * @param resource on which the lock has to be acquired + * @return {@link ActiveLock} instance + */ + private ActiveLock getLockForLocking(final R resource) { + /* + * While getting a lock object for locking we should + * atomically increment the active count of the lock. + * + * This is to avoid cases where the selected lock could + * be removed from the activeLocks map and returned to + * the object pool. + */ + return activeLocks.compute(resource, (k, v) -> { + final ActiveLock lock; try { if (v == null) { lock = lockPool.borrowObject(); @@ -73,22 +187,34 @@ public void lock(T resource) { throw new RuntimeException(ex); } return lock; - }).lock(); + }); } /** - * Releases the lock on given resource. + * Returns {@link ActiveLock} instance for the given resource, + * for which the lock has to be released. + * + * @param resource for which the lock has to be released + * @return {@link ActiveLock} instance */ - public void unlock(T resource) { - ActiveLock lock = activeLocks.get(resource); - if (lock == null) { - // Someone is releasing a lock which was never acquired. Log and return. - LOG.error("Trying to release the lock on {}, which was never acquired.", - resource); - throw new IllegalMonitorStateException("Releasing lock on resource " - + resource + " without acquiring lock"); + private ActiveLock getLockForReleasing(final R resource) { + if (activeLocks.containsKey(resource)) { + return activeLocks.get(resource); } - lock.unlock(); + // Someone is releasing a lock which was never acquired. + LOG.error("Trying to release the lock on {}, which was never acquired.", + resource); + throw new IllegalMonitorStateException("Releasing lock on resource " + + resource + " without acquiring lock"); + } + + /** + * Decrements the active lock count and returns the {@link ActiveLock} + * object to pool if the active count is 0. + * + * @param resource resource to which the ActiveLock is associated + */ + private void decrementActiveLockCount(final R resource) { activeLocks.computeIfPresent(resource, (k, v) -> { v.decrementActiveCount(); if (v.getActiveLockCount() != 0) { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/web/utils/JsonUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/web/utils/JsonUtils.java index af56da394cd48..4177b96a354c3 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/web/utils/JsonUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/web/utils/JsonUtils.java @@ -43,10 +43,9 @@ private JsonUtils() { // Never constructed } - public static String toJsonStringWithDefaultPrettyPrinter(String jsonString) + public static String toJsonStringWithDefaultPrettyPrinter(Object obj) throws IOException { - Object json = READER.readValue(jsonString); - return WRITTER.writeValueAsString(json); + return WRITTER.writeValueAsString(obj); } public static String toJsonString(Object obj) throws IOException { diff --git a/hadoop-hdds/common/src/main/proto/SCMSecurityProtocol.proto b/hadoop-hdds/common/src/main/proto/SCMSecurityProtocol.proto index 5b6dd27bf84fb..72e0e9f66f7d4 100644 --- a/hadoop-hdds/common/src/main/proto/SCMSecurityProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/SCMSecurityProtocol.proto @@ -30,17 +30,61 @@ option java_generic_services = true; option java_generate_equals_and_hash = true; -package hadoop.hdds; +package hadoop.hdds.security; import "hdds.proto"; +/** +All commands is send as request and all response come back via +Response class. If adding new functions please follow this protocol, since +our tracing and visibility tools depend on this pattern. +*/ +message SCMSecurityRequest { + required Type cmdType = 1; // Type of the command + + optional string traceID = 2; + + optional SCMGetDataNodeCertRequestProto getDataNodeCertRequest = 3; + optional SCMGetOMCertRequestProto getOMCertRequest = 4; + optional SCMGetCertificateRequestProto getCertificateRequest = 5; + optional SCMGetCACertificateRequestProto getCACertificateRequest = 6; + +} + +message SCMSecurityResponse { + required Type cmdType = 1; // Type of the command + + // A string that identifies this command, we generate Trace ID in Ozone + // frontend and this allows us to trace that command all over ozone. + optional string traceID = 2; + + optional bool success = 3 [default = true]; + + optional string message = 4; + + required Status status = 5; + + optional SCMGetCertResponseProto getCertResponseProto = 6; + +} + +enum Type { + GetDataNodeCertificate = 1; + GetOMCertificate = 2; + GetCertificate = 3; + GetCACertificate = 4; +} + +enum Status { + OK = 1; +} /** * This message is send by data node to prove its identity and get an SCM * signed certificate. */ message SCMGetDataNodeCertRequestProto { - required DatanodeDetailsProto datanodeDetails = 1; - required string CSR = 2; + required DatanodeDetailsProto datanodeDetails = 1; + required string CSR = 2; } /** @@ -48,15 +92,15 @@ message SCMGetDataNodeCertRequestProto { * signed certificate. */ message SCMGetOMCertRequestProto { - required OzoneManagerDetailsProto omDetails = 1; - required string CSR = 2; + required OzoneManagerDetailsProto omDetails = 1; + required string CSR = 2; } /** * Proto request to get a certificate with given serial id. */ message SCMGetCertificateRequestProto { - required string certSerialId = 1; + required string certSerialId = 1; } /** @@ -69,39 +113,17 @@ message SCMGetCACertificateRequestProto { * Returns a certificate signed by SCM. */ message SCMGetCertResponseProto { - enum ResponseCode { - success = 1; - authenticationFailed = 2; - invalidCSR = 3; - } - required ResponseCode responseCode = 1; - required string x509Certificate = 2; // Base64 encoded X509 certificate. - optional string x509CACertificate = 3; // Base64 encoded CA X509 certificate. + enum ResponseCode { + success = 1; + authenticationFailed = 2; + invalidCSR = 3; + } + required ResponseCode responseCode = 1; + required string x509Certificate = 2; // Base64 encoded X509 certificate. + optional string x509CACertificate = 3; // Base64 encoded CA X509 certificate. } service SCMSecurityProtocolService { - /** - * Get SCM signed certificate for DataNode. - */ - rpc getDataNodeCertificate (SCMGetDataNodeCertRequestProto) returns - (SCMGetCertResponseProto); - - /** - * Get SCM signed certificate for DataNode. - */ - rpc getOMCertificate (SCMGetOMCertRequestProto) returns - (SCMGetCertResponseProto); - - /** - * Get SCM signed certificate for DataNode. - */ - rpc getCertificate (SCMGetCertificateRequestProto) returns - (SCMGetCertResponseProto); - - /** - * Get SCM signed certificate for DataNode. - */ - rpc getCACertificate (SCMGetCACertificateRequestProto) returns - (SCMGetCertResponseProto); + rpc submitRequest (SCMSecurityRequest) returns (SCMSecurityResponse); } diff --git a/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto index ded0d027f6bff..fc7a5988ce669 100644 --- a/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto @@ -26,7 +26,7 @@ option java_package = "org.apache.hadoop.hdds.protocol.proto"; option java_outer_classname = "ScmBlockLocationProtocolProtos"; option java_generic_services = true; option java_generate_equals_and_hash = true; -package hadoop.hdds; +package hadoop.hdds.block; import "hdds.proto"; diff --git a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto index 0c358760360d1..8ea72b6cd1780 100644 --- a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto +++ b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto @@ -26,10 +26,100 @@ option java_package = "org.apache.hadoop.hdds.protocol.proto"; option java_outer_classname = "StorageContainerLocationProtocolProtos"; option java_generic_services = true; option java_generate_equals_and_hash = true; -package hadoop.hdds; +package hadoop.hdds.container; import "hdds.proto"; +/** + All functions are dispatched as Request/Response under Ozone. + if you add newe functions, please add them using same pattern. +*/ +message ScmContainerLocationRequest { + required Type cmdType = 1; // Type of the command + + // A string that identifies this command, we generate Trace ID in Ozone + // frontend and this allows us to trace that command all over ozone. + optional string traceID = 2; + + optional ContainerRequestProto containerRequest = 6; + optional GetContainerRequestProto getContainerRequest = 7; + optional GetContainerWithPipelineRequestProto getContainerWithPipelineRequest = 8; + optional SCMListContainerRequestProto scmListContainerRequest = 9; + optional SCMDeleteContainerRequestProto scmDeleteContainerRequest = 10; + optional NodeQueryRequestProto nodeQueryRequest = 11; + optional ObjectStageChangeRequestProto objectStageChangeRequest = 12; + optional PipelineRequestProto pipelineRequest = 13; + optional ListPipelineRequestProto listPipelineRequest = 14; + optional ActivatePipelineRequestProto activatePipelineRequest = 15; + optional DeactivatePipelineRequestProto deactivatePipelineRequest = 16; + optional ClosePipelineRequestProto closePipelineRequest = 17; + optional GetScmInfoRequestProto getScmInfoRequest = 18; + optional InSafeModeRequestProto inSafeModeRequest = 19; + optional ForceExitSafeModeRequestProto forceExitSafeModeRequest = 20; + optional StartReplicationManagerRequestProto startReplicationManagerRequest = 21; + optional StopReplicationManagerRequestProto stopReplicationManagerRequest = 22; + optional ReplicationManagerStatusRequestProto seplicationManagerStatusRequest = 23; + +} + +message ScmContainerLocationResponse { + required Type cmdType = 1; // Type of the command + + optional string traceID = 2; + + optional bool success = 3 [default = true]; + + optional string message = 4; + + required Status status = 5; + + optional ContainerResponseProto containerResponse = 6; + optional GetContainerResponseProto getContainerResponse = 7; + optional GetContainerWithPipelineResponseProto getContainerWithPipelineResponse = 8; + optional SCMListContainerResponseProto scmListContainerResponse = 9; + optional SCMDeleteContainerResponseProto scmDeleteContainerResponse = 10; + optional NodeQueryResponseProto nodeQueryResponse = 11; + optional ObjectStageChangeResponseProto objectStageChangeResponse = 12; + optional PipelineResponseProto pipelineResponse = 13; + optional ListPipelineResponseProto listPipelineResponse = 14; + optional ActivatePipelineResponseProto activatePipelineResponse = 15; + optional DeactivatePipelineResponseProto deactivatePipelineResponse = 16; + optional ClosePipelineResponseProto closePipelineResponse = 17; + optional GetScmInfoResponseProto getScmInfoResponse = 18; + optional InSafeModeResponseProto inSafeModeResponse = 19; + optional ForceExitSafeModeResponseProto forceExitSafeModeResponse = 20; + optional StartReplicationManagerResponseProto startReplicationManagerResponse = 21; + optional StopReplicationManagerResponseProto stopReplicationManagerResponse = 22; + optional ReplicationManagerStatusResponseProto replicationManagerStatusResponse = 23; + enum Status { + OK = 1; + CONTAINER_ALREADY_EXISTS = 2; + CONTAINER_IS_MISSING = 3; + } +} + +enum Type { + + AllocateContainer = 1; + GetContainer = 2; + GetContainerWithPipeline = 3; + ListContainer = 4; + DeleteContainer = 5; + QueryNode = 6; + NotifyObjectStageChange = 7; + AllocatePipeline = 8; + ListPipelines = 9; + ActivatePipeline = 10; + DeactivatePipeline = 11; + ClosePipeline = 12; + GetScmInfo = 13; + InSafeMode = 14; + ForceExitSafeMode = 15; + StartReplicationManager = 16; + StopReplicationManager = 17; + GetReplicationManagerStatus = 18; +} + /** * Request send to SCM asking where the container should be created. */ @@ -235,97 +325,6 @@ message ReplicationManagerStatusResponseProto { * and response messages for details of the RPC calls. */ service StorageContainerLocationProtocolService { + rpc submitRequest (ScmContainerLocationRequest) returns (ScmContainerLocationResponse); - /** - * Creates a container entry in SCM. - */ - rpc allocateContainer(ContainerRequestProto) returns (ContainerResponseProto); - - /** - * Returns the pipeline for a given container. - */ - rpc getContainer(GetContainerRequestProto) returns (GetContainerResponseProto); - - /** - * Returns the pipeline for a given container. - */ - rpc getContainerWithPipeline(GetContainerWithPipelineRequestProto) returns (GetContainerWithPipelineResponseProto); - - rpc listContainer(SCMListContainerRequestProto) returns (SCMListContainerResponseProto); - - /** - * Deletes a container in SCM. - */ - rpc deleteContainer(SCMDeleteContainerRequestProto) returns (SCMDeleteContainerResponseProto); - - /** - * Returns a set of Nodes that meet a criteria. - */ - rpc queryNode(NodeQueryRequestProto) returns (NodeQueryResponseProto); - - /** - * Notify from client when begin or finish container or pipeline operations on datanodes. - */ - rpc notifyObjectStageChange(ObjectStageChangeRequestProto) returns (ObjectStageChangeResponseProto); - - /* - * Apis that Manage Pipelines. - * - * Pipelines are abstractions offered by SCM and Datanode that allows users - * to create a replication pipeline. - * - * These following APIs allow command line programs like SCM CLI to list - * and manage pipelines. - */ - - /** - * Creates a replication pipeline. - */ - rpc allocatePipeline(PipelineRequestProto) - returns (PipelineResponseProto); - - /** - * Returns the list of Pipelines managed by SCM. - */ - rpc listPipelines(ListPipelineRequestProto) - returns (ListPipelineResponseProto); - - rpc activatePipeline(ActivatePipelineRequestProto) - returns (ActivatePipelineResponseProto); - - rpc deactivatePipeline(DeactivatePipelineRequestProto) - returns (DeactivatePipelineResponseProto); - - /** - * Closes a pipeline. - */ - rpc closePipeline(ClosePipelineRequestProto) - returns (ClosePipelineResponseProto); - - /** - * Returns information about SCM. - */ - rpc getScmInfo(GetScmInfoRequestProto) - returns (GetScmInfoResponseProto); - - /** - * Checks if SCM is in SafeMode. - */ - rpc inSafeMode(InSafeModeRequestProto) - returns (InSafeModeResponseProto); - - /** - * Returns information about SCM. - */ - rpc forceExitSafeMode(ForceExitSafeModeRequestProto) - returns (ForceExitSafeModeResponseProto); - - rpc startReplicationManager(StartReplicationManagerRequestProto) - returns (StartReplicationManagerResponseProto); - - rpc stopReplicationManager(StopReplicationManagerRequestProto) - returns (StopReplicationManagerResponseProto); - - rpc getReplicationManagerStatus(ReplicationManagerStatusRequestProto) - returns (ReplicationManagerStatusResponseProto); } diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java new file mode 100644 index 0000000000000..bbe6ab7cca7a3 --- /dev/null +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdds.ratis; + +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; +import org.apache.hadoop.ozone.common.Checksum; +import org.apache.hadoop.ozone.common.ChecksumData; +import org.apache.hadoop.ozone.common.OzoneChecksumException; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Random; +import java.util.UUID; +import java.util.function.BiFunction; + +/** Testing {@link ContainerCommandRequestMessage}. */ +public class TestContainerCommandRequestMessage { + static final Random RANDOM = new Random(); + + static ByteString newData(int length, Random random) { + final ByteString.Output out = ByteString.newOutput(); + for(int i = 0; i < length; i++) { + out.write(random.nextInt()); + } + return out.toByteString(); + } + + static ChecksumData checksum(ByteString data) { + try { + return new Checksum().computeChecksum(data.toByteArray()); + } catch (OzoneChecksumException e) { + throw new IllegalStateException(e); + } + } + + static ContainerCommandRequestProto newPutSmallFile( + BlockID blockID, ByteString data) { + final BlockData.Builder blockData + = BlockData.newBuilder() + .setBlockID(blockID.getDatanodeBlockIDProtobuf()); + final PutBlockRequestProto.Builder putBlockRequest + = PutBlockRequestProto.newBuilder() + .setBlockData(blockData); + final KeyValue keyValue = KeyValue.newBuilder() + .setKey("OverWriteRequested") + .setValue("true") + .build(); + final ChunkInfo chunk = ChunkInfo.newBuilder() + .setChunkName(blockID.getLocalID() + "_chunk") + .setOffset(0) + .setLen(data.size()) + .addMetadata(keyValue) + .setChecksumData(checksum(data).getProtoBufMessage()) + .build(); + final PutSmallFileRequestProto putSmallFileRequest + = PutSmallFileRequestProto.newBuilder() + .setChunkInfo(chunk) + .setBlock(putBlockRequest) + .setData(data) + .build(); + return ContainerCommandRequestProto.newBuilder() + .setCmdType(Type.PutSmallFile) + .setContainerID(blockID.getContainerID()) + .setDatanodeUuid(UUID.randomUUID().toString()) + .setPutSmallFile(putSmallFileRequest) + .build(); + } + + static ContainerCommandRequestProto newWriteChunk( + BlockID blockID, ByteString data) { + final ChunkInfo chunk = ChunkInfo.newBuilder() + .setChunkName(blockID.getLocalID() + "_chunk_" + 1) + .setOffset(0) + .setLen(data.size()) + .setChecksumData(checksum(data).getProtoBufMessage()) + .build(); + + final WriteChunkRequestProto.Builder writeChunkRequest + = WriteChunkRequestProto.newBuilder() + .setBlockID(blockID.getDatanodeBlockIDProtobuf()) + .setChunkData(chunk) + .setData(data); + return ContainerCommandRequestProto.newBuilder() + .setCmdType(Type.WriteChunk) + .setContainerID(blockID.getContainerID()) + .setDatanodeUuid(UUID.randomUUID().toString()) + .setWriteChunk(writeChunkRequest) + .build(); + } + + @Test + public void testPutSmallFile() throws Exception { + runTest(TestContainerCommandRequestMessage::newPutSmallFile); + } + + @Test + public void testWriteChunk() throws Exception { + runTest(TestContainerCommandRequestMessage::newWriteChunk); + } + + static void runTest( + BiFunction method) + throws Exception { + for(int i = 0; i < 2; i++) { + runTest(i, method); + } + for(int i = 2; i < 1 << 10;) { + runTest(i + 1 + RANDOM.nextInt(i - 1), method); + i <<= 1; + runTest(i, method); + } + } + + static void runTest(int length, + BiFunction method) + throws Exception { + System.out.println("length=" + length); + final BlockID blockID = new BlockID(RANDOM.nextLong(), RANDOM.nextLong()); + final ByteString data = newData(length, RANDOM); + + final ContainerCommandRequestProto original = method.apply(blockID, data); + final ContainerCommandRequestMessage message + = ContainerCommandRequestMessage.toMessage(original, null); + final ContainerCommandRequestProto computed + = ContainerCommandRequestMessage.toProto(message.getContent(), null); + Assert.assertEquals(original, computed); + } +} diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/db/TestDBStoreBuilder.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/db/TestDBStoreBuilder.java index 1ea6110587c15..d406060165f32 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/db/TestDBStoreBuilder.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/db/TestDBStoreBuilder.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hdds.utils.db; import org.apache.commons.lang3.RandomStringUtils; -import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -50,14 +50,14 @@ public void setUp() throws Exception { @Test public void builderWithoutAnyParams() throws IOException { - Configuration conf = new Configuration(); + OzoneConfiguration conf = new OzoneConfiguration(); thrown.expect(IOException.class); DBStoreBuilder.newBuilder(conf).build(); } @Test public void builderWithOneParamV1() throws IOException { - Configuration conf = new Configuration(); + OzoneConfiguration conf = new OzoneConfiguration(); thrown.expect(IOException.class); DBStoreBuilder.newBuilder(conf) .setName("Test.db") @@ -66,7 +66,7 @@ public void builderWithOneParamV1() throws IOException { @Test public void builderWithOneParamV2() throws IOException { - Configuration conf = new Configuration(); + OzoneConfiguration conf = new OzoneConfiguration(); File newFolder = folder.newFolder(); if(!newFolder.exists()) { Assert.assertTrue(newFolder.mkdirs()); @@ -79,7 +79,7 @@ public void builderWithOneParamV2() throws IOException { @Test public void builderWithOpenClose() throws Exception { - Configuration conf = new Configuration(); + OzoneConfiguration conf = new OzoneConfiguration(); File newFolder = folder.newFolder(); if(!newFolder.exists()) { Assert.assertTrue(newFolder.mkdirs()); @@ -94,7 +94,7 @@ public void builderWithOpenClose() throws Exception { @Test public void builderWithDoubleTableName() throws Exception { - Configuration conf = new Configuration(); + OzoneConfiguration conf = new OzoneConfiguration(); File newFolder = folder.newFolder(); if(!newFolder.exists()) { Assert.assertTrue(newFolder.mkdirs()); @@ -112,7 +112,7 @@ public void builderWithDoubleTableName() throws Exception { @Test public void builderWithDataWrites() throws Exception { - Configuration conf = new Configuration(); + OzoneConfiguration conf = new OzoneConfiguration(); File newFolder = folder.newFolder(); if(!newFolder.exists()) { Assert.assertTrue(newFolder.mkdirs()); @@ -141,7 +141,7 @@ public void builderWithDataWrites() throws Exception { @Test public void builderWithDiskProfileWrites() throws Exception { - Configuration conf = new Configuration(); + OzoneConfiguration conf = new OzoneConfiguration(); File newFolder = folder.newFolder(); if(!newFolder.exists()) { Assert.assertTrue(newFolder.mkdirs()); diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumByteBuffer.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumByteBuffer.java new file mode 100644 index 0000000000000..2f466377b4b2c --- /dev/null +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/common/TestChecksumByteBuffer.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.ozone.common; + +import org.apache.hadoop.util.PureJavaCrc32; +import org.apache.hadoop.util.PureJavaCrc32C; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.Random; +import java.util.zip.Checksum; + +/** + * Test {@link ChecksumByteBuffer} implementations. + */ +public class TestChecksumByteBuffer { + @Test + public void testPureJavaCrc32ByteBuffer() { + final Checksum expected = new PureJavaCrc32(); + final ChecksumByteBuffer testee = new PureJavaCrc32ByteBuffer(); + new VerifyChecksumByteBuffer(expected, testee).testCorrectness(); + } + + @Test + public void testPureJavaCrc32CByteBuffer() { + final Checksum expected = new PureJavaCrc32C(); + final ChecksumByteBuffer testee = new PureJavaCrc32CByteBuffer(); + new VerifyChecksumByteBuffer(expected, testee).testCorrectness(); + } + + static class VerifyChecksumByteBuffer { + private final Checksum expected; + private final ChecksumByteBuffer testee; + + VerifyChecksumByteBuffer(Checksum expected, ChecksumByteBuffer testee) { + this.expected = expected; + this.testee = testee; + } + + void testCorrectness() { + checkSame(); + + checkBytes("hello world!".getBytes(StandardCharsets.UTF_8)); + + final Random random = new Random(); + final byte[] bytes = new byte[1 << 10]; + for (int i = 0; i < 1000; i++) { + random.nextBytes(bytes); + checkBytes(bytes, random.nextInt(bytes.length)); + } + } + + void checkBytes(byte[] bytes) { + checkBytes(bytes, bytes.length); + } + + void checkBytes(byte[] bytes, int length) { + expected.reset(); + testee.reset(); + checkSame(); + + for (byte b : bytes) { + expected.update(b); + testee.update(b); + checkSame(); + } + + expected.reset(); + testee.reset(); + + for (int i = 0; i < length; i++) { + expected.update(bytes, 0, i); + testee.update(bytes, 0, i); + checkSame(); + } + + expected.reset(); + testee.reset(); + checkSame(); + } + + private void checkSame() { + Assert.assertEquals(expected.getValue(), testee.getValue()); + } + } +} diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lock/TestLockManager.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lock/TestLockManager.java index fa3030d0c3519..e88b1bb121b62 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lock/TestLockManager.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lock/TestLockManager.java @@ -29,34 +29,143 @@ public class TestLockManager { @Test(timeout = 1000) - public void testWithDifferentResource() { - LockManager manager = new LockManager<>(new OzoneConfiguration()); - manager.lock("/resourceOne"); + public void testWriteLockWithDifferentResource() { + final LockManager manager = + new LockManager<>(new OzoneConfiguration()); + manager.writeLock("/resourceOne"); // This should work, as they are different resource. - manager.lock("/resourceTwo"); - manager.unlock("/resourceOne"); - manager.unlock("/resourceTwo"); + manager.writeLock("/resourceTwo"); + manager.writeUnlock("/resourceOne"); + manager.writeUnlock("/resourceTwo"); Assert.assertTrue(true); } @Test - public void testWithSameResource() throws Exception { - LockManager manager = new LockManager<>(new OzoneConfiguration()); - manager.lock("/resourceOne"); - AtomicBoolean gotLock = new AtomicBoolean(false); + public void testWriteLockWithSameResource() throws Exception { + final LockManager manager = + new LockManager<>(new OzoneConfiguration()); + final AtomicBoolean gotLock = new AtomicBoolean(false); + manager.writeLock("/resourceOne"); new Thread(() -> { - manager.lock("/resourceOne"); + manager.writeLock("/resourceOne"); gotLock.set(true); - manager.unlock("/resourceOne"); + manager.writeUnlock("/resourceOne"); }).start(); - // Let's give some time for the new thread to run + // Let's give some time for the other thread to run Thread.sleep(100); - // Since the new thread is trying to get lock on same object, it will wait. + // Since the other thread is trying to get write lock on same object, + // it will wait. Assert.assertFalse(gotLock.get()); - manager.unlock("/resourceOne"); - // Since we have released the lock, the new thread should have the lock - // now - // Let's give some time for the new thread to run + manager.writeUnlock("/resourceOne"); + // Since we have released the write lock, the other thread should have + // the lock now + // Let's give some time for the other thread to run + Thread.sleep(100); + Assert.assertTrue(gotLock.get()); + } + + @Test(timeout = 1000) + public void testReadLockWithDifferentResource() { + final LockManager manager = + new LockManager<>(new OzoneConfiguration()); + manager.readLock("/resourceOne"); + manager.readLock("/resourceTwo"); + manager.readUnlock("/resourceOne"); + manager.readUnlock("/resourceTwo"); + Assert.assertTrue(true); + } + + @Test + public void testReadLockWithSameResource() throws Exception { + final LockManager manager = + new LockManager<>(new OzoneConfiguration()); + final AtomicBoolean gotLock = new AtomicBoolean(false); + manager.readLock("/resourceOne"); + new Thread(() -> { + manager.readLock("/resourceOne"); + gotLock.set(true); + manager.readUnlock("/resourceOne"); + }).start(); + // Let's give some time for the other thread to run + Thread.sleep(100); + // Since the new thread is trying to get read lock, it should work. + Assert.assertTrue(gotLock.get()); + manager.readUnlock("/resourceOne"); + } + + @Test + public void testWriteReadLockWithSameResource() throws Exception { + final LockManager manager = + new LockManager<>(new OzoneConfiguration()); + final AtomicBoolean gotLock = new AtomicBoolean(false); + manager.writeLock("/resourceOne"); + new Thread(() -> { + manager.readLock("/resourceOne"); + gotLock.set(true); + manager.readUnlock("/resourceOne"); + }).start(); + // Let's give some time for the other thread to run + Thread.sleep(100); + // Since the other thread is trying to get read lock on same object, + // it will wait. + Assert.assertFalse(gotLock.get()); + manager.writeUnlock("/resourceOne"); + // Since we have released the write lock, the other thread should have + // the lock now + // Let's give some time for the other thread to run + Thread.sleep(100); + Assert.assertTrue(gotLock.get()); + } + + @Test + public void testReadWriteLockWithSameResource() throws Exception { + final LockManager manager = + new LockManager<>(new OzoneConfiguration()); + final AtomicBoolean gotLock = new AtomicBoolean(false); + manager.readLock("/resourceOne"); + new Thread(() -> { + manager.writeLock("/resourceOne"); + gotLock.set(true); + manager.writeUnlock("/resourceOne"); + }).start(); + // Let's give some time for the other thread to run + Thread.sleep(100); + // Since the other thread is trying to get write lock on same object, + // it will wait. + Assert.assertFalse(gotLock.get()); + manager.readUnlock("/resourceOne"); + // Since we have released the read lock, the other thread should have + // the lock now + // Let's give some time for the other thread to run + Thread.sleep(100); + Assert.assertTrue(gotLock.get()); + } + + @Test + public void testMultiReadWriteLockWithSameResource() throws Exception { + final LockManager manager = + new LockManager<>(new OzoneConfiguration()); + final AtomicBoolean gotLock = new AtomicBoolean(false); + manager.readLock("/resourceOne"); + manager.readLock("/resourceOne"); + new Thread(() -> { + manager.writeLock("/resourceOne"); + gotLock.set(true); + manager.writeUnlock("/resourceOne"); + }).start(); + // Let's give some time for the other thread to run + Thread.sleep(100); + // Since the other thread is trying to get write lock on same object, + // it will wait. + Assert.assertFalse(gotLock.get()); + manager.readUnlock("/resourceOne"); + //We have only released one read lock, we still hold another read lock. + Thread.sleep(100); + Assert.assertFalse(gotLock.get()); + manager.readUnlock("/resourceOne"); + // Since we have released the read lock, the other thread should have + // the lock now + // Let's give some time for the other thread to run Thread.sleep(100); Assert.assertTrue(gotLock.get()); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java index 0535763537cc3..489a640d8bb37 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; @@ -150,7 +151,7 @@ public class ContainerStateMachine extends BaseStateMachine { private final Cache stateMachineDataCache; private final boolean isBlockTokenEnabled; private final TokenVerifier tokenVerifier; - private final AtomicBoolean isStateMachineHealthy; + private final AtomicBoolean stateMachineHealthy; private final Semaphore applyTransactionSemaphore; /** @@ -190,7 +191,7 @@ public ContainerStateMachine(RaftGroupId gid, ContainerDispatcher dispatcher, ScmConfigKeys. DFS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS_DEFAULT); applyTransactionSemaphore = new Semaphore(maxPendingApplyTransactions); - isStateMachineHealthy = new AtomicBoolean(true); + stateMachineHealthy = new AtomicBoolean(true); this.executors = new ExecutorService[numContainerOpExecutors]; for (int i = 0; i < numContainerOpExecutors; i++) { final int index = i; @@ -271,11 +272,15 @@ public void persistContainerSet(OutputStream out) throws IOException { IOUtils.write(builder.build().toByteArray(), out); } + public boolean isStateMachineHealthy() { + return stateMachineHealthy.get(); + } + @Override public long takeSnapshot() throws IOException { TermIndex ti = getLastAppliedTermIndex(); long startTime = Time.monotonicNow(); - if (!isStateMachineHealthy.get()) { + if (!isStateMachineHealthy()) { String msg = "Failed to take snapshot " + " for " + gid + " as the stateMachine" + " is unhealthy. The last applied index is at " + ti; @@ -309,7 +314,7 @@ public TransactionContext startTransaction(RaftClientRequest request) throws IOException { long startTime = Time.monotonicNowNanos(); final ContainerCommandRequestProto proto = - getContainerCommandRequestProto(request.getMessage().getContent()); + message2ContainerCommandRequestProto(request.getMessage()); Preconditions.checkArgument(request.getRaftGroupId().equals(gid)); try { dispatcher.validateContainerCommand(proto); @@ -359,7 +364,7 @@ public TransactionContext startTransaction(RaftClientRequest request) .setStateMachine(this) .setServerRole(RaftPeerRole.LEADER) .setStateMachineContext(startTime) - .setLogData(request.getMessage().getContent()) + .setLogData(proto.toByteString()) .build(); } @@ -379,6 +384,11 @@ private ContainerCommandRequestProto getContainerCommandRequestProto( .setPipelineID(gid.getUuid().toString()).build(); } + private ContainerCommandRequestProto message2ContainerCommandRequestProto( + Message message) throws InvalidProtocolBufferException { + return ContainerCommandRequestMessage.toProto(message.getContent(), gid); + } + private ContainerCommandResponseProto dispatchCommand( ContainerCommandRequestProto requestProto, DispatcherContext context) { LOG.trace("{}: dispatch {} containerID={} pipelineID={} traceID={}", gid, @@ -526,7 +536,7 @@ public CompletableFuture query(Message request) { try { metrics.incNumQueryStateMachineOps(); final ContainerCommandRequestProto requestProto = - getContainerCommandRequestProto(request.getContent()); + message2ContainerCommandRequestProto(request); return CompletableFuture .completedFuture(runCommand(requestProto, null)::toByteString); } catch (IOException e) { @@ -731,7 +741,11 @@ public CompletableFuture applyTransaction(TransactionContext trx) { metrics.incPipelineLatency(cmdType, Time.monotonicNowNanos() - startTime); } - if (r.getResult() != ContainerProtos.Result.SUCCESS) { + // ignore close container exception while marking the stateMachine + // unhealthy + if (r.getResult() != ContainerProtos.Result.SUCCESS + && r.getResult() != ContainerProtos.Result.CONTAINER_NOT_OPEN + && r.getResult() != ContainerProtos.Result.CLOSED_CONTAINER_IO) { StorageContainerException sce = new StorageContainerException(r.getMessage(), r.getResult()); LOG.error( @@ -744,7 +758,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) { // caught in stateMachineUpdater in Ratis and ratis server will // shutdown. applyTransactionFuture.completeExceptionally(sce); - isStateMachineHealthy.compareAndSet(true, false); + stateMachineHealthy.compareAndSet(true, false); ratisServer.handleApplyTransactionFailure(gid, trx.getServerRole()); } else { LOG.debug( @@ -759,7 +773,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) { // add the entry to the applyTransactionCompletionMap only if the // stateMachine is healthy i.e, there has been no applyTransaction // failures before. - if (isStateMachineHealthy.get()) { + if (isStateMachineHealthy()) { final Long previous = applyTransactionCompletionMap .put(index, trx.getLogEntry().getTerm()); Preconditions.checkState(previous == null); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java index 179547b8444f2..80e91cdf55de0 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ClosePipelineInfo; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineAction; +import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage; import org.apache.hadoop.hdds.scm.HddsServerUtil; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.security.x509.SecurityConfig; @@ -516,8 +517,8 @@ private RaftClientRequest createRaftClientRequest( RaftClientRequest.Type type) { return new RaftClientRequest(clientId, server.getId(), RaftGroupId.valueOf(PipelineID.getFromProtobuf(pipelineID).getId()), - nextCallId(), Message.valueOf(request.toByteString()), type, - null); + nextCallId(), ContainerCommandRequestMessage.toMessage(request, null), + type, null); } private GroupInfoRequest createGroupInfoRequest( diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java index a043cdce2e0f3..747bc3eb77470 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java @@ -80,7 +80,7 @@ public static void writeData(File chunkFile, ChunkInfo chunkInfo, ByteBuffer data, VolumeIOStats volumeIOStats, boolean sync) throws StorageContainerException, ExecutionException, InterruptedException, NoSuchAlgorithmException { - int bufferSize = data.capacity(); + final int bufferSize = data.remaining(); Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class); if (bufferSize != chunkInfo.getLen()) { String err = String.format("data array does not match the length " + diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java index 4e1e27e180172..9b446666e5d11 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java @@ -24,6 +24,9 @@ import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.ContainerReportsProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMDatanodeRequest; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMDatanodeRequest.Builder; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMDatanodeResponse; import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto; import org.apache.hadoop.hdds.protocol.proto @@ -38,6 +41,7 @@ .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; import org.apache.hadoop.hdds.protocol.proto .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.Type; import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtocolTranslator; import org.apache.hadoop.ipc.RPC; @@ -45,6 +49,7 @@ import java.io.Closeable; import java.io.IOException; +import java.util.function.Consumer; /** * This class is the client-side translator to translate the requests made on @@ -96,6 +101,25 @@ public Object getUnderlyingProxyObject() { return rpcProxy; } + /** + * Helper method to wrap the request and send the message. + */ + private SCMDatanodeResponse submitRequest(Type type, + Consumer builderConsumer) throws IOException { + final SCMDatanodeResponse response; + try { + Builder builder = SCMDatanodeRequest.newBuilder() + .setCmdType(type); + builderConsumer.accept(builder); + SCMDatanodeRequest wrapper = builder.build(); + + response = rpcProxy.submitRequest(NULL_RPC_CONTROLLER, wrapper); + } catch (ServiceException ex) { + throw ProtobufHelper.getRemoteException(ex); + } + return response; + } + /** * Returns SCM version. * @@ -104,16 +128,11 @@ public Object getUnderlyingProxyObject() { */ @Override public SCMVersionResponseProto getVersion(SCMVersionRequestProto - unused) throws IOException { - SCMVersionRequestProto request = - SCMVersionRequestProto.newBuilder().build(); - final SCMVersionResponseProto response; - try { - response = rpcProxy.getVersion(NULL_RPC_CONTROLLER, request); - } catch (ServiceException ex) { - throw ProtobufHelper.getRemoteException(ex); - } - return response; + request) throws IOException { + return submitRequest(Type.GetVersion, + (builder) -> builder + .setGetVersionRequest(SCMVersionRequestProto.newBuilder().build())) + .getGetVersionResponse(); } /** @@ -126,13 +145,9 @@ public SCMVersionResponseProto getVersion(SCMVersionRequestProto @Override public SCMHeartbeatResponseProto sendHeartbeat( SCMHeartbeatRequestProto heartbeat) throws IOException { - final SCMHeartbeatResponseProto resp; - try { - resp = rpcProxy.sendHeartbeat(NULL_RPC_CONTROLLER, heartbeat); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } - return resp; + return submitRequest(Type.SendHeartbeat, + (builder) -> builder.setSendHeartbeatRequest(heartbeat)) + .getSendHeartbeatResponse(); } /** @@ -155,13 +170,8 @@ public SCMRegisteredResponseProto register( req.setContainerReport(containerReportsRequestProto); req.setPipelineReports(pipelineReportsProto); req.setNodeReport(nodeReport); - final SCMRegisteredResponseProto response; - try { - response = rpcProxy.register(NULL_RPC_CONTROLLER, req.build()); - } catch (ServiceException e) { - throw ProtobufHelper.getRemoteException(e); - } - return response; + return submitRequest(Type.Register, + (builder) -> builder.setRegisterRequest(req)) + .getRegisterResponse(); } - } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java index 862233276aa2e..ed704ebf4310c 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java @@ -16,29 +16,24 @@ */ package org.apache.hadoop.ozone.protocolPB; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.PipelineReportsProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.NodeReportProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMRegisterRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.ContainerReportsProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto; +import java.io.IOException; + +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMDatanodeRequest; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMDatanodeResponse; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMRegisterRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.Status; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.Type; +import org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher; import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol; -import java.io.IOException; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class is the server-side translator that forwards requests received on @@ -48,47 +43,71 @@ public class StorageContainerDatanodeProtocolServerSideTranslatorPB implements StorageContainerDatanodeProtocolPB { + private static final Logger LOG = LoggerFactory + .getLogger(StorageContainerDatanodeProtocolServerSideTranslatorPB.class); + private final StorageContainerDatanodeProtocol impl; + private final OzoneProtocolMessageDispatcher dispatcher; public StorageContainerDatanodeProtocolServerSideTranslatorPB( - StorageContainerDatanodeProtocol impl) { + StorageContainerDatanodeProtocol impl, + ProtocolMessageMetrics protocolMessageMetrics) { this.impl = impl; + dispatcher = + new OzoneProtocolMessageDispatcher<>("SCMDatanodeProtocol", + protocolMessageMetrics, + LOG); } - @Override - public SCMVersionResponseProto getVersion(RpcController controller, - SCMVersionRequestProto request) - throws ServiceException { - try { - return impl.getVersion(request); - } catch (IOException e) { - throw new ServiceException(e); - } + public SCMRegisteredResponseProto register( + SCMRegisterRequestProto request) throws IOException { + ContainerReportsProto containerRequestProto = request + .getContainerReport(); + NodeReportProto dnNodeReport = request.getNodeReport(); + PipelineReportsProto pipelineReport = request.getPipelineReports(); + return impl.register(request.getDatanodeDetails(), dnNodeReport, + containerRequestProto, pipelineReport); + } @Override - public SCMRegisteredResponseProto register(RpcController controller, - SCMRegisterRequestProto request) throws ServiceException { - try { - ContainerReportsProto containerRequestProto = request - .getContainerReport(); - NodeReportProto dnNodeReport = request.getNodeReport(); - PipelineReportsProto pipelineReport = request.getPipelineReports(); - return impl.register(request.getDatanodeDetails(), dnNodeReport, - containerRequestProto, pipelineReport); - } catch (IOException e) { - throw new ServiceException(e); - } + public SCMDatanodeResponse submitRequest(RpcController controller, + SCMDatanodeRequest request) throws ServiceException { + return dispatcher.processRequest(request, this::processMessage, + request.getCmdType(), request.getTraceID()); } - @Override - public SCMHeartbeatResponseProto sendHeartbeat(RpcController controller, - SCMHeartbeatRequestProto request) throws ServiceException { + public SCMDatanodeResponse processMessage(SCMDatanodeRequest request) + throws ServiceException { try { - return impl.sendHeartbeat(request); + Type cmdType = request.getCmdType(); + switch (cmdType) { + case GetVersion: + return SCMDatanodeResponse.newBuilder() + .setCmdType(cmdType) + .setStatus(Status.OK) + .setGetVersionResponse( + impl.getVersion(request.getGetVersionRequest())) + .build(); + case SendHeartbeat: + return SCMDatanodeResponse.newBuilder() + .setCmdType(cmdType) + .setStatus(Status.OK) + .setSendHeartbeatResponse( + impl.sendHeartbeat(request.getSendHeartbeatRequest())) + .build(); + case Register: + return SCMDatanodeResponse.newBuilder() + .setCmdType(cmdType) + .setStatus(Status.OK) + .setRegisterResponse(register(request.getRegisterRequest())) + .build(); + default: + throw new ServiceException("Unknown command type: " + cmdType); + } } catch (IOException e) { throw new ServiceException(e); } } - } \ No newline at end of file diff --git a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto index 1d09dfa902bc3..a975cd5605fc7 100644 --- a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto +++ b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto @@ -34,6 +34,45 @@ package hadoop.hdds; import "hdds.proto"; + +message SCMDatanodeRequest { + required Type cmdType = 1; // Type of the command + + optional string traceID = 2; + + optional SCMVersionRequestProto getVersionRequest = 3; + optional SCMRegisterRequestProto registerRequest = 4; + optional SCMHeartbeatRequestProto sendHeartbeatRequest = 5; +} + +message SCMDatanodeResponse { + required Type cmdType = 1; // Type of the command + + optional string traceID = 2; + + optional bool success = 3 [default = true]; + + optional string message = 4; + + required Status status = 5; + + optional SCMVersionResponseProto getVersionResponse = 6; + optional SCMRegisteredResponseProto registerResponse = 7; + optional SCMHeartbeatResponseProto sendHeartbeatResponse = 8; + +} + +enum Type { + GetVersion = 1; + Register = 2; + SendHeartbeat = 3; +} + +enum Status { + OK = 1; + ERROR = 2; +} + /** * Request for version info of the software stack on the server. */ @@ -385,21 +424,6 @@ message ReplicateContainerCommandProto { */ service StorageContainerDatanodeProtocolService { - /** - * Gets the version information from the SCM. - */ - rpc getVersion (SCMVersionRequestProto) returns (SCMVersionResponseProto); - - /** - * Registers a data node with SCM. - */ - rpc register (SCMRegisterRequestProto) returns (SCMRegisteredResponseProto); - - /** - * Send heartbeat from datanode to SCM. HB's under SCM looks more - * like life line protocol than HB's under HDFS. In other words, it is - * extremely light weight and contains no data payload. - */ - rpc sendHeartbeat (SCMHeartbeatRequestProto) returns (SCMHeartbeatResponseProto); - + //Message sent from Datanode to SCM as a heartbeat. + rpc submitRequest (SCMDatanodeRequest) returns (SCMDatanodeResponse); } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java index 514c8224bca9d..5a7c30ca68f79 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java @@ -29,12 +29,14 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol; +import org.apache.hadoop.ozone.protocolPB.ProtocolMessageMetrics; import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB; import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolServerSideTranslatorPB; import org.apache.hadoop.test.GenericTestUtils; import com.google.protobuf.BlockingService; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY; +import org.mockito.Mockito; /** * Test Endpoint class. @@ -91,7 +93,7 @@ public static RPC.Server startScmRpcServer(Configuration configuration, StorageContainerDatanodeProtocolService. newReflectiveBlockingService( new StorageContainerDatanodeProtocolServerSideTranslatorPB( - server)); + server, Mockito.mock(ProtocolMessageMetrics.class))); RPC.Server scmServer = startRpcServer(configuration, rpcServerAddresss, StorageContainerDatanodeProtocolPB.class, scmDatanodeService, diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java index d8890fb0b920e..fd8bb87ceb12e 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java @@ -192,11 +192,11 @@ void processNodeReport(DatanodeDetails datanodeDetails, DatanodeDetails getNodeByUuid(String uuid); /** - * Given datanode address(Ipaddress or hostname), returns the DatanodeDetails - * for the node. + * Given datanode address(Ipaddress or hostname), returns a list of + * DatanodeDetails for the datanodes running at that address. * * @param address datanode address - * @return the given datanode, or null if not found + * @return the given datanode, or empty list if none found */ - DatanodeDetails getNodeByAddress(String address); + List getNodesByAddress(String address); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index d3df858e6e6e1..ed65ed3555321 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -25,11 +25,13 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.LinkedList; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledFuture; import java.util.stream.Collectors; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; @@ -98,7 +100,7 @@ public class SCMNodeManager implements NodeManager { private final NetworkTopology clusterMap; private final DNSToSwitchMapping dnsToSwitchMapping; private final boolean useHostname; - private final ConcurrentHashMap dnsToUuidMap = + private final ConcurrentHashMap> dnsToUuidMap = new ConcurrentHashMap<>(); /** @@ -260,7 +262,7 @@ public RegisteredCommand register( } nodeStateManager.addNode(datanodeDetails); clusterMap.add(datanodeDetails); - dnsToUuidMap.put(dnsName, datanodeDetails.getUuidString()); + addEntryTodnsToUuidMap(dnsName, datanodeDetails.getUuidString()); // Updating Node Report, as registration is successful processNodeReport(datanodeDetails, nodeReport); LOG.info("Registered Data node : {}", datanodeDetails); @@ -275,6 +277,26 @@ public RegisteredCommand register( .build(); } + /** + * Add an entry to the dnsToUuidMap, which maps hostname / IP to the DNs + * running on that host. As each address can have many DNs running on it, + * this is a one to many mapping. + * @param dnsName String representing the hostname or IP of the node + * @param uuid String representing the UUID of the registered node. + */ + @SuppressFBWarnings(value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION", + justification="The method is synchronized and this is the only place "+ + "dnsToUuidMap is modified") + private synchronized void addEntryTodnsToUuidMap( + String dnsName, String uuid) { + Set dnList = dnsToUuidMap.get(dnsName); + if (dnList == null) { + dnList = ConcurrentHashMap.newKeySet(); + dnsToUuidMap.put(dnsName, dnList); + } + dnList.add(uuid); + } + /** * Send heartbeat to indicate the datanode is alive and doing well. * @@ -584,29 +606,34 @@ public DatanodeDetails getNodeByUuid(String uuid) { } /** - * Given datanode address(Ipaddress or hostname), returns the DatanodeDetails - * for the node. + * Given datanode address(Ipaddress or hostname), return a list of + * DatanodeDetails for the datanodes registered on that address. * * @param address datanode address - * @return the given datanode, or null if not found + * @return the given datanode, or empty list if none found */ @Override - public DatanodeDetails getNodeByAddress(String address) { + public List getNodesByAddress(String address) { + List results = new LinkedList<>(); if (Strings.isNullOrEmpty(address)) { LOG.warn("address is null"); - return null; + return results; } - String uuid = dnsToUuidMap.get(address); - if (uuid != null) { + Set uuids = dnsToUuidMap.get(address); + if (uuids == null) { + LOG.warn("Cannot find node for address {}", address); + return results; + } + + for (String uuid : uuids) { DatanodeDetails temp = DatanodeDetails.newBuilder().setUuid(uuid).build(); try { - return nodeStateManager.getNode(temp); + results.add(nodeStateManager.getNode(temp)); } catch (NodeNotFoundException e) { LOG.warn("Cannot find node for uuid {}", uuid); } } - LOG.warn("Cannot find node for address {}", address); - return null; + return results; } private String nodeResolve(String hostname) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/SCMSecurityProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/SCMSecurityProtocolServerSideTranslatorPB.java new file mode 100644 index 0000000000000..2d14fa6b060a1 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/SCMSecurityProtocolServerSideTranslatorPB.java @@ -0,0 +1,186 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.hadoop.hdds.scm.protocol; + +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertificateRequestProto; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetDataNodeCertRequestProto; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto.ResponseCode; +import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetOMCertRequestProto; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMSecurityRequest; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMSecurityResponse; +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.Status; +import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolPB; +import org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher; +import org.apache.hadoop.ozone.protocolPB.ProtocolMessageMetrics; + +/** + * This class is the server-side translator that forwards requests received on + * {@link SCMSecurityProtocolPB} to the {@link + * SCMSecurityProtocol} server implementation. + */ +public class SCMSecurityProtocolServerSideTranslatorPB + implements SCMSecurityProtocolPB { + + private static final Logger LOG = + LoggerFactory.getLogger(SCMSecurityProtocolServerSideTranslatorPB.class); + + private final SCMSecurityProtocol impl; + + private OzoneProtocolMessageDispatcher + dispatcher; + + public SCMSecurityProtocolServerSideTranslatorPB(SCMSecurityProtocol impl, + ProtocolMessageMetrics messageMetrics) { + this.impl = impl; + this.dispatcher = + new OzoneProtocolMessageDispatcher<>("ScmSecurityProtocol", + messageMetrics, LOG); + } + + @Override + public SCMSecurityResponse submitRequest(RpcController controller, + SCMSecurityRequest request) throws ServiceException { + return dispatcher.processRequest(request, this::processRequest, + request.getCmdType(), request.getTraceID()); + } + + public SCMSecurityResponse processRequest(SCMSecurityRequest request) + throws ServiceException { + try { + switch (request.getCmdType()) { + case GetCertificate: + return SCMSecurityResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setGetCertResponseProto( + getCertificate(request.getGetCertificateRequest())) + .build(); + case GetCACertificate: + return SCMSecurityResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setGetCertResponseProto( + getCACertificate(request.getGetCACertificateRequest())) + .build(); + case GetOMCertificate: + return SCMSecurityResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setGetCertResponseProto( + getOMCertificate(request.getGetOMCertRequest())) + .build(); + case GetDataNodeCertificate: + return SCMSecurityResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setGetCertResponseProto( + getDataNodeCertificate(request.getGetDataNodeCertRequest())) + .build(); + default: + throw new IllegalArgumentException( + "Unknown request type: " + request.getCmdType()); + } + } catch (IOException e) { + throw new ServiceException(e); + } + } + + /** + * Get SCM signed certificate for DataNode. + * + * @param request + * @return SCMGetDataNodeCertResponseProto. + */ + + public SCMGetCertResponseProto getDataNodeCertificate( + SCMGetDataNodeCertRequestProto request) + throws IOException { + + String certificate = impl + .getDataNodeCertificate(request.getDatanodeDetails(), + request.getCSR()); + SCMGetCertResponseProto.Builder builder = + SCMGetCertResponseProto + .newBuilder() + .setResponseCode(ResponseCode.success) + .setX509Certificate(certificate) + .setX509CACertificate(impl.getCACertificate()); + + return builder.build(); + + } + + /** + * Get SCM signed certificate for OzoneManager. + * + * @param request + * @return SCMGetCertResponseProto. + */ + public SCMGetCertResponseProto getOMCertificate( + SCMGetOMCertRequestProto request) throws IOException { + String certificate = impl + .getOMCertificate(request.getOmDetails(), + request.getCSR()); + SCMGetCertResponseProto.Builder builder = + SCMGetCertResponseProto + .newBuilder() + .setResponseCode(ResponseCode.success) + .setX509Certificate(certificate) + .setX509CACertificate(impl.getCACertificate()); + return builder.build(); + + } + + public SCMGetCertResponseProto getCertificate( + SCMGetCertificateRequestProto request) throws IOException { + + String certificate = impl.getCertificate(request.getCertSerialId()); + SCMGetCertResponseProto.Builder builder = + SCMGetCertResponseProto + .newBuilder() + .setResponseCode(ResponseCode.success) + .setX509Certificate(certificate); + return builder.build(); + + } + + public SCMGetCertResponseProto getCACertificate( + SCMSecurityProtocolProtos.SCMGetCACertificateRequestProto request) + throws IOException { + + String certificate = impl.getCACertificate(); + SCMGetCertResponseProto.Builder builder = + SCMGetCertResponseProto + .newBuilder() + .setResponseCode(ResponseCode.success) + .setX509Certificate(certificate); + return builder.build(); + + } + +} \ No newline at end of file diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java index 9d53dbf7d3b0c..0d2f470000387 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java @@ -1,4 +1,3 @@ - /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -7,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * + *

* http://www.apache.org/licenses/LICENSE-2.0 - * + *

* Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -42,16 +41,18 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.InSafeModeResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ListPipelineRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ListPipelineResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.NodeQueryResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ObjectStageChangeResponseProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.PipelineRequestProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.PipelineResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ReplicationManagerStatusRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ReplicationManagerStatusResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMDeleteContainerRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMDeleteContainerResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ScmContainerLocationRequest; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ScmContainerLocationResponse; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ScmContainerLocationResponse.Status; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartReplicationManagerRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartReplicationManagerResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StopReplicationManagerRequestProto; @@ -61,11 +62,13 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB; -import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher; +import org.apache.hadoop.ozone.protocolPB.ProtocolMessageMetrics; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; -import io.opentracing.Scope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class is the server-side translator that forwards requests received on @@ -76,288 +79,315 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB implements StorageContainerLocationProtocolPB { + private static final Logger LOG = + LoggerFactory.getLogger( + StorageContainerLocationProtocolServerSideTranslatorPB.class); + private final StorageContainerLocationProtocol impl; + private OzoneProtocolMessageDispatcher + dispatcher; + /** * Creates a new StorageContainerLocationProtocolServerSideTranslatorPB. * - * @param impl {@link StorageContainerLocationProtocol} server implementation + * @param impl {@link StorageContainerLocationProtocol} server + * implementation + * @param protocolMetrics */ public StorageContainerLocationProtocolServerSideTranslatorPB( - StorageContainerLocationProtocol impl) throws IOException { + StorageContainerLocationProtocol impl, + ProtocolMessageMetrics protocolMetrics) throws IOException { this.impl = impl; + this.dispatcher = + new OzoneProtocolMessageDispatcher<>("ScmContainerLocation", + protocolMetrics, LOG); } @Override - public ContainerResponseProto allocateContainer(RpcController unused, - ContainerRequestProto request) throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("allocateContainer", request.getTraceID())) { - ContainerWithPipeline containerWithPipeline = impl - .allocateContainer(request.getReplicationType(), - request.getReplicationFactor(), request.getOwner()); - return ContainerResponseProto.newBuilder() - .setContainerWithPipeline(containerWithPipeline.getProtobuf()) - .setErrorCode(ContainerResponseProto.Error.success) - .build(); + public ScmContainerLocationResponse submitRequest(RpcController controller, + ScmContainerLocationRequest request) throws ServiceException { + return dispatcher + .processRequest(request, this::processRequest, request.getCmdType(), + request.getTraceID()); + } + + public ScmContainerLocationResponse processRequest( + ScmContainerLocationRequest request) throws ServiceException { + try { + switch (request.getCmdType()) { + case AllocateContainer: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setContainerResponse( + allocateContainer(request.getContainerRequest())) + .build(); + case GetContainer: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setGetContainerResponse( + getContainer(request.getGetContainerRequest())) + .build(); + case GetContainerWithPipeline: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setGetContainerWithPipelineResponse(getContainerWithPipeline( + request.getGetContainerWithPipelineRequest())) + .build(); + case ListContainer: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setScmListContainerResponse(listContainer( + request.getScmListContainerRequest())) + .build(); + case QueryNode: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setNodeQueryResponse(queryNode(request.getNodeQueryRequest())) + .build(); + case NotifyObjectStageChange: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setObjectStageChangeResponse(notifyObjectStageChange( + request.getObjectStageChangeRequest())) + .build(); + case ListPipelines: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setListPipelineResponse(listPipelines( + request.getListPipelineRequest())) + .build(); + case ActivatePipeline: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setActivatePipelineResponse(activatePipeline( + request.getActivatePipelineRequest())) + .build(); + case GetScmInfo: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setGetScmInfoResponse(getScmInfo( + request.getGetScmInfoRequest())) + .build(); + case InSafeMode: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setInSafeModeResponse(inSafeMode( + request.getInSafeModeRequest())) + .build(); + case ForceExitSafeMode: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setForceExitSafeModeResponse(forceExitSafeMode( + request.getForceExitSafeModeRequest())) + .build(); + case StartReplicationManager: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setStartReplicationManagerResponse(startReplicationManager( + request.getStartReplicationManagerRequest())) + .build(); + case StopReplicationManager: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setStopReplicationManagerResponse(stopReplicationManager( + request.getStopReplicationManagerRequest())) + .build(); + case GetReplicationManagerStatus: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setReplicationManagerStatusResponse(getReplicationManagerStatus( + request.getSeplicationManagerStatusRequest())) + .build(); + default: + throw new IllegalArgumentException( + "Unknown command type: " + request.getCmdType()); + } } catch (IOException e) { throw new ServiceException(e); } } - @Override + public ContainerResponseProto allocateContainer(ContainerRequestProto request) + throws IOException { + ContainerWithPipeline containerWithPipeline = impl + .allocateContainer(request.getReplicationType(), + request.getReplicationFactor(), request.getOwner()); + return ContainerResponseProto.newBuilder() + .setContainerWithPipeline(containerWithPipeline.getProtobuf()) + .setErrorCode(ContainerResponseProto.Error.success) + .build(); + + } + public GetContainerResponseProto getContainer( - RpcController controller, GetContainerRequestProto request) - throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("getContainer", request.getTraceID())) { - ContainerInfo container = impl.getContainer(request.getContainerID()); - return GetContainerResponseProto.newBuilder() - .setContainerInfo(container.getProtobuf()) - .build(); - } catch (IOException e) { - throw new ServiceException(e); - } + GetContainerRequestProto request) throws IOException { + ContainerInfo container = impl.getContainer(request.getContainerID()); + return GetContainerResponseProto.newBuilder() + .setContainerInfo(container.getProtobuf()) + .build(); } - @Override public GetContainerWithPipelineResponseProto getContainerWithPipeline( - RpcController controller, GetContainerWithPipelineRequestProto request) - throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("getContainerWithPipeline", - request.getTraceID())) { - ContainerWithPipeline container = impl - .getContainerWithPipeline(request.getContainerID()); - return GetContainerWithPipelineResponseProto.newBuilder() - .setContainerWithPipeline(container.getProtobuf()) - .build(); - } catch (IOException e) { - throw new ServiceException(e); - } + GetContainerWithPipelineRequestProto request) + throws IOException { + ContainerWithPipeline container = impl + .getContainerWithPipeline(request.getContainerID()); + return GetContainerWithPipelineResponseProto.newBuilder() + .setContainerWithPipeline(container.getProtobuf()) + .build(); } - @Override - public SCMListContainerResponseProto listContainer(RpcController controller, - SCMListContainerRequestProto request) throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("listContainer", request.getTraceID())) { - long startContainerID = 0; - int count = -1; - - // Arguments check. - if (request.hasStartContainerID()) { - // End container name is given. - startContainerID = request.getStartContainerID(); - } - count = request.getCount(); - List containerList = - impl.listContainer(startContainerID, count); - SCMListContainerResponseProto.Builder builder = - SCMListContainerResponseProto.newBuilder(); - for (ContainerInfo container : containerList) { - builder.addContainers(container.getProtobuf()); - } - return builder.build(); - } catch (IOException e) { - throw new ServiceException(e); + public SCMListContainerResponseProto listContainer( + SCMListContainerRequestProto request) throws IOException { + + long startContainerID = 0; + int count = -1; + + // Arguments check. + if (request.hasStartContainerID()) { + // End container name is given. + startContainerID = request.getStartContainerID(); } + count = request.getCount(); + List containerList = + impl.listContainer(startContainerID, count); + SCMListContainerResponseProto.Builder builder = + SCMListContainerResponseProto.newBuilder(); + for (ContainerInfo container : containerList) { + builder.addContainers(container.getProtobuf()); + } + return builder.build(); } - @Override public SCMDeleteContainerResponseProto deleteContainer( - RpcController controller, SCMDeleteContainerRequestProto request) - throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("deleteContainer", request.getTraceID())) { - impl.deleteContainer(request.getContainerID()); - return SCMDeleteContainerResponseProto.newBuilder().build(); - } catch (IOException e) { - throw new ServiceException(e); - } + SCMDeleteContainerRequestProto request) + throws IOException { + impl.deleteContainer(request.getContainerID()); + return SCMDeleteContainerResponseProto.newBuilder().build(); + } - @Override - public StorageContainerLocationProtocolProtos.NodeQueryResponseProto - queryNode(RpcController controller, + public NodeQueryResponseProto queryNode( StorageContainerLocationProtocolProtos.NodeQueryRequestProto request) - throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("queryNode", request.getTraceID())) { - HddsProtos.NodeState nodeState = request.getState(); - List datanodes = impl.queryNode(nodeState, - request.getScope(), request.getPoolName()); - return StorageContainerLocationProtocolProtos - .NodeQueryResponseProto.newBuilder() - .addAllDatanodes(datanodes) - .build(); - } catch (Exception e) { - throw new ServiceException(e); - } - } + throws IOException { + + HddsProtos.NodeState nodeState = request.getState(); + List datanodes = impl.queryNode(nodeState, + request.getScope(), request.getPoolName()); + return NodeQueryResponseProto.newBuilder() + .addAllDatanodes(datanodes) + .build(); - @Override - public ObjectStageChangeResponseProto notifyObjectStageChange( - RpcController controller, ObjectStageChangeRequestProto request) - throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("notifyObjectStageChange", - request.getTraceID())) { - impl.notifyObjectStageChange(request.getType(), request.getId(), - request.getOp(), request.getStage()); - return ObjectStageChangeResponseProto.newBuilder().build(); - } catch (IOException e) { - throw new ServiceException(e); - } } - @Override - public PipelineResponseProto allocatePipeline( - RpcController controller, PipelineRequestProto request) - throws ServiceException { - // TODO : Wiring this up requires one more patch. - return null; + public ObjectStageChangeResponseProto notifyObjectStageChange( + ObjectStageChangeRequestProto request) + throws IOException { + impl.notifyObjectStageChange(request.getType(), request.getId(), + request.getOp(), request.getStage()); + return ObjectStageChangeResponseProto.newBuilder().build(); } - @Override public ListPipelineResponseProto listPipelines( - RpcController controller, ListPipelineRequestProto request) - throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("listPipelines", request.getTraceID())) { - ListPipelineResponseProto.Builder builder = ListPipelineResponseProto - .newBuilder(); - List pipelines = impl.listPipelines(); - for (Pipeline pipeline : pipelines) { - HddsProtos.Pipeline protobufMessage = pipeline.getProtobufMessage(); - builder.addPipelines(protobufMessage); - } - return builder.build(); - } catch (IOException e) { - throw new ServiceException(e); + ListPipelineRequestProto request) + throws IOException { + ListPipelineResponseProto.Builder builder = ListPipelineResponseProto + .newBuilder(); + List pipelines = impl.listPipelines(); + for (Pipeline pipeline : pipelines) { + HddsProtos.Pipeline protobufMessage = pipeline.getProtobufMessage(); + builder.addPipelines(protobufMessage); } + return builder.build(); } - @Override public ActivatePipelineResponseProto activatePipeline( - RpcController controller, ActivatePipelineRequestProto request) - throws ServiceException { - try (Scope ignored = TracingUtil - .importAndCreateScope("activatePipeline", request.getTraceID())) { - impl.activatePipeline(request.getPipelineID()); - return ActivatePipelineResponseProto.newBuilder().build(); - } catch (IOException e) { - throw new ServiceException(e); - } + ActivatePipelineRequestProto request) + throws IOException { + impl.activatePipeline(request.getPipelineID()); + return ActivatePipelineResponseProto.newBuilder().build(); } - @Override public DeactivatePipelineResponseProto deactivatePipeline( - RpcController controller, DeactivatePipelineRequestProto request) - throws ServiceException { - try (Scope ignored = TracingUtil - .importAndCreateScope("deactivatePipeline", request.getTraceID())) { - impl.deactivatePipeline(request.getPipelineID()); - return DeactivatePipelineResponseProto.newBuilder().build(); - } catch (IOException e) { - throw new ServiceException(e); - } + DeactivatePipelineRequestProto request) + throws IOException { + impl.deactivatePipeline(request.getPipelineID()); + return DeactivatePipelineResponseProto.newBuilder().build(); } - @Override public ClosePipelineResponseProto closePipeline( RpcController controller, ClosePipelineRequestProto request) - throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("closePipeline", request.getTraceID())) { - impl.closePipeline(request.getPipelineID()); - return ClosePipelineResponseProto.newBuilder().build(); - } catch (IOException e) { - throw new ServiceException(e); - } + throws IOException { + + impl.closePipeline(request.getPipelineID()); + return ClosePipelineResponseProto.newBuilder().build(); + } - @Override public HddsProtos.GetScmInfoResponseProto getScmInfo( - RpcController controller, HddsProtos.GetScmInfoRequestProto req) - throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("getScmInfo", req.getTraceID())) { - ScmInfo scmInfo = impl.getScmInfo(); - return HddsProtos.GetScmInfoResponseProto.newBuilder() - .setClusterId(scmInfo.getClusterId()) - .setScmId(scmInfo.getScmId()) - .build(); - } catch (IOException ex) { - throw new ServiceException(ex); - } + HddsProtos.GetScmInfoRequestProto req) + throws IOException { + ScmInfo scmInfo = impl.getScmInfo(); + return HddsProtos.GetScmInfoResponseProto.newBuilder() + .setClusterId(scmInfo.getClusterId()) + .setScmId(scmInfo.getScmId()) + .build(); } - @Override public InSafeModeResponseProto inSafeMode( - RpcController controller, - InSafeModeRequestProto request) throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("inSafeMode", request.getTraceID())) { - return InSafeModeResponseProto.newBuilder() - .setInSafeMode(impl.inSafeMode()).build(); - } catch (IOException ex) { - throw new ServiceException(ex); - } + InSafeModeRequestProto request) throws IOException { + + return InSafeModeResponseProto.newBuilder() + .setInSafeMode(impl.inSafeMode()).build(); + } - @Override public ForceExitSafeModeResponseProto forceExitSafeMode( - RpcController controller, ForceExitSafeModeRequestProto request) - throws ServiceException { - try (Scope scope = TracingUtil - .importAndCreateScope("forceExitSafeMode", request.getTraceID())) { - return ForceExitSafeModeResponseProto.newBuilder() - .setExitedSafeMode(impl.forceExitSafeMode()).build(); - } catch (IOException ex) { - throw new ServiceException(ex); - } + ForceExitSafeModeRequestProto request) + throws IOException { + return ForceExitSafeModeResponseProto.newBuilder() + .setExitedSafeMode(impl.forceExitSafeMode()).build(); + } - @Override public StartReplicationManagerResponseProto startReplicationManager( - RpcController controller, StartReplicationManagerRequestProto request) - throws ServiceException { - try (Scope ignored = TracingUtil.importAndCreateScope( - "startReplicationManager", request.getTraceID())) { - impl.startReplicationManager(); - return StartReplicationManagerResponseProto.newBuilder().build(); - } catch (IOException ex) { - throw new ServiceException(ex); - } + StartReplicationManagerRequestProto request) + throws IOException { + impl.startReplicationManager(); + return StartReplicationManagerResponseProto.newBuilder().build(); } - @Override public StopReplicationManagerResponseProto stopReplicationManager( - RpcController controller, StopReplicationManagerRequestProto request) - throws ServiceException { - try (Scope ignored = TracingUtil.importAndCreateScope( - "stopReplicationManager", request.getTraceID())) { - impl.stopReplicationManager(); - return StopReplicationManagerResponseProto.newBuilder().build(); - } catch (IOException ex) { - throw new ServiceException(ex); - } + StopReplicationManagerRequestProto request) + throws IOException { + impl.stopReplicationManager(); + return StopReplicationManagerResponseProto.newBuilder().build(); + } - @Override public ReplicationManagerStatusResponseProto getReplicationManagerStatus( - RpcController controller, ReplicationManagerStatusRequestProto request) - throws ServiceException { - try (Scope ignored = TracingUtil.importAndCreateScope( - "getReplicationManagerStatus", request.getTraceID())) { - return ReplicationManagerStatusResponseProto.newBuilder() - .setIsRunning(impl.getReplicationManagerStatus()).build(); - } catch (IOException ex) { - throw new ServiceException(ex); - } + ReplicationManagerStatusRequestProto request) + throws IOException { + return ReplicationManagerStatusResponseProto.newBuilder() + .setIsRunning(impl.getReplicationManagerStatus()).build(); } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java index 496d481b61145..8eadeb355436c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java @@ -19,6 +19,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; @@ -63,19 +64,18 @@ public ContainerSafeModeRule(String ruleName, EventQueue eventQueue, " value should be >= 0.0 and <= 1.0"); containerMap = new ConcurrentHashMap<>(); - if(containers != null) { - containers.forEach(c -> { - // TODO: There can be containers in OPEN state which were never - // created by the client. We are not considering these containers for - // now. These containers can be handled by tracking pipelines. - if (c != null && c.getState() != null && - !c.getState().equals(HddsProtos.LifeCycleState.OPEN)) { - containerMap.put(c.getContainerID(), c); - } - }); - maxContainer = containerMap.size(); - } - + containers.forEach(container -> { + // There can be containers in OPEN/CLOSING state which were never + // created by the client. We are not considering these containers for + // now. These containers can be handled by tracking pipelines. + + Optional.ofNullable(container.getState()) + .filter(state -> state != HddsProtos.LifeCycleState.OPEN) + .filter(state -> state != HddsProtos.LifeCycleState.CLOSING) + .ifPresent(s -> containerMap.put(container.getContainerID(), + container)); + }); + maxContainer = containerMap.size(); long cutOff = (long) Math.ceil(maxContainer * safeModeCutoff); getSafeModeMetrics().setNumContainerWithOneReplicaReportedThreshold(cutOff); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java index 5500891d98e95..9c69758d5a0a8 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java @@ -295,7 +295,12 @@ public List sortDatanodes(List nodes, boolean auditSuccess = true; try{ NodeManager nodeManager = scm.getScmNodeManager(); - Node client = nodeManager.getNodeByAddress(clientMachine); + Node client = null; + List possibleClients = + nodeManager.getNodesByAddress(clientMachine); + if (possibleClients.size()>0){ + client = possibleClients.get(0); + } List nodeList = new ArrayList(); nodes.stream().forEach(uuid -> { DatanodeDetails node = nodeManager.getNodeByUuid(uuid); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java index e0136e81e5912..9c27f6a64d620 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java @@ -61,6 +61,7 @@ import org.apache.hadoop.ozone.audit.Auditor; import org.apache.hadoop.ozone.audit.SCMAction; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB; +import org.apache.hadoop.ozone.protocolPB.ProtocolMessageMetrics; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -103,6 +104,7 @@ public class SCMClientProtocolServer implements private final StorageContainerManager scm; private final OzoneConfiguration conf; private SafeModePrecheck safeModePrecheck; + private final ProtocolMessageMetrics protocolMetrics; public SCMClientProtocolServer(OzoneConfiguration conf, StorageContainerManager scm) throws IOException { @@ -115,10 +117,16 @@ public SCMClientProtocolServer(OzoneConfiguration conf, RPC.setProtocolEngine(conf, StorageContainerLocationProtocolPB.class, ProtobufRpcEngine.class); + protocolMetrics = ProtocolMessageMetrics + .create("ScmContainerLocationProtocol", + "SCM ContainerLocation protocol metrics", + StorageContainerLocationProtocolProtos.Type.values()); + // SCM Container Service RPC BlockingService storageProtoPbService = newReflectiveBlockingService( - new StorageContainerLocationProtocolServerSideTranslatorPB(this)); + new StorageContainerLocationProtocolServerSideTranslatorPB(this, + protocolMetrics)); final InetSocketAddress scmAddress = HddsServerUtil .getScmClientBindAddress(conf); @@ -147,6 +155,7 @@ public InetSocketAddress getClientRpcAddress() { } public void start() { + protocolMetrics.register(); LOG.info( StorageContainerManager.buildRpcServerStartMessage( "RPC server for Client ", getClientRpcAddress())); @@ -154,6 +163,7 @@ public void start() { } public void stop() { + protocolMetrics.unregister(); try { LOG.info("Stopping the RPC server for Client Protocol"); getClientRpcServer().stop(); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java index 6dd9dab827fe6..530c0a6d2383b 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java @@ -21,61 +21,32 @@ */ package org.apache.hadoop.hdds.scm.server; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.Maps; -import com.google.protobuf.BlockingService; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; - -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.ContainerReportsProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.PipelineReportsProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.ReregisterCommandProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.NodeReportProto; - -import static org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto - .Type.closeContainerCommand; -import static org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto - .Type.deleteBlocksCommand; -import static org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto - .Type.deleteContainerCommand; -import static org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type - .replicateContainerCommand; -import static org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto - .Type.reregisterCommand; - - - +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReregisterCommandProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto; import org.apache.hadoop.hdds.scm.HddsServerUtil; import org.apache.hadoop.hdds.scm.events.SCMEvents; -import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher - .ReportFromDatanode; -import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher - .PipelineReportFromDatanode; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ReportFromDatanode; import org.apache.hadoop.hdds.server.events.EventPublisher; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ipc.ProtobufRpcEngine; @@ -95,27 +66,28 @@ import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand; import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand; +import org.apache.hadoop.ozone.protocolPB.ProtocolMessageMetrics; import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB; -import org.apache.hadoop.ozone.protocolPB - .StorageContainerDatanodeProtocolServerSideTranslatorPB; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; +import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolServerSideTranslatorPB; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import com.google.protobuf.BlockingService; +import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.closeContainerCommand; +import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.deleteBlocksCommand; +import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.deleteContainerCommand; +import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.replicateContainerCommand; +import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.reregisterCommand; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY; - import static org.apache.hadoop.hdds.scm.events.SCMEvents.CONTAINER_REPORT; import static org.apache.hadoop.hdds.scm.events.SCMEvents.PIPELINE_REPORT; import static org.apache.hadoop.hdds.scm.server.StorageContainerManager.startRpcServer; import static org.apache.hadoop.hdds.server.ServerUtils.updateRPCListenAddress; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Protocol Handler for Datanode Protocol. @@ -138,6 +110,7 @@ public class SCMDatanodeProtocolServer implements private final InetSocketAddress datanodeRpcAddress; private final SCMDatanodeHeartbeatDispatcher heartbeatDispatcher; private final EventPublisher eventPublisher; + private final ProtocolMessageMetrics protocolMessageMetrics; public SCMDatanodeProtocolServer(final OzoneConfiguration conf, StorageContainerManager scm, EventPublisher eventPublisher) @@ -157,12 +130,17 @@ public SCMDatanodeProtocolServer(final OzoneConfiguration conf, RPC.setProtocolEngine(conf, StorageContainerDatanodeProtocolPB.class, ProtobufRpcEngine.class); + + protocolMessageMetrics = ProtocolMessageMetrics + .create("SCMDatanodeProtocol", "SCM Datanode protocol", + StorageContainerDatanodeProtocolProtos.Type.values()); + BlockingService dnProtoPbService = StorageContainerDatanodeProtocolProtos .StorageContainerDatanodeProtocolService .newReflectiveBlockingService( new StorageContainerDatanodeProtocolServerSideTranslatorPB( - this)); + this, protocolMessageMetrics)); InetSocketAddress datanodeRpcAddr = HddsServerUtil.getScmDataNodeBindAddress(conf); @@ -191,6 +169,7 @@ public void start() { LOG.info( StorageContainerManager.buildRpcServerStartMessage( "RPC server for DataNodes", datanodeRpcAddress)); + protocolMessageMetrics.register(); datanodeRpcServer.start(); } @@ -370,6 +349,7 @@ public void stop() { LOG.error(" datanodeRpcServer stop failed.", ex); } IOUtils.cleanupWithLogger(LOG, scm.getScmNodeManager()); + protocolMessageMetrics.unregister(); } @Override diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java index 05a1e04466b8f..c4b4efd30e0ad 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java @@ -5,9 +5,9 @@ * licenses this file to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance with the License. * You may obtain a copy of the License at - * + *

* http://www.apache.org/licenses/LICENSE-2.0 - * + *

* Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the @@ -17,6 +17,7 @@ package org.apache.hadoop.hdds.scm.server; import com.google.protobuf.BlockingService; + import java.io.IOException; import java.net.InetSocketAddress; import java.security.cert.CertificateException; @@ -32,7 +33,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.OzoneManagerDetailsProto; import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos; import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolPB; -import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolServerSideTranslatorPB; +import org.apache.hadoop.hdds.scm.protocol.SCMSecurityProtocolServerSideTranslatorPB; import org.apache.hadoop.hdds.scm.HddsServerUtil; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol; @@ -41,7 +42,9 @@ import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec; import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ozone.protocolPB.ProtocolMessageMetrics; import org.apache.hadoop.security.KerberosInfo; + import org.bouncycastle.cert.X509CertificateHolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,6 +65,7 @@ public class SCMSecurityProtocolServer implements SCMSecurityProtocol { private final CertificateServer certificateServer; private final RPC.Server rpcServer; private final InetSocketAddress rpcAddress; + private final ProtocolMessageMetrics metrics; SCMSecurityProtocolServer(OzoneConfiguration conf, CertificateServer certificateServer) throws IOException { @@ -76,10 +80,13 @@ public class SCMSecurityProtocolServer implements SCMSecurityProtocol { // SCM security service RPC service. RPC.setProtocolEngine(conf, SCMSecurityProtocolPB.class, ProtobufRpcEngine.class); + metrics = new ProtocolMessageMetrics("ScmSecurityProtocol", + "SCM Security protocol metrics", + SCMSecurityProtocolProtos.Type.values()); BlockingService secureProtoPbService = SCMSecurityProtocolProtos.SCMSecurityProtocolService .newReflectiveBlockingService( - new SCMSecurityProtocolServerSideTranslatorPB(this)); + new SCMSecurityProtocolServerSideTranslatorPB(this, metrics)); this.rpcServer = StorageContainerManager.startRpcServer( conf, @@ -96,8 +103,8 @@ public class SCMSecurityProtocolServer implements SCMSecurityProtocol { /** * Get SCM signed certificate for DataNode. * - * @param dnDetails - DataNode Details. - * @param certSignReq - Certificate signing request. + * @param dnDetails - DataNode Details. + * @param certSignReq - Certificate signing request. * @return String - SCM signed pem encoded certificate. */ @Override @@ -122,8 +129,8 @@ public String getDataNodeCertificate( /** * Get SCM signed certificate for OM. * - * @param omDetails - OzoneManager Details. - * @param certSignReq - Certificate signing request. + * @param omDetails - OzoneManager Details. + * @param certSignReq - Certificate signing request. * @return String - SCM signed pem encoded certificate. */ @Override @@ -147,7 +154,7 @@ public String getOMCertificate(OzoneManagerDetailsProto omDetails, /** * Get SCM signed certificate with given serial id. * - * @param certSerialId - Certificate serial id. + * @param certSerialId - Certificate serial id. * @return string - pem encoded SCM signed certificate. */ @Override @@ -196,12 +203,14 @@ public InetSocketAddress getRpcAddress() { public void start() { LOGGER.info(StorageContainerManager.buildRpcServerStartMessage("Starting" + " RPC server for SCMSecurityProtocolServer.", getRpcAddress())); + metrics.register(); getRpcServer().start(); } public void stop() { try { LOGGER.info("Stopping the SCMSecurityProtocolServer."); + metrics.unregister(); getRpcServer().stop(); } catch (Exception ex) { LOGGER.error("SCMSecurityProtocolServer stop failed.", ex); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java index b7a9813483d1d..6f5d4356fb3dc 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java @@ -88,7 +88,7 @@ public class MockNodeManager implements NodeManager { private final Node2PipelineMap node2PipelineMap; private final Node2ContainerMap node2ContainerMap; private NetworkTopology clusterMap; - private ConcurrentHashMap dnsToUuidMap; + private ConcurrentHashMap> dnsToUuidMap; public MockNodeManager(boolean initializeFakeNodes, int nodeCount) { this.healthyNodes = new LinkedList<>(); @@ -386,7 +386,7 @@ public RegisteredCommand register(DatanodeDetails datanodeDetails, try { node2ContainerMap.insertNewDatanode(datanodeDetails.getUuid(), Collections.emptySet()); - dnsToUuidMap.put(datanodeDetails.getIpAddress(), + addEntryTodnsToUuidMap(datanodeDetails.getIpAddress(), datanodeDetails.getUuidString()); if (clusterMap != null) { datanodeDetails.setNetworkName(datanodeDetails.getUuidString()); @@ -398,6 +398,23 @@ public RegisteredCommand register(DatanodeDetails datanodeDetails, return null; } + /** + * Add an entry to the dnsToUuidMap, which maps hostname / IP to the DNs + * running on that host. As each address can have many DNs running on it, + * this is a one to many mapping. + * @param dnsName String representing the hostname or IP of the node + * @param uuid String representing the UUID of the registered node. + */ + private synchronized void addEntryTodnsToUuidMap( + String dnsName, String uuid) { + Set dnList = dnsToUuidMap.get(dnsName); + if (dnList == null) { + dnList = ConcurrentHashMap.newKeySet(); + dnsToUuidMap.put(dnsName, dnList); + } + dnList.add(uuid); + } + /** * Send heartbeat to indicate the datanode is alive and doing well. * @@ -484,8 +501,19 @@ public DatanodeDetails getNodeByUuid(String uuid) { } @Override - public DatanodeDetails getNodeByAddress(String address) { - return getNodeByUuid(dnsToUuidMap.get(address)); + public List getNodesByAddress(String address) { + List results = new LinkedList<>(); + Set uuids = dnsToUuidMap.get(address); + if (uuids == null) { + return results; + } + for(String uuid : uuids) { + DatanodeDetails dn = getNodeByUuid(uuid); + if (dn != null) { + results.add(dn); + } + } + return results; } public void setNetworkTopology(NetworkTopology topology) { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java index d028851168576..db76d6678789a 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java @@ -1064,6 +1064,25 @@ public void testScmRegisterNodeWithHostname() testScmRegisterNodeWithNetworkTopology(true); } + /** + * Test getNodesByAddress when using IPs. + * + */ + @Test + public void testgetNodesByAddressWithIpAddress() + throws IOException, InterruptedException, AuthenticationException { + testGetNodesByAddress(false); + } + + /** + * Test getNodesByAddress when using hostnames. + */ + @Test + public void testgetNodesByAddressWithHostname() + throws IOException, InterruptedException, AuthenticationException { + testGetNodesByAddress(true); + } + /** * Test add node into a 4-layer network topology during node register. */ @@ -1152,11 +1171,55 @@ private void testScmRegisterNodeWithNetworkTopology(boolean useHostname) // test get node if (useHostname) { Arrays.stream(hostNames).forEach(hostname -> - Assert.assertNotNull(nodeManager.getNodeByAddress(hostname))); + Assert.assertNotEquals(0, nodeManager.getNodesByAddress(hostname) + .size())); } else { Arrays.stream(ipAddress).forEach(ip -> - Assert.assertNotNull(nodeManager.getNodeByAddress(ip))); + Assert.assertNotEquals(0, nodeManager.getNodesByAddress(ip) + .size())); } } } + + /** + * Test add node into a 4-layer network topology during node register. + */ + private void testGetNodesByAddress(boolean useHostname) + throws IOException, InterruptedException, AuthenticationException { + OzoneConfiguration conf = getConf(); + conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 1000, + MILLISECONDS); + + // create a set of hosts - note two hosts on "host1" + String[] hostNames = {"host1", "host1", "host2", "host3", "host4"}; + String[] ipAddress = + {"1.2.3.4", "1.2.3.4", "2.3.4.5", "3.4.5.6", "4.5.6.7"}; + + if (useHostname) { + conf.set(DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME, "true"); + } + final int nodeCount = hostNames.length; + try (SCMNodeManager nodeManager = createNodeManager(conf)) { + DatanodeDetails[] nodes = new DatanodeDetails[nodeCount]; + for (int i = 0; i < nodeCount; i++) { + DatanodeDetails node = TestUtils.createDatanodeDetails( + UUID.randomUUID().toString(), hostNames[i], ipAddress[i], null); + nodeManager.register(node, null, null); + } + // test get node + Assert.assertEquals(0, nodeManager.getNodesByAddress(null).size()); + if (useHostname) { + Assert.assertEquals(2, + nodeManager.getNodesByAddress("host1").size()); + Assert.assertEquals(1, nodeManager.getNodesByAddress("host2").size()); + Assert.assertEquals(0, nodeManager.getNodesByAddress("unknown").size()); + } else { + Assert.assertEquals(2, + nodeManager.getNodesByAddress("1.2.3.4").size()); + Assert.assertEquals(1, nodeManager.getNodesByAddress("2.3.4.5").size()); + Assert.assertEquals(0, nodeManager.getNodesByAddress("1.9.8.7").size()); + } + } + } + } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java index ba92035466cb5..247b38afc7f52 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java @@ -23,6 +23,7 @@ import java.io.File; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.UUID; @@ -60,7 +61,7 @@ public class TestSCMSafeModeManager { private static EventQueue queue; private SCMSafeModeManager scmSafeModeManager; private static Configuration config; - private List containers; + private List containers = Collections.emptyList(); @Rule public Timeout timeout = new Timeout(1000 * 300); @@ -85,7 +86,8 @@ public void testSafeModeState() throws Exception { @Test public void testSafeModeStateWithNullContainers() { - new SCMSafeModeManager(config, null, null, queue); + new SCMSafeModeManager(config, Collections.emptyList(), + null, queue); } private void testSafeMode(int numContainers) throws Exception { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java index 30a75efb19424..0ecff3f541a7c 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java @@ -44,6 +44,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.LinkedList; /** * A Node Manager to test replication. @@ -323,7 +324,7 @@ public DatanodeDetails getNodeByUuid(String address) { } @Override - public DatanodeDetails getNodeByAddress(String address) { - return null; + public List getNodesByAddress(String address) { + return new LinkedList<>(); } } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java index 288d9faf80177..5169c80779902 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java @@ -24,7 +24,6 @@ import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.scm.client.ScmClient; import org.apache.hadoop.hdds.scm.container.ContainerInfo; -import org.apache.hadoop.ozone.web.utils.JsonUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,8 +60,7 @@ public class ListSubcommand implements Callable { private void outputContainerInfo(ContainerInfo containerInfo) throws IOException { // Print container report info. - LOG.info("{}", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - containerInfo.toJsonString())); + LOG.info("{}", containerInfo.toJsonString()); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java index e90af846d7d50..8fd38bdb3b795 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java @@ -248,6 +248,8 @@ private int readToBuffer(BlockReader blockReader, DFSClient.LOG.warn("Found Checksum error for " + currentBlock + " from " + currentNode + " at " + ce.getPos()); + //Clear buffer to make next decode success + strategy.getReadBuffer().clear(); // we want to remember which block replicas we have tried corruptedBlocks.addCorruptedBlock(currentBlock, currentNode); throw ce; @@ -255,6 +257,8 @@ private int readToBuffer(BlockReader blockReader, DFSClient.LOG.warn("Exception while reading from " + currentBlock + " of " + dfsStripedInputStream.getSrc() + " from " + currentNode, e); + //Clear buffer to make next decode success + strategy.getReadBuffer().clear(); throw e; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java index 7251c7b67f833..012d70895021d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -356,7 +356,8 @@ public static AlignedStripe[] divideOneStripe(ErasureCodingPolicy ecPolicy, cells); // Step 3: merge into stripes - AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges); + AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges, + blockGroup, cellSize); // Step 4: calculate each chunk's position in destination buffer. Since the // whole read range is within a single stripe, the logic is simpler here. @@ -417,7 +418,8 @@ public static AlignedStripe[] divideByteRangeIntoStripes( cells); // Step 3: merge into at most 5 stripes - AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges); + AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges, + blockGroup, cellSize); // Step 4: calculate each chunk's position in destination buffer calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf); @@ -513,7 +515,8 @@ private static VerticalRange[] getRangesForInternalBlocks( * {@link AlignedStripe} instances. */ private static AlignedStripe[] mergeRangesForInternalBlocks( - ErasureCodingPolicy ecPolicy, VerticalRange[] ranges) { + ErasureCodingPolicy ecPolicy, VerticalRange[] ranges, + LocatedStripedBlock blockGroup, int cellSize) { int dataBlkNum = ecPolicy.getNumDataUnits(); int parityBlkNum = ecPolicy.getNumParityUnits(); List stripes = new ArrayList<>(); @@ -525,6 +528,17 @@ private static AlignedStripe[] mergeRangesForInternalBlocks( } } + // Add block group last cell offset in stripePoints if it is fall in to read + // offset range. + int lastCellIdxInBG = (int) (blockGroup.getBlockSize() / cellSize); + int idxInInternalBlk = lastCellIdxInBG / ecPolicy.getNumDataUnits(); + long lastCellEndOffset = (idxInInternalBlk * (long)cellSize) + + (blockGroup.getBlockSize() % cellSize); + if (stripePoints.first() < lastCellEndOffset + && stripePoints.last() > lastCellEndOffset) { + stripePoints.add(lastCellEndOffset); + } + long prev = -1; for (long point : stripePoints) { if (prev >= 0) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml index b5c14723d2fbf..1c3a5f044edf5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml @@ -236,4 +236,52 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> + + + parallel-tests + + + + org.apache.hadoop + hadoop-maven-plugins + + + parallel-tests-createdir + + parallel-tests-createdir + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + ${testsThreadCount} + false + ${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true + + ${testsThreadCount} + ${test.build.data}/${surefire.forkNumber} + ${test.build.dir}/${surefire.forkNumber} + ${hadoop.tmp.dir}/${surefire.forkNumber} + + + + + + ${test.build.data} + + + + + + fork-${surefire.forkNumber} + + + + + + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 462c81d2e811a..3b776bab109d3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -1357,7 +1357,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys { "dfs.namenode.send.qop.enabled"; public static final boolean DFS_NAMENODE_SEND_QOP_ENABLED_DEFAULT = false; - // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry + public static final String DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY = + "dfs.namenode.state.context.enabled"; + public static final boolean DFS_NAMENODE_STATE_CONTEXT_ENABLED_DEFAULT = false; + + // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry @Deprecated public static final String DFS_CLIENT_RETRY_POLICY_ENABLED_KEY = HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java index d160f61fc8f54..dc6cf3266a5fc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java @@ -181,6 +181,12 @@ public int getCapacity() { /** @return true if there is no datanode storage associated with the block */ abstract boolean hasNoStorage(); + /** + * Checks whether this block has a Provided replica. + * @return true if this block has a replica on Provided storage. + */ + abstract boolean isProvided(); + /** * Find specified DatanodeStorageInfo. * @return DatanodeStorageInfo or null if not found. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java index 149efc93091d8..7378e6f21b765 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java @@ -19,6 +19,7 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockType; @@ -80,6 +81,19 @@ boolean removeStorage(DatanodeStorageInfo storage) { return true; } + @Override + boolean isProvided() { + int len = getCapacity(); + for (int idx = 0; idx < len; idx++) { + DatanodeStorageInfo storage = getStorageInfo(idx); + if (storage != null + && storage.getStorageType().equals(StorageType.PROVIDED)) { + return true; + } + } + return false; + } + @Override public int numNodes() { assert this.storages != null : "BlockInfo is not initialized"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java index 8bc63c1214d28..16265ded88bca 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -244,6 +244,15 @@ final boolean hasNoStorage() { return true; } + /** + * Striped blocks on Provided Storage is not supported. All blocks on + * Provided storage are assumed to be "contiguous". + */ + @Override + boolean isProvided() { + return false; + } + /** * This class contains datanode storage information and block index in the * block group. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 6a62928700cbd..66802d29bc081 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -2009,6 +2009,7 @@ boolean hasEnoughEffectiveReplicas(BlockInfo block, (pendingReplicaNum > 0 || isPlacementPolicySatisfied(block)); } + @VisibleForTesting BlockReconstructionWork scheduleReconstruction(BlockInfo block, int priority) { // skip abandoned block or block reopened for append @@ -2053,7 +2054,9 @@ BlockReconstructionWork scheduleReconstruction(BlockInfo block, additionalReplRequired = requiredRedundancy - numReplicas.liveReplicas() - pendingNum; } else { - additionalReplRequired = 1; // Needed on a new rack + // Violates placement policy. Needed on a new rack or domain etc. + BlockPlacementStatus placementStatus = getBlockPlacementStatus(block); + additionalReplRequired = placementStatus.getAdditionalReplicasRequired(); } final BlockCollection bc = getBlockCollection(block); @@ -2086,20 +2089,6 @@ BlockReconstructionWork scheduleReconstruction(BlockInfo block, } } - private boolean isInNewRack(DatanodeDescriptor[] srcs, - DatanodeDescriptor target) { - LOG.debug("check if target {} increases racks, srcs={}", target, - Arrays.asList(srcs)); - for (DatanodeDescriptor src : srcs) { - if (!src.isDecommissionInProgress() && - src.getNetworkLocation().equals(target.getNetworkLocation())) { - LOG.debug("the target {} is in the same rack with src {}", target, src); - return false; - } - } - return true; - } - private boolean validateReconstructionWork(BlockReconstructionWork rw) { BlockInfo block = rw.getBlock(); int priority = rw.getPriority(); @@ -2125,10 +2114,16 @@ private boolean validateReconstructionWork(BlockReconstructionWork rw) { } DatanodeStorageInfo[] targets = rw.getTargets(); + BlockPlacementStatus placementStatus = getBlockPlacementStatus(block); if ((numReplicas.liveReplicas() >= requiredRedundancy) && - (!isPlacementPolicySatisfied(block)) ) { - if (!isInNewRack(rw.getSrcNodes(), targets[0].getDatanodeDescriptor())) { - // No use continuing, unless a new rack in this case + (!placementStatus.isPlacementPolicySatisfied())) { + BlockPlacementStatus newPlacementStatus = + getBlockPlacementStatus(block, targets); + if (!newPlacementStatus.isPlacementPolicySatisfied() && + (newPlacementStatus.getAdditionalReplicasRequired() >= + placementStatus.getAdditionalReplicasRequired())) { + // If the new targets do not meet the placement policy, or at least + // reduce the number of replicas needed, then no use continuing. return false; } // mark that the reconstruction work is to replicate internal block to a @@ -4562,7 +4557,25 @@ public boolean containsInvalidateBlock(final DatanodeInfo dn, } boolean isPlacementPolicySatisfied(BlockInfo storedBlock) { + return getBlockPlacementStatus(storedBlock, null) + .isPlacementPolicySatisfied(); + } + + BlockPlacementStatus getBlockPlacementStatus(BlockInfo storedBlock) { + return getBlockPlacementStatus(storedBlock, null); + } + + BlockPlacementStatus getBlockPlacementStatus(BlockInfo storedBlock, + DatanodeStorageInfo[] additionalStorage) { List liveNodes = new ArrayList<>(); + if (additionalStorage != null) { + // additionalNodes, are potential new targets for the block. If there are + // any passed, include them when checking the placement policy to see if + // the policy is met, when it may not have been met without these nodes. + for (DatanodeStorageInfo s : additionalStorage) { + liveNodes.add(getDatanodeDescriptorFromStorage(s)); + } + } Collection corruptNodes = corruptReplicas .getNodes(storedBlock); for (DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) { @@ -4570,7 +4583,22 @@ boolean isPlacementPolicySatisfied(BlockInfo storedBlock) { && storage.getState() == State.NORMAL) { // assume the policy is satisfied for blocks on PROVIDED storage // as long as the storage is in normal state. - return true; + return new BlockPlacementStatus() { + @Override + public boolean isPlacementPolicySatisfied() { + return true; + } + + @Override + public String getErrorDescription() { + return null; + } + + @Override + public int getAdditionalReplicasRequired() { + return 0; + } + }; } final DatanodeDescriptor cur = getDatanodeDescriptorFromStorage(storage); // Nodes under maintenance should be counted as valid replicas from @@ -4586,8 +4614,7 @@ boolean isPlacementPolicySatisfied(BlockInfo storedBlock) { .getPolicy(blockType); int numReplicas = blockType == STRIPED ? ((BlockInfoStriped) storedBlock) .getRealTotalBlockNum() : storedBlock.getReplication(); - return placementPolicy.verifyBlockPlacement(locs, numReplicas) - .isPlacementPolicySatisfied(); + return placementPolicy.verifyBlockPlacement(locs, numReplicas); } boolean isNeededReconstructionForMaintenance(BlockInfo storedBlock, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java index e2ac54a35373c..a227666b8719d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java @@ -39,4 +39,12 @@ public interface BlockPlacementStatus { */ public String getErrorDescription(); + /** + * Return the number of additional replicas needed to ensure the block + * placement policy is satisfied. + * @return The number of new replicas needed to satisify the placement policy + * or zero if no extra are needed + */ + int getAdditionalReplicasRequired(); + } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java index 75bb65d90147c..761214234c5a5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java @@ -45,4 +45,12 @@ public String getErrorDescription() { " more rack(s). Total number of racks in the cluster: " + totalRacks; } + @Override + public int getAdditionalReplicasRequired() { + if (isPlacementPolicySatisfied()) { + return 0; + } else { + return requiredRacks - currentRacks; + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithNodeGroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithNodeGroup.java index b98b3dac3621b..ac5a5b51d9c85 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithNodeGroup.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithNodeGroup.java @@ -78,4 +78,15 @@ public String getErrorDescription() { } return errorDescription.toString(); } + + @Override + public int getAdditionalReplicasRequired() { + if (isPlacementPolicySatisfied()) { + return 0; + } else { + int parent = parentBlockPlacementStatus.getAdditionalReplicasRequired(); + int child = requiredNodeGroups - currentNodeGroups.size(); + return Math.max(parent, child); + } + } } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java index 4b3c3cc383000..b839cede2bf3b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java @@ -85,4 +85,24 @@ public String getErrorDescription() { } return errorDescription.toString(); } -} \ No newline at end of file + + @Override + public int getAdditionalReplicasRequired() { + if (isPlacementPolicySatisfied()) { + return 0; + } else { + // It is possible for a block to have the correct number of upgrade + // domains, but only a single rack, or be on multiple racks, but only in + // one upgrade domain. + int parent = parentBlockPlacementStatus.getAdditionalReplicasRequired(); + int child; + + if (numberOfReplicas <= upgradeDomainFactor) { + child = numberOfReplicas - upgradeDomains.size(); + } else { + child = upgradeDomainFactor - upgradeDomains.size(); + } + return Math.max(parent, child); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java index 2ba943a07780d..e7da44e689a60 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java @@ -447,9 +447,14 @@ public void clearDirectory() throws IOException { throw new IOException("Cannot create directory " + curDir); } if (permission != null) { - Set permissions = - PosixFilePermissions.fromString(permission.toString()); - Files.setPosixFilePermissions(curDir.toPath(), permissions); + try { + Set permissions = + PosixFilePermissions.fromString(permission.toString()); + Files.setPosixFilePermissions(curDir.toPath(), permissions); + } catch (UnsupportedOperationException uoe) { + // Default to FileUtil for non posix file systems + FileUtil.setPermission(curDir, permission); + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java index d91c8575ef0c3..3a34aa7868483 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java @@ -27,6 +27,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_AUXILIARY_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_PATH_DEPTH; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_PATH_LENGTH; import static org.apache.hadoop.util.Time.now; @@ -447,6 +449,16 @@ public NameNodeRpcServer(Configuration conf, NameNode nn) } LOG.info("RPC server is binding to " + bindHost + ":" + rpcAddr.getPort()); + boolean enableStateContext = conf.getBoolean( + DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY, + DFS_NAMENODE_STATE_CONTEXT_ENABLED_DEFAULT); + LOG.info("Enable NameNode state context:" + enableStateContext); + + GlobalStateIdContext stateIdContext = null; + if (enableStateContext) { + stateIdContext = new GlobalStateIdContext((namesystem)); + } + clientRpcServer = new RPC.Builder(conf) .setProtocol( org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class) @@ -456,7 +468,7 @@ public NameNodeRpcServer(Configuration conf, NameNode nn) .setNumHandlers(handlerCount) .setVerbose(false) .setSecretManager(namesystem.getDelegationTokenSecretManager()) - .setAlignmentContext(new GlobalStateIdContext(namesystem)) + .setAlignmentContext(stateIdContext) .build(); // Add all the RPC protocols that the namenode implements diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java index b3ab32a4ddca2..04960e3c3e2ce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java @@ -434,7 +434,7 @@ static int run(DistributedFileSystem dfs, String[] argv, int idx) throws IOExcep private static final String commonUsageSummary = "\t[-report [-live] [-dead] [-decommissioning] " + "[-enteringmaintenance] [-inmaintenance]]\n" + - "\t[-safemode ]\n" + + "\t[-safemode ]\n" + "\t[-saveNamespace [-beforeShutdown]]\n" + "\t[-rollEdits]\n" + "\t[-restoreFailedStorage true|false|check]\n" + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index bb0a478438574..9b999643f7ed3 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -3302,6 +3302,17 @@ + + dfs.namenode.state.context.enabled + false + + Whether enable namenode sending back its current txnid back to client. + Setting this to true is required by Consistent Read from Standby feature. + But for regular cases, this should be set to false to avoid the overhead + of updating and maintaining this state. + + + dfs.namenode.ec.system.default.policy RS-6-3-1024k diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html index df6e4e8797f0a..05c04b5eeea82 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html @@ -278,7 +278,7 @@ -

+ @@ -420,7 +420,7 @@ {/DecomNodes}
{:else} -No nodes are decommissioning +No nodes are decommissioning. {/DecomNodes}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md index 07c384c1bcf1a..af1569c6d0114 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ObserverNameNode.md @@ -120,6 +120,20 @@ Deployment To enable consistent reads from Observer NameNode, you'll need to add a few configurations to your **hdfs-site.xml**: +* **dfs.namenode.state.context.enabled** - to enable NameNode to maintain + and update server state and id. + + This will lead to NameNode creating alignment context instance, which + keeps track of current server state id. Server state id will be carried + back to client. It is disabled by default to optimize performance of + Observer read cases. But this is **required to be turned on** + for the Observer NameNode feature. + + + dfs.namenode.state.context.enabled + true + + * **dfs.ha.tail-edits.in-progress** - to enable fast tailing on in-progress edit logs. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java index adc29a1e0030d..36f2eb2d8050f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java @@ -556,17 +556,24 @@ public static boolean allBlockReplicasCorrupt(MiniDFSCluster cluster, } } + public static void waitForReplication(MiniDFSCluster cluster, ExtendedBlock b, + int racks, int replicas, int neededReplicas) + throws TimeoutException, InterruptedException { + waitForReplication(cluster, b, racks, replicas, neededReplicas, 0); + } + /* * Wait up to 20s for the given block to be replicated across * the requested number of racks, with the requested number of * replicas, and the requested number of replicas still needed. */ public static void waitForReplication(MiniDFSCluster cluster, ExtendedBlock b, - int racks, int replicas, int neededReplicas) + int racks, int replicas, int neededReplicas, int neededDomains) throws TimeoutException, InterruptedException { int curRacks = 0; int curReplicas = 0; int curNeededReplicas = 0; + int curDomains = 0; int count = 0; final int ATTEMPTS = 20; @@ -577,17 +584,21 @@ public static void waitForReplication(MiniDFSCluster cluster, ExtendedBlock b, curRacks = r[0]; curReplicas = r[1]; curNeededReplicas = r[2]; + curDomains = r[3]; count++; } while ((curRacks != racks || curReplicas != replicas || - curNeededReplicas != neededReplicas) && count < ATTEMPTS); + curNeededReplicas != neededReplicas || + (neededDomains != 0 && curDomains != neededDomains)) + && count < ATTEMPTS); if (count == ATTEMPTS) { throw new TimeoutException("Timed out waiting for replication." + " Needed replicas = "+neededReplicas + " Cur needed replicas = "+curNeededReplicas + " Replicas = "+replicas+" Cur replicas = "+curReplicas - + " Racks = "+racks+" Cur racks = "+curRacks); + + " Racks = "+racks+" Cur racks = "+curRacks + + " Domains = "+neededDomains+" Cur domains = "+curDomains); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index 604a8c6b0eb88..8c2367d38b706 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -19,8 +19,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.LocatedBlock; @@ -561,6 +564,50 @@ public void testCloseDoesNotAllocateNewBuffer() throws Exception { } } + @Test + public void testReadWhenLastIncompleteCellComeInToDecodeAlignedStripe() + throws IOException { + DataNodeProperties stopDataNode = null; + try { + cluster.waitActive(); + ErasureCodingPolicy policy = getEcPolicy(); + DistributedFileSystem filesystem = cluster.getFileSystem(); + filesystem.enableErasureCodingPolicy(policy.getName()); + Path dir = new Path("/tmp"); + filesystem.mkdirs(dir); + filesystem.getClient().setErasureCodingPolicy(dir.toString(), + policy.getName()); + Path f = new Path(dir, "file"); + + //1. File with one stripe, last data cell should be half filed. + long fileLength = (policy.getCellSize() * policy.getNumDataUnits()) + - (policy.getCellSize() / 2); + DFSTestUtil.createFile(filesystem, f, fileLength, (short) 1, 0); + + //2. Stop first DN from stripe. + LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations( + f.toString(), 0, fileLength); + LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0)); + final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(bg, + cellSize, dataBlocks, parityBlocks); + cluster.stopDataNode(blocks[0].getLocations()[0].getName()); + + //3. Do pread for fist cell, reconstruction should happen + try (FSDataInputStream in = filesystem.open(f)) { + DFSStripedInputStream stripedIn = (DFSStripedInputStream) in + .getWrappedStream(); + byte[] b = new byte[policy.getCellSize()]; + stripedIn.read(0, b, 0, policy.getCellSize()); + } + } catch (HadoopIllegalArgumentException e) { + fail(e.getMessage()); + } finally { + if (stopDataNode != null) { + cluster.restartDataNode(stopDataNode, true); + } + } + } + /** * Empties the pool for the specified buffer type, for the current ecPolicy. *

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStateAlignmentContextWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStateAlignmentContextWithHA.java index 3dbeea77693a8..3056b43679db1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStateAlignmentContextWithHA.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStateAlignmentContextWithHA.java @@ -94,6 +94,7 @@ public static void startUpCluster() throws IOException { CONF.setBoolean(String.format( "fs.%s.impl.disable.cache", HdfsConstants.HDFS_URI_SCHEME), true); CONF.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, NUMDATANODES); + CONF.setBoolean(DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY, true); qjmhaCluster = HATestUtil.setUpObserverCluster(CONF, 1, NUMDATANODES, true); cluster = qjmhaCluster.getDfsCluster(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java index ae61f8cac437a..4d6f202c9d44d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java @@ -81,7 +81,8 @@ public static void updateState(final BlockManager blockManager) { /** * @return a tuple of the replica state (number racks, number live - * replicas, and number needed replicas) for the given block. + * replicas, number needed replicas and number of UpgradeDomains) for the + * given block. */ public static int[] getReplicaInfo(final FSNamesystem namesystem, final Block b) { final BlockManager bm = namesystem.getBlockManager(); @@ -90,7 +91,8 @@ public static int[] getReplicaInfo(final FSNamesystem namesystem, final Block b) final BlockInfo storedBlock = bm.getStoredBlock(b); return new int[]{getNumberOfRacks(bm, b), bm.countNodes(storedBlock).liveReplicas(), - bm.neededReconstruction.contains(storedBlock) ? 1 : 0}; + bm.neededReconstruction.contains(storedBlock) ? 1 : 0, + getNumberOfDomains(bm, b)}; } finally { namesystem.readUnlock(); } @@ -120,6 +122,30 @@ private static int getNumberOfRacks(final BlockManager blockManager, return rackSet.size(); } + /** + * @return the number of UpgradeDomains over which a given block is replicated + * decommissioning/decommissioned nodes are not counted. corrupt replicas + * are also ignored. + */ + private static int getNumberOfDomains(final BlockManager blockManager, + final Block b) { + final Set domSet = new HashSet(0); + final Collection corruptNodes = + getCorruptReplicas(blockManager).getNodes(b); + for(DatanodeStorageInfo storage : blockManager.blocksMap.getStorages(b)) { + final DatanodeDescriptor cur = storage.getDatanodeDescriptor(); + if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { + if ((corruptNodes == null) || !corruptNodes.contains(cur)) { + String domain = cur.getUpgradeDomain(); + if (domain != null && !domSet.contains(domain)) { + domSet.add(domain); + } + } + } + } + return domSet.size(); + } + /** * @return redundancy monitor thread instance from block manager. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java index fa0dd70a7e0d8..3c5c5d9fb2fee 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java @@ -19,7 +19,10 @@ import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID; import static org.hamcrest.core.Is.is; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import org.apache.hadoop.fs.StorageType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hdfs.DFSTestUtil; @@ -64,6 +67,39 @@ public void testAddStorage() throws Exception { Assert.assertEquals(storage, blockInfo.getStorageInfo(0)); } + @Test + public void testAddProvidedStorage() throws Exception { + // block with only provided storage + BlockInfo blockInfo = new BlockInfoContiguous((short) 3); + DatanodeStorageInfo providedStorage = mock(DatanodeStorageInfo.class); + when(providedStorage.getStorageType()).thenReturn(StorageType.PROVIDED); + boolean added = blockInfo.addStorage(providedStorage, blockInfo); + Assert.assertTrue(added); + Assert.assertEquals(providedStorage, blockInfo.getStorageInfo(0)); + Assert.assertTrue(blockInfo.isProvided()); + } + + @Test + public void testAddTwoStorageTypes() throws Exception { + // block with only disk storage + BlockInfo blockInfo = new BlockInfoContiguous((short) 3); + DatanodeStorageInfo diskStorage = mock(DatanodeStorageInfo.class); + DatanodeDescriptor mockDN = mock(DatanodeDescriptor.class); + when(diskStorage.getDatanodeDescriptor()).thenReturn(mockDN); + when(diskStorage.getStorageType()).thenReturn(StorageType.DISK); + boolean added = blockInfo.addStorage(diskStorage, blockInfo); + Assert.assertTrue(added); + Assert.assertEquals(diskStorage, blockInfo.getStorageInfo(0)); + Assert.assertFalse(blockInfo.isProvided()); + + // now add provided storage + DatanodeStorageInfo providedStorage = mock(DatanodeStorageInfo.class); + when(providedStorage.getStorageType()).thenReturn(StorageType.PROVIDED); + added = blockInfo.addStorage(providedStorage, blockInfo); + Assert.assertTrue(added); + Assert.assertTrue(blockInfo.isProvided()); + } + @Test public void testReplaceStorage() throws Exception { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusDefault.java new file mode 100644 index 0000000000000..6b0733452caa8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusDefault.java @@ -0,0 +1,57 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.blockmanagement; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; +import org.junit.Test; + +/** + * Unit tests to validate the BlockPlacementStatusDefault policy, focusing on + * the getAdditionAlReplicasRequired method. + */ +public class TestBlockPlacementStatusDefault { + + @Test + public void testIsPolicySatisfiedCorrectly() { + // 2 current racks and 2 expected + BlockPlacementStatusDefault bps = + new BlockPlacementStatusDefault(2, 2, 5); + assertTrue(bps.isPlacementPolicySatisfied()); + assertEquals(0, bps.getAdditionalReplicasRequired()); + + // 1 current rack and 2 expected + bps = + new BlockPlacementStatusDefault(1, 2, 5); + assertFalse(bps.isPlacementPolicySatisfied()); + assertEquals(1, bps.getAdditionalReplicasRequired()); + + // 3 current racks and 2 expected + bps = + new BlockPlacementStatusDefault(3, 2, 5); + assertTrue(bps.isPlacementPolicySatisfied()); + assertEquals(0, bps.getAdditionalReplicasRequired()); + + // 1 current rack and 2 expected, but only 1 rack on the cluster + bps = + new BlockPlacementStatusDefault(1, 2, 1); + assertTrue(bps.isPlacementPolicySatisfied()); + assertEquals(0, bps.getAdditionalReplicasRequired()); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusWithUpgradeDomain.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusWithUpgradeDomain.java index bfff9328a66a9..1e0fb76d92a20 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusWithUpgradeDomain.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusWithUpgradeDomain.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -49,11 +50,13 @@ public void setup() { @Test public void testIsPolicySatisfiedParentFalse() { when(bpsd.isPlacementPolicySatisfied()).thenReturn(false); + when(bpsd.getAdditionalReplicasRequired()).thenReturn(1); BlockPlacementStatusWithUpgradeDomain bps = new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 3, 3); // Parent policy is not satisfied but upgrade domain policy is assertFalse(bps.isPlacementPolicySatisfied()); + assertEquals(1, bps.getAdditionalReplicasRequired()); } @Test @@ -63,21 +66,73 @@ public void testIsPolicySatisfiedAllEqual() { // Number of domains, replicas and upgradeDomainFactor is equal and parent // policy is satisfied assertTrue(bps.isPlacementPolicySatisfied()); + assertEquals(0, bps.getAdditionalReplicasRequired()); } @Test - public void testIsPolicySatisifedSmallDomains() { + public void testIsPolicySatisfiedSmallDomains() { // Number of domains is less than replicas but equal to factor BlockPlacementStatusWithUpgradeDomain bps = new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 4, 3); assertTrue(bps.isPlacementPolicySatisfied()); + assertEquals(0, bps.getAdditionalReplicasRequired()); // Same as above but replicas is greater than factor bps = new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 4, 2); assertTrue(bps.isPlacementPolicySatisfied()); + assertEquals(0, bps.getAdditionalReplicasRequired()); // Number of domains is less than replicas and factor bps = new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 4, 4); assertFalse(bps.isPlacementPolicySatisfied()); + assertEquals(1, bps.getAdditionalReplicasRequired()); } -} \ No newline at end of file + + @Test + public void testIsPolicySatisfiedSmallReplicas() { + // Replication factor 1 file + upgradeDomains.clear(); + upgradeDomains.add("1"); + BlockPlacementStatusWithUpgradeDomain bps = + new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 1, 3); + assertTrue(bps.isPlacementPolicySatisfied()); + assertEquals(0, bps.getAdditionalReplicasRequired()); + + // Replication factor 2 file, but one domain + bps = + new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 2, 3); + assertFalse(bps.isPlacementPolicySatisfied()); + assertEquals(1, bps.getAdditionalReplicasRequired()); + + // Replication factor 2 file, but two domains + upgradeDomains.add("2"); + bps = + new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 2, 3); + assertTrue(bps.isPlacementPolicySatisfied()); + assertEquals(0, bps.getAdditionalReplicasRequired()); + } + + @Test + public void testPolicyIsNotSatisfiedInsufficientDomains() { + // Insufficient Domains - 1 domain, replication factor 3 + upgradeDomains.clear(); + upgradeDomains.add("1"); + BlockPlacementStatusWithUpgradeDomain bps = + new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 3, 3); + assertFalse(bps.isPlacementPolicySatisfied()); + assertEquals(2, bps.getAdditionalReplicasRequired()); + + // One domain, replication factor 2 file + bps = + new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 2, 3); + assertFalse(bps.isPlacementPolicySatisfied()); + assertEquals(1, bps.getAdditionalReplicasRequired()); + + // 2 domains, replication factor 3 + upgradeDomains.add("2"); + bps = + new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 3, 3); + assertFalse(bps.isPlacementPolicySatisfied()); + assertEquals(1, bps.getAdditionalReplicasRequired()); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java index 17043672840f0..c0cf7ea396384 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java @@ -18,14 +18,10 @@ package org.apache.hadoop.hdfs.server.blockmanagement; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.fail; -import static org.junit.Assert.assertTrue; - +import java.io.IOException; import java.util.ArrayList; +import java.util.List; import java.util.concurrent.TimeoutException; - import org.apache.hadoop.hdfs.server.datanode.InternalDataNodeTestUtils; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.slf4j.Logger; @@ -48,6 +44,8 @@ import org.junit.Test; import org.slf4j.event.Level; +import static org.junit.Assert.*; + public class TestBlocksWithNotEnoughRacks { public static final Logger LOG = LoggerFactory.getLogger(TestBlocksWithNotEnoughRacks.class); @@ -545,4 +543,105 @@ public void testNodeDecomissionWithOverreplicationRespectsRackPolicy() hostsFileWriter.cleanup(); } } + + @Test + public void testMultipleReplicasScheduledForUpgradeDomain() throws Exception { + Configuration conf = getConf(); + final short replicationFactor = 3; + final Path filePath = new Path("/testFile"); + + conf.set("dfs.block.replicator.classname", + "org.apache.hadoop.hdfs.server.blockmanagement." + + "BlockPlacementPolicyWithUpgradeDomain"); + MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(6).build(); + cluster.waitClusterUp(); + + List dnDescriptors = getDnDescriptors(cluster); + + try { + // Create a file with one block with a replication factor of 3 + // No upgrade domains are set. + final FileSystem fs = cluster.getFileSystem(); + DFSTestUtil.createFile(fs, filePath, 1L, replicationFactor, 1L); + ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath); + + BlockManager bm = cluster.getNamesystem().getBlockManager(); + BlockInfo storedBlock = bm.getStoredBlock(b.getLocalBlock()); + + // The block should be replicated OK - so Reconstruction Work will be null + BlockReconstructionWork work = bm.scheduleReconstruction(storedBlock, 2); + assertNull(work); + // Set the upgradeDomain to "3" for the 3 nodes hosting the block. + // Then alternately set the remaining 3 nodes to have an upgradeDomain + // of 0 or 1 giving a total of 3 upgradeDomains. + for (int i=0; i dnDescriptors = getDnDescriptors(cluster); + for (int i=0; i < dnDescriptors.size(); i++) { + dnDescriptors.get(i).setUpgradeDomain(Integer.toString(i%3)); + } + try { + final FileSystem fs = cluster.getFileSystem(); + DFSTestUtil.createFile(fs, filePath, 1L, (short)1, 1L); + ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath); + fs.setReplication(filePath, replicationFactor); + DFSTestUtil.waitForReplication(cluster, b, 2, replicationFactor, 0, 3); + } finally { + cluster.shutdown(); + } + } + + private List getDnDescriptors(MiniDFSCluster cluster) + throws IOException { + List dnDesc = new ArrayList<>(); + DatanodeManager dnManager = cluster.getNamesystem().getBlockManager() + .getDatanodeManager(); + for (DataNode dn : cluster.getDataNodes()) { + DatanodeDescriptor d = dnManager.getDatanode(dn.getDatanodeUuid()); + if (d == null) { + throw new IOException("DatanodeDescriptor not found for DN "+ + dn.getDatanodeUuid()); + } + dnDesc.add(d); + } + return dnDesc; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java index a6fac80b9bcc9..96067858c9df0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.namenode.ha; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -71,6 +72,7 @@ public class TestConsistentReadsObserver { @BeforeClass public static void startUpCluster() throws Exception { conf = new Configuration(); + conf.setBoolean(DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY, true); // disable fast tailing here because this test's assertions are based on the // timing of explicitly called rollEditLogAndTail. Although this means this // test takes some time to run diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestMultiObserverNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestMultiObserverNode.java index a8e124568d982..a0913e4c5e447 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestMultiObserverNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestMultiObserverNode.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.namenode.ha; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY; import static org.junit.Assert.assertTrue; import java.io.IOException; @@ -46,6 +47,7 @@ public class TestMultiObserverNode { @BeforeClass public static void startUpCluster() throws Exception { conf = new Configuration(); + conf.setBoolean(DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY, true); qjmhaCluster = HATestUtil.setUpObserverCluster(conf, 2, 0, true); dfsCluster = qjmhaCluster.getDfsCluster(); dfs = HATestUtil.configureObserverReadFs( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestObserverNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestObserverNode.java index 20e0bbdfcddc0..b89a157e5c165 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestObserverNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestObserverNode.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.namenode.ha; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY; import static org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getServiceState; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -77,6 +78,7 @@ public class TestObserverNode { @BeforeClass public static void startUpCluster() throws Exception { conf = new Configuration(); + conf.setBoolean(DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY, true); qjmhaCluster = HATestUtil.setUpObserverCluster(conf, 1, 0, true); dfsCluster = qjmhaCluster.getDfsCluster(); } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java index 5b6d620c8de9f..ebe4477955a5d 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java @@ -94,6 +94,7 @@ import java.io.IOException; import java.net.URI; import java.security.InvalidKeyException; +import java.security.SecureRandom; import java.util.*; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -587,7 +588,7 @@ public OzoneOutputStream createKey( if(Boolean.valueOf(metadata.get(OzoneConsts.GDPR_FLAG))){ try{ - GDPRSymmetricKey gKey = new GDPRSymmetricKey(); + GDPRSymmetricKey gKey = new GDPRSymmetricKey(new SecureRandom()); metadata.putAll(gKey.getKeyDetails()); }catch (Exception e) { if(e instanceof InvalidKeyException && diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java index ded99fe77fa61..8e129c9d23011 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java @@ -18,32 +18,37 @@ package org.apache.hadoop.ozone; import com.google.common.base.Joiner; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStream; import java.net.InetSocketAddress; import java.nio.charset.StandardCharsets; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; +import java.security.SecureRandom; import java.util.Collection; import java.util.Collections; import java.util.Optional; -import java.util.zip.GZIPOutputStream; +import java.util.stream.Collectors; import com.google.common.base.Strings; -import org.apache.commons.compress.archivers.tar.TarArchiveEntry; + +import org.apache.commons.compress.archivers.ArchiveEntry; +import org.apache.commons.compress.archivers.ArchiveOutputStream; import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; +import org.apache.commons.compress.compressors.CompressorException; +import org.apache.commons.compress.compressors.CompressorOutputStream; +import org.apache.commons.compress.compressors.CompressorStreamFactory; import org.apache.commons.compress.utils.IOUtils; -import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.scm.HddsServerUtil; import org.apache.hadoop.hdds.server.ServerUtils; +import org.apache.hadoop.hdds.utils.db.DBCheckpoint; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; @@ -73,6 +78,8 @@ */ public final class OmUtils { public static final Logger LOG = LoggerFactory.getLogger(OmUtils.class); + private static final SecureRandom SRAND = new SecureRandom(); + private static byte[] randomBytes = new byte[32]; private OmUtils() { } @@ -274,9 +281,9 @@ public static byte[] getMD5Digest(String input) throws IOException { public static byte[] getSHADigest() throws IOException { try { + SRAND.nextBytes(randomBytes); MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH); - return sha.digest(RandomStringUtils.random(32) - .getBytes(StandardCharsets.UTF_8)); + return sha.digest(randomBytes); } catch (NoSuchAlgorithmException ex) { throw new IOException("Error creating an instance of SHA-256 digest.\n" + "This could possibly indicate a faulty JRE"); @@ -344,78 +351,64 @@ public static Collection emptyAsSingletonNull(Collection } /** - * Given a source directory, create a tar.gz file from it. - * - * @param sourcePath the path to the directory to be archived. - * @return tar.gz file + * Write OM DB Checkpoint to an output stream as a compressed file (tgz). + * @param checkpoint checkpoint file + * @param destination desination output stream. * @throws IOException */ - public static File createTarFile(Path sourcePath) throws IOException { - TarArchiveOutputStream tarOs = null; - try { - String sourceDir = sourcePath.toString(); - String fileName = sourceDir.concat(".tar.gz"); - FileOutputStream fileOutputStream = new FileOutputStream(fileName); - GZIPOutputStream gzipOutputStream = - new GZIPOutputStream(new BufferedOutputStream(fileOutputStream)); - tarOs = new TarArchiveOutputStream(gzipOutputStream); - File folder = new File(sourceDir); - File[] filesInDir = folder.listFiles(); - if (filesInDir != null) { - for (File file : filesInDir) { - addFilesToArchive(file.getName(), file, tarOs); + public static void writeOmDBCheckpointToStream(DBCheckpoint checkpoint, + OutputStream destination) + throws IOException { + + try (CompressorOutputStream gzippedOut = new CompressorStreamFactory() + .createCompressorOutputStream(CompressorStreamFactory.GZIP, + destination)) { + + try (ArchiveOutputStream archiveOutputStream = + new TarArchiveOutputStream(gzippedOut)) { + + Path checkpointPath = checkpoint.getCheckpointLocation(); + for (Path path : Files.list(checkpointPath) + .collect(Collectors.toList())) { + if (path != null) { + Path fileName = path.getFileName(); + if (fileName != null) { + includeFile(path.toFile(), fileName.toString(), + archiveOutputStream); + } + } } } - return new File(fileName); - } finally { - try { - org.apache.hadoop.io.IOUtils.closeStream(tarOs); - } catch (Exception e) { - LOG.error("Exception encountered when closing " + - "TAR file output stream: " + e); - } + } catch (CompressorException e) { + throw new IOException( + "Can't compress the checkpoint: " + + checkpoint.getCheckpointLocation(), e); } } - private static void addFilesToArchive(String source, File file, - TarArchiveOutputStream - tarFileOutputStream) + private static void includeFile(File file, String entryName, + ArchiveOutputStream archiveOutputStream) throws IOException { - tarFileOutputStream.putArchiveEntry(new TarArchiveEntry(file, source)); - if (file.isFile()) { - FileInputStream fileInputStream = new FileInputStream(file); - BufferedInputStream bufferedInputStream = - new BufferedInputStream(fileInputStream); - IOUtils.copy(bufferedInputStream, tarFileOutputStream); - tarFileOutputStream.closeArchiveEntry(); - fileInputStream.close(); - } else if (file.isDirectory()) { - tarFileOutputStream.closeArchiveEntry(); - File[] filesInDir = file.listFiles(); - if (filesInDir != null) { - for (File cFile : filesInDir) { - addFilesToArchive(cFile.getAbsolutePath(), cFile, - tarFileOutputStream); - } - } + ArchiveEntry archiveEntry = + archiveOutputStream.createArchiveEntry(file, entryName); + archiveOutputStream.putArchiveEntry(archiveEntry); + try (FileInputStream fis = new FileInputStream(file)) { + IOUtils.copy(fis, archiveOutputStream); } + archiveOutputStream.closeArchiveEntry(); } /** * If a OM conf is only set with key suffixed with OM Node ID, return the * set value. - * @return null if base conf key is set, otherwise the value set for - * key suffixed with Node ID. + * @return if the value is set for key suffixed with OM Node ID, return the + * value, else return null. */ public static String getConfSuffixedWithOMNodeId(Configuration conf, String confKey, String omServiceID, String omNodeId) { - String confValue = conf.getTrimmed(confKey); - if (StringUtils.isNotEmpty(confValue)) { - return null; - } String suffixedConfKey = OmUtils.addKeySuffixes( confKey, omServiceID, omNodeId); - confValue = conf.getTrimmed(suffixedConfKey); + String confValue = conf.getTrimmed(suffixedConfKey); if (StringUtils.isNotEmpty(confValue)) { return confValue; } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java index 1d80f97a9b6a0..673d26ade6688 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java @@ -30,7 +30,7 @@ import org.apache.hadoop.ozone.om.helpers.S3SecretValue; import org.apache.hadoop.ozone.om.lock.OzoneManagerLock; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .VolumeList; + .UserVolumeInfo; import org.apache.hadoop.ozone.security.OzoneTokenIdentifier; import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.hdds.utils.db.Table; @@ -225,7 +225,7 @@ List listVolumes(String userName, String prefix, * * @return UserTable. */ - Table getUserTable(); + Table getUserTable(); /** * Returns the Volume Table. diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/VolumeListCodec.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/UserVolumeInfoCodec.java similarity index 78% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/VolumeListCodec.java rename to hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/UserVolumeInfoCodec.java index c8190837cb398..2545454a16fd5 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/VolumeListCodec.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/codec/UserVolumeInfoCodec.java @@ -18,31 +18,32 @@ package org.apache.hadoop.ozone.om.codec; import java.io.IOException; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList; +import org.apache.hadoop.ozone.protocol.proto + .OzoneManagerProtocolProtos.UserVolumeInfo; import org.apache.hadoop.hdds.utils.db.Codec; import com.google.common.base.Preconditions; import com.google.protobuf.InvalidProtocolBufferException; /** - * Codec to encode VolumeList as byte array. + * Codec to encode UserVolumeInfo as byte array. */ -public class VolumeListCodec implements Codec { +public class UserVolumeInfoCodec implements Codec { @Override - public byte[] toPersistedFormat(VolumeList object) throws IOException { + public byte[] toPersistedFormat(UserVolumeInfo object) throws IOException { Preconditions .checkNotNull(object, "Null object can't be converted to byte array."); return object.toByteArray(); } @Override - public VolumeList fromPersistedFormat(byte[] rawData) throws IOException { + public UserVolumeInfo fromPersistedFormat(byte[] rawData) throws IOException { Preconditions .checkNotNull(rawData, "Null byte array can't converted to real object."); try { - return VolumeList.parseFrom(rawData); + return UserVolumeInfo.parseFrom(rawData); } catch (InvalidProtocolBufferException e) { throw new IllegalArgumentException( "Can't encode the the raw data from the byte array", e); diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/GDPRSymmetricKey.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/GDPRSymmetricKey.java index b5e69091196b7..0fd6b08bccd6f 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/GDPRSymmetricKey.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/GDPRSymmetricKey.java @@ -20,6 +20,7 @@ import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.ozone.OzoneConsts; +import java.security.SecureRandom; import java.util.HashMap; import java.util.Map; @@ -48,10 +49,11 @@ public Cipher getCipher() { * Default constructor creates key with default values. * @throws Exception */ - public GDPRSymmetricKey() throws Exception { + public GDPRSymmetricKey(SecureRandom secureRandom) throws Exception { algorithm = OzoneConsts.GDPR_ALGORITHM_NAME; - secret = RandomStringUtils - .randomAlphabetic(OzoneConsts.GDPR_DEFAULT_RANDOM_SECRET_LENGTH); + secret = RandomStringUtils.random( + OzoneConsts.GDPR_DEFAULT_RANDOM_SECRET_LENGTH, + 0, 0, true, true, null, secureRandom); this.secretKey = new SecretKeySpec( secret.getBytes(OzoneConsts.GDPR_CHARSET), algorithm); this.cipher = Cipher.getInstance(algorithm); diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java index 52e6d79d1e153..7e03095cdc45c 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneDelegationTokenSecretManager.java @@ -84,13 +84,16 @@ public class OzoneDelegationTokenSecretManager * milliseconds * @param dtRemoverScanInterval how often the tokens are scanned for expired * tokens in milliseconds + * @param certClient certificate client to SCM CA */ public OzoneDelegationTokenSecretManager(OzoneConfiguration conf, long tokenMaxLifetime, long tokenRenewInterval, long dtRemoverScanInterval, Text service, - S3SecretManager s3SecretManager) throws IOException { + S3SecretManager s3SecretManager, CertificateClient certClient) + throws IOException { super(new SecurityConfig(conf), tokenMaxLifetime, tokenRenewInterval, service, LOG); + setCertClient(certClient); currentTokens = new ConcurrentHashMap(); this.tokenRemoverScanInterval = dtRemoverScanInterval; this.s3SecretManager = (S3SecretManagerImpl) s3SecretManager; diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneSecretManager.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneSecretManager.java index 45d6e6613b750..78f0565b81dc7 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneSecretManager.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/security/OzoneSecretManager.java @@ -70,6 +70,7 @@ public abstract class OzoneSecretManager * @param tokenRenewInterval how often the tokens must be renewed in * milliseconds * @param service name of service + * @param logger logger for the secret manager */ public OzoneSecretManager(SecurityConfig secureConf, long tokenMaxLifetime, long tokenRenewInterval, Text service, Logger logger) { @@ -188,7 +189,7 @@ public String formatTokenId(T id) { public synchronized void start(CertificateClient client) throws IOException { Preconditions.checkState(!isRunning()); - this.certClient = client; + setCertClient(client); updateCurrentKey(new KeyPair(certClient.getPublicKey(), certClient.getPrivateKey())); setIsRunning(true); @@ -247,5 +248,9 @@ public AtomicInteger getTokenSequenceNumber() { public CertificateClient getCertClient() { return certClient; } + + public void setCertClient(CertificateClient client) { + this.certClient = client; + } } diff --git a/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto index 2b119a68da0ef..d82fdf2a8e263 100644 --- a/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto +++ b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto @@ -352,7 +352,7 @@ message CreateVolumeResponse { } -message VolumeList { +message UserVolumeInfo { repeated string volumeNames = 1; optional uint64 objectID = 2; optional uint64 updateID = 3; diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/TestOmUtils.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/TestOmUtils.java index a788d0cb3a5fb..ce743fead31a8 100644 --- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/TestOmUtils.java +++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/TestOmUtils.java @@ -19,31 +19,39 @@ package org.apache.hadoop.ozone; import org.apache.commons.io.FileUtils; +import org.apache.hadoop.hdds.utils.db.DBCheckpoint; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.om.OMConfigKeys; -import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; import org.junit.rules.Timeout; import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Path; import java.nio.file.Paths; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; /** * Unit tests for {@link OmUtils}. */ public class TestOmUtils { + + @Rule + public TemporaryFolder folder = new TemporaryFolder(); + @Rule public Timeout timeout = new Timeout(60_000); @@ -96,22 +104,13 @@ public void testNoOmDbDirConfigured() { } @Test - public void testCreateTarFile() throws Exception { + public void testWriteCheckpointToOutputStream() throws Exception { - File tempSnapshotDir = null; FileInputStream fis = null; FileOutputStream fos = null; - File tarFile = null; try { - String testDirName = System.getProperty("java.io.tmpdir"); - if (!testDirName.endsWith("/")) { - testDirName += "/"; - } - testDirName += "TestCreateTarFile_Dir" + System.currentTimeMillis(); - tempSnapshotDir = new File(testDirName); - tempSnapshotDir.mkdirs(); - + String testDirName = folder.newFolder().getAbsolutePath(); File file = new File(testDirName + "/temp1.txt"); FileWriter writer = new FileWriter(file); writer.write("Test data 1"); @@ -122,14 +121,60 @@ public void testCreateTarFile() throws Exception { writer.write("Test data 2"); writer.close(); - tarFile = OmUtils.createTarFile(Paths.get(testDirName)); - Assert.assertNotNull(tarFile); - + File outputFile = + new File(Paths.get(testDirName, "output_file.tgz").toString()); + TestDBCheckpoint dbCheckpoint = new TestDBCheckpoint( + Paths.get(testDirName)); + OmUtils.writeOmDBCheckpointToStream(dbCheckpoint, + new FileOutputStream(outputFile)); + assertNotNull(outputFile); } finally { IOUtils.closeStream(fis); IOUtils.closeStream(fos); - FileUtils.deleteDirectory(tempSnapshotDir); - FileUtils.deleteQuietly(tarFile); } } + +} + +class TestDBCheckpoint implements DBCheckpoint { + + private Path checkpointFile; + + TestDBCheckpoint(Path checkpointFile) { + this.checkpointFile = checkpointFile; + } + + @Override + public Path getCheckpointLocation() { + return checkpointFile; + } + + @Override + public long getCheckpointTimestamp() { + return 0; + } + + @Override + public long getLatestSequenceNumber() { + return 0; + } + + @Override + public long checkpointCreationTimeTaken() { + return 0; + } + + @Override + public void cleanupCheckpoint() throws IOException { + FileUtils.deleteDirectory(checkpointFile.toFile()); + } + + @Override + public void setRatisSnapshotIndex(long omRatisSnapshotIndex) { + } + + @Override + public long getRatisSnapshotIndex() { + return 0; + } } diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/security/TestGDPRSymmetricKey.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/security/TestGDPRSymmetricKey.java index e0fdc90ceed58..39c622043ba2e 100644 --- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/security/TestGDPRSymmetricKey.java +++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/security/TestGDPRSymmetricKey.java @@ -21,6 +21,8 @@ import org.junit.Assert; import org.junit.Test; +import java.security.SecureRandom; + /** * Tests GDPRSymmetricKey structure. */ @@ -28,7 +30,7 @@ public class TestGDPRSymmetricKey { @Test public void testKeyGenerationWithDefaults() throws Exception { - GDPRSymmetricKey gkey = new GDPRSymmetricKey(); + GDPRSymmetricKey gkey = new GDPRSymmetricKey(new SecureRandom()); Assert.assertTrue(gkey.getCipher().getAlgorithm() .equalsIgnoreCase(OzoneConsts.GDPR_ALGORITHM_NAME)); diff --git a/hadoop-ozone/dev-support/checks/blockade.sh b/hadoop-ozone/dev-support/checks/blockade.sh index f8b25c176ac41..a48d2b592ba24 100755 --- a/hadoop-ozone/dev-support/checks/blockade.sh +++ b/hadoop-ozone/dev-support/checks/blockade.sh @@ -21,7 +21,7 @@ OZONE_VERSION=$(grep "" "$DIR/../../pom.xml" | sed 's/<[^>]*>//g' cd "$DIR/../../dist/target/ozone-$OZONE_VERSION/tests" || exit 1 source ${DIR}/../../dist/target/ozone-${OZONE_VERSION}/compose/ozoneblockade/.env -export HADOOP_RUNNER_VERSION +export OZONE_RUNNER_VERSION export HDDS_VERSION python -m pytest -s blockade diff --git a/hadoop-ozone/dev-support/checks/integration.sh b/hadoop-ozone/dev-support/checks/integration.sh index ccd499d432192..52e35765952e0 100755 --- a/hadoop-ozone/dev-support/checks/integration.sh +++ b/hadoop-ozone/dev-support/checks/integration.sh @@ -19,7 +19,7 @@ cd "$DIR/../../.." || exit 1 export MAVEN_OPTS="-Xmx4096m" mvn -B install -f pom.ozone.xml -DskipTests mvn -B -fn test -f pom.ozone.xml -pl :hadoop-ozone-integration-test,:hadoop-ozone-filesystem,:hadoop-ozone-tools \ - -Dtest=\!TestMiniChaosOzoneCluster + -Dtest=\!TestMiniChaosOzoneCluster "$@" REPORT_DIR=${OUTPUT_DIR:-"$DIR/../../../target/integration"} mkdir -p "$REPORT_DIR" diff --git a/hadoop-ozone/dev-support/checks/unit.sh b/hadoop-ozone/dev-support/checks/unit.sh index 9429026bd7ac0..6a124127edba9 100755 --- a/hadoop-ozone/dev-support/checks/unit.sh +++ b/hadoop-ozone/dev-support/checks/unit.sh @@ -17,7 +17,7 @@ DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" cd "$DIR/../../.." || exit 1 export MAVEN_OPTS="-Xmx4096m" -mvn -B -fn test -f pom.ozone.xml -pl \!:hadoop-ozone-integration-test,\!:hadoop-ozone-filesystem,\!:hadoop-ozone-tools +mvn -B -fn test -f pom.ozone.xml -pl \!:hadoop-ozone-integration-test,\!:hadoop-ozone-filesystem,\!:hadoop-ozone-tools "$@" REPORT_DIR=${OUTPUT_DIR:-"$DIR/../../../target/unit"} mkdir -p "$REPORT_DIR" diff --git a/hadoop-ozone/dist/src/main/compose/ozone-hdfs/.env b/hadoop-ozone/dist/src/main/compose/ozone-hdfs/.env index 8916fc3b75213..df9065c5ff47c 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-hdfs/.env +++ b/hadoop-ozone/dist/src/main/compose/ozone-hdfs/.env @@ -15,4 +15,4 @@ # limitations under the License. HADOOP_VERSION=3 -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozone-hdfs/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozone-hdfs/docker-compose.yaml index cd066355064f1..7d8295d8817ef 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-hdfs/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozone-hdfs/docker-compose.yaml @@ -37,7 +37,7 @@ services: env_file: - ./docker-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -48,7 +48,7 @@ services: - ./docker-config command: ["ozone","om"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -59,7 +59,7 @@ services: ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION command: ["ozone","scm"] s3g: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: diff --git a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/.env b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/.env index 1ec33aefa9993..27fc57662d668 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/.env +++ b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/.env @@ -19,4 +19,4 @@ HDDS_VERSION=@hdds.version@ # See: HADOOP-16092 for more details. HADOOP_IMAGE=flokkr/hadoop HADOOP_VERSION=2.7.7 -HADOOP_RUNNER_VERSION=@docker.ozone-runner.version@ +OZONE_RUNNER_VERSION=@docker.ozone-runner.version@ diff --git a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/docker-compose.yaml index a23566b8a4727..17f5ee535524f 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/docker-compose.yaml @@ -17,7 +17,7 @@ version: "3" services: datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../../..:/opt/hadoop ports: @@ -27,7 +27,7 @@ services: - docker-config - ../common-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: om volumes: - ../../..:/opt/hadoop @@ -41,7 +41,7 @@ services: - ../common-config command: ["/opt/hadoop/bin/ozone","om"] s3g: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: s3g volumes: - ../../..:/opt/hadoop @@ -52,7 +52,7 @@ services: - ../common-config command: ["/opt/hadoop/bin/ozone","s3g"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: scm volumes: - ../../..:/opt/hadoop diff --git a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/.env b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/.env index c6ef057b5188f..4cb42717f680d 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/.env +++ b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/.env @@ -19,4 +19,4 @@ HDDS_VERSION=@hdds.version@ # See: HADOOP-16092 for more details. HADOOP_IMAGE=flokkr/hadoop HADOOP_VERSION=3.1.2 -HADOOP_RUNNER_VERSION=@docker.ozone-runner.version@ +OZONE_RUNNER_VERSION=@docker.ozone-runner.version@ diff --git a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/docker-compose.yaml index af1c960918518..e3696fcf70a78 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/docker-compose.yaml @@ -17,7 +17,7 @@ version: "3" services: datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../../..:/opt/hadoop ports: @@ -27,7 +27,7 @@ services: - docker-config - ../common-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: om volumes: - ../../..:/opt/hadoop @@ -41,7 +41,7 @@ services: - ../common-config command: ["/opt/hadoop/bin/ozone","om"] s3g: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: s3g volumes: - ../../..:/opt/hadoop @@ -52,7 +52,7 @@ services: - ../common-config command: ["/opt/hadoop/bin/ozone","s3g"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: scm volumes: - ../../..:/opt/hadoop diff --git a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/.env b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/.env index d2c5aad9f98ef..70ba4b692d4e3 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/.env +++ b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/.env @@ -17,4 +17,4 @@ HDDS_VERSION=@hdds.version@ HADOOP_IMAGE=apache/hadoop HADOOP_VERSION=3 -HADOOP_RUNNER_VERSION=@docker.ozone-runner.version@ +OZONE_RUNNER_VERSION=@docker.ozone-runner.version@ diff --git a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/docker-compose.yaml index 755b279749ca2..c25d36cb9043d 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/docker-compose.yaml @@ -17,7 +17,7 @@ version: "3" services: datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../../..:/opt/hadoop ports: @@ -27,7 +27,7 @@ services: - docker-config - ../common-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: om volumes: - ../../..:/opt/hadoop @@ -41,7 +41,7 @@ services: - ../common-config command: ["/opt/hadoop/bin/ozone","om"] s3g: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: s3g volumes: - ../../..:/opt/hadoop @@ -52,7 +52,7 @@ services: - ../common-config command: ["/opt/hadoop/bin/ozone","s3g"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: scm volumes: - ../../..:/opt/hadoop diff --git a/hadoop-ozone/dist/src/main/compose/ozone-om-ha/.env b/hadoop-ozone/dist/src/main/compose/ozone-om-ha/.env index 8753b1dc9bb01..96ab163b47470 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-om-ha/.env +++ b/hadoop-ozone/dist/src/main/compose/ozone-om-ha/.env @@ -15,4 +15,4 @@ # limitations under the License. HDDS_VERSION=${hdds.version} -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozone-om-ha/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozone-om-ha/docker-compose.yaml index 116419e78a0e3..2cd2ce80c16a6 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-om-ha/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozone-om-ha/docker-compose.yaml @@ -17,7 +17,7 @@ version: "3" services: datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -27,7 +27,7 @@ services: env_file: - ./docker-config om1: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -40,7 +40,7 @@ services: - ./docker-config command: ["/opt/hadoop/bin/ozone","om"] om2: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -53,7 +53,7 @@ services: - ./docker-config command: ["/opt/hadoop/bin/ozone","om"] om3: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -66,7 +66,7 @@ services: - ./docker-config command: ["/opt/hadoop/bin/ozone","om"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop diff --git a/hadoop-ozone/dist/src/main/compose/ozone-recon/.env b/hadoop-ozone/dist/src/main/compose/ozone-recon/.env index 8753b1dc9bb01..96ab163b47470 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-recon/.env +++ b/hadoop-ozone/dist/src/main/compose/ozone-recon/.env @@ -15,4 +15,4 @@ # limitations under the License. HDDS_VERSION=${hdds.version} -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozone-recon/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozone-recon/docker-compose.yaml index 4cec246b4ffc0..38e2ef3309400 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-recon/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozone-recon/docker-compose.yaml @@ -17,7 +17,7 @@ version: "3" services: datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -28,7 +28,7 @@ services: env_file: - ./docker-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -40,7 +40,7 @@ services: - ./docker-config command: ["/opt/hadoop/bin/ozone","om"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -52,7 +52,7 @@ services: ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION command: ["/opt/hadoop/bin/ozone","scm"] recon: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop diff --git a/hadoop-ozone/dist/src/main/compose/ozone-topology/.env b/hadoop-ozone/dist/src/main/compose/ozone-topology/.env index 19a73160ab89d..249827bc91a9d 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-topology/.env +++ b/hadoop-ozone/dist/src/main/compose/ozone-topology/.env @@ -15,4 +15,4 @@ # limitations under the License. HDDS_VERSION=0.5.0-SNAPSHOT -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozone-topology/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozone-topology/docker-compose.yaml index 7b99a7b21f09e..a66eff617edb3 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-topology/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozone-topology/docker-compose.yaml @@ -17,7 +17,7 @@ version: "3" services: datanode_1: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -31,7 +31,7 @@ services: net: ipv4_address: 10.5.0.4 datanode_2: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -45,7 +45,7 @@ services: net: ipv4_address: 10.5.0.5 datanode_3: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -59,7 +59,7 @@ services: net: ipv4_address: 10.5.0.6 datanode_4: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -73,7 +73,7 @@ services: net: ipv4_address: 10.5.0.7 om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -88,7 +88,7 @@ services: net: ipv4_address: 10.5.0.70 scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop diff --git a/hadoop-ozone/dist/src/main/compose/ozone/.env b/hadoop-ozone/dist/src/main/compose/ozone/.env index 8753b1dc9bb01..96ab163b47470 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone/.env +++ b/hadoop-ozone/dist/src/main/compose/ozone/.env @@ -15,4 +15,4 @@ # limitations under the License. HDDS_VERSION=${hdds.version} -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozone/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozone/docker-compose.yaml index 6bfe36c7d7c54..145ce3ebb1763 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozone/docker-compose.yaml @@ -17,7 +17,7 @@ version: "3" services: datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -28,7 +28,7 @@ services: env_file: - ./docker-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop @@ -40,7 +40,7 @@ services: - ./docker-config command: ["/opt/hadoop/bin/ozone","om"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} privileged: true #required by the profiler volumes: - ../..:/opt/hadoop diff --git a/hadoop-ozone/dist/src/main/compose/ozone/test.sh b/hadoop-ozone/dist/src/main/compose/ozone/test.sh index fbae76da434eb..e06f817f3d814 100755 --- a/hadoop-ozone/dist/src/main/compose/ozone/test.sh +++ b/hadoop-ozone/dist/src/main/compose/ozone/test.sh @@ -31,6 +31,8 @@ start_docker_env execute_robot_test scm basic/basic.robot +execute_robot_test scm gdpr/gdpr.robot + stop_docker_env generate_report diff --git a/hadoop-ozone/dist/src/main/compose/ozoneblockade/.env b/hadoop-ozone/dist/src/main/compose/ozoneblockade/.env index 8753b1dc9bb01..96ab163b47470 100644 --- a/hadoop-ozone/dist/src/main/compose/ozoneblockade/.env +++ b/hadoop-ozone/dist/src/main/compose/ozoneblockade/.env @@ -15,4 +15,4 @@ # limitations under the License. HDDS_VERSION=${hdds.version} -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozoneblockade/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozoneblockade/docker-compose.yaml index ac548890512b6..703329fe144c5 100644 --- a/hadoop-ozone/dist/src/main/compose/ozoneblockade/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozoneblockade/docker-compose.yaml @@ -17,7 +17,7 @@ version: "3" services: datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -26,7 +26,7 @@ services: env_file: - ./docker-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -37,7 +37,7 @@ services: - ./docker-config command: ["/opt/hadoop/bin/ozone","om"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -48,7 +48,7 @@ services: ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION command: ["/opt/hadoop/bin/ozone","scm"] ozone_client: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: diff --git a/hadoop-ozone/dist/src/main/compose/ozoneperf/.env b/hadoop-ozone/dist/src/main/compose/ozoneperf/.env index 8753b1dc9bb01..96ab163b47470 100644 --- a/hadoop-ozone/dist/src/main/compose/ozoneperf/.env +++ b/hadoop-ozone/dist/src/main/compose/ozoneperf/.env @@ -15,4 +15,4 @@ # limitations under the License. HDDS_VERSION=${hdds.version} -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozoneperf/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozoneperf/docker-compose.yaml index a13aeaf37427f..fa205407e8722 100644 --- a/hadoop-ozone/dist/src/main/compose/ozoneperf/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozoneperf/docker-compose.yaml @@ -17,7 +17,7 @@ version: "3" services: datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -26,7 +26,7 @@ services: env_file: - ./docker-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -37,7 +37,7 @@ services: - ./docker-config command: ["ozone","om"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -61,7 +61,7 @@ services: ports: - 9090:9090 freon: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop environment: @@ -79,7 +79,7 @@ services: ports: - 3000:3000 s3g: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: diff --git a/hadoop-ozone/dist/src/main/compose/ozones3-haproxy/.env b/hadoop-ozone/dist/src/main/compose/ozones3-haproxy/.env index 8753b1dc9bb01..96ab163b47470 100644 --- a/hadoop-ozone/dist/src/main/compose/ozones3-haproxy/.env +++ b/hadoop-ozone/dist/src/main/compose/ozones3-haproxy/.env @@ -15,4 +15,4 @@ # limitations under the License. HDDS_VERSION=${hdds.version} -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozones3-haproxy/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozones3-haproxy/docker-compose.yaml index 829792950e593..78fd996a70ca2 100644 --- a/hadoop-ozone/dist/src/main/compose/ozones3-haproxy/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozones3-haproxy/docker-compose.yaml @@ -24,7 +24,7 @@ services: ports: - 9878:9878 datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -33,7 +33,7 @@ services: env_file: - ./docker-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -44,7 +44,7 @@ services: - ./docker-config command: ["ozone","om"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -55,7 +55,7 @@ services: ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION command: ["ozone","scm"] s3g1: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -64,7 +64,7 @@ services: - ./docker-config command: ["ozone","s3g"] s3g2: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -73,7 +73,7 @@ services: - ./docker-config command: ["ozone","s3g"] s3g3: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: diff --git a/hadoop-ozone/dist/src/main/compose/ozones3/.env b/hadoop-ozone/dist/src/main/compose/ozones3/.env index 8753b1dc9bb01..96ab163b47470 100644 --- a/hadoop-ozone/dist/src/main/compose/ozones3/.env +++ b/hadoop-ozone/dist/src/main/compose/ozones3/.env @@ -15,4 +15,4 @@ # limitations under the License. HDDS_VERSION=${hdds.version} -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozones3/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozones3/docker-compose.yaml index f4b8cc2d66f0f..cc4bfd2268c63 100644 --- a/hadoop-ozone/dist/src/main/compose/ozones3/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozones3/docker-compose.yaml @@ -17,7 +17,7 @@ version: "3" services: datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -26,7 +26,7 @@ services: env_file: - ./docker-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -37,7 +37,7 @@ services: - ./docker-config command: ["ozone","om"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -48,7 +48,7 @@ services: ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION command: ["ozone","scm"] s3g: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: diff --git a/hadoop-ozone/dist/src/main/compose/ozonescripts/.env b/hadoop-ozone/dist/src/main/compose/ozonescripts/.env index 8753b1dc9bb01..96ab163b47470 100644 --- a/hadoop-ozone/dist/src/main/compose/ozonescripts/.env +++ b/hadoop-ozone/dist/src/main/compose/ozonescripts/.env @@ -15,4 +15,4 @@ # limitations under the License. HDDS_VERSION=${hdds.version} -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/.env b/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/.env index c4253faa341ed..37227ac42bf60 100644 --- a/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/.env +++ b/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/.env @@ -16,4 +16,4 @@ HDDS_VERSION=${hdds.version} HADOOP_VERSION=3 -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-compose.yaml index 31f74f087edda..53e0142b2b65a 100644 --- a/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-compose.yaml @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -version: "3" +version: "3.5" services: kdc: build: @@ -23,17 +23,23 @@ services: args: buildno: 1 hostname: kdc + networks: + - ozone volumes: - ../..:/opt/hadoop kms: - image: apache/hadoop:${HADOOP_VERSION} - ports: + image: apache/hadoop:${HADOOP_VERSION} + networks: + - ozone + ports: - 9600:9600 - env_file: + env_file: - ./docker-config - command: ["hadoop", "kms"] + command: ["hadoop", "kms"] datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} + networks: + - ozone volumes: - ../..:/opt/hadoop ports: @@ -42,8 +48,10 @@ services: env_file: - docker-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: om + networks: + - ozone volumes: - ../..:/opt/hadoop ports: @@ -54,8 +62,10 @@ services: - docker-config command: ["/opt/hadoop/bin/ozone","om"] s3g: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: s3g + networks: + - ozone volumes: - ../..:/opt/hadoop ports: @@ -64,8 +74,10 @@ services: - ./docker-config command: ["/opt/hadoop/bin/ozone","s3g"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: scm + networks: + - ozone volumes: - ../..:/opt/hadoop ports: @@ -78,6 +90,8 @@ services: rm: image: apache/hadoop:${HADOOP_VERSION} hostname: rm + networks: + - ozone volumes: - ../..:/opt/ozone ports: @@ -90,6 +104,8 @@ services: nm: image: apache/hadoop:${HADOOP_VERSION} hostname: nm + networks: + - ozone volumes: - ../..:/opt/ozone env_file: @@ -100,7 +116,10 @@ services: command: ["yarn","nodemanager"] jhs: image: apache/hadoop:${HADOOP_VERSION} + container_name: jhs hostname: jhs + networks: + - ozone volumes: - ../..:/opt/ozone ports: @@ -111,13 +130,6 @@ services: HADOOP_CLASSPATH: /opt/ozone/share/ozone/lib/hadoop-ozone-filesystem-lib-current-@project.version@.jar WAIT_FOR: rm:8088 command: ["yarn","timelineserver"] - spark: - image: ahadoop/spark-2.4:hadoop-3.2 - hostname: spark - volumes: - - ../..:/opt/hadoop - ports: - - 4040:4040 - env_file: - - docker-config - command: ["watch","-n","100000","ls"] +networks: + ozone: + name: ozone diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-config b/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-config index f5c5fbd230bda..be9dc1e3b51c2 100644 --- a/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-config +++ b/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-config @@ -31,7 +31,7 @@ OZONE-SITE.XML_ozone.om.kerberos.principal=om/om@EXAMPLE.COM OZONE-SITE.XML_ozone.om.kerberos.keytab.file=/etc/security/keytabs/om.keytab OZONE-SITE.XML_ozone.s3g.keytab.file=/etc/security/keytabs/HTTP.keytab OZONE-SITE.XML_ozone.s3g.authentication.kerberos.principal=HTTP/s3g@EXAMPLE.COM -OZONE_SITE.XML_ozone.administrators=* +OZONE-SITE.XML_ozone.administrators=* OZONE-SITE.XML_ozone.security.enabled=true OZONE-SITE.XML_hdds.scm.http.kerberos.principal=HTTP/scm@EXAMPLE.COM @@ -62,9 +62,8 @@ HADOOP-POLICY.XML_org.apache.hadoop.yarn.server.api.ResourceTracker.acl=* HDFS-SITE.XML_rpc.metrics.quantile.enable=true HDFS-SITE.XML_rpc.metrics.percentiles.intervals=60,300 -CORE-SITE.xml_fs.o3fs.impl=org.apache.hadoop.fs.ozone.OzoneFileSystem -CORE-SITE.xml_fs.AbstractFileSystem.o3fs.impl=org.apache.hadoop.fs.ozone.OzFs -CORE-SITE.xml_fs.defaultFS=o3fs://bucket1.vol1/ +CORE-SITE.XML_fs.AbstractFileSystem.o3fs.impl=org.apache.hadoop.fs.ozone.OzFs +CORE-SITE.XML_fs.defaultFS=o3fs://bucket1.vol1/ MAPRED-SITE.XML_mapreduce.framework.name=yarn MAPRED-SITE.XML_yarn.app.mapreduce.am.env=HADOOP_MAPRED_HOME=$HADOOP_HOME @@ -76,12 +75,12 @@ MAPRED-SITE.XML_mapreduce.reduce.memory.mb=2048 MAPRED-SITE.XML_mapreduce.application.classpath=/opt/hadoop/share/hadoop/mapreduce/*:/opt/hadoop/share/hadoop/mapreduce/lib/*:/opt/ozone/share/ozone/lib/hadoop-ozone-filesystem-lib-current-@project.version@.jar YARN-SITE.XML_yarn.app.mapreduce.am.staging-dir=/user -YARN_SITE.XML_yarn.timeline-service.enabled=true -YARN_SITE.XML_yarn.timeline-service.generic.application.history.enabled=true -YARN_SITE.XML_yarn.timeline-service.hostname=jhs +YARN-SITE.XML_yarn.timeline-service.enabled=true +YARN-SITE.XML_yarn.timeline-service.generic.application.history.enabled=true +YARN-SITE.XML_yarn.timeline-service.hostname=jhs YARN-SITE.XML_yarn.timeline-service.principal=jhs/jhs@EXAMPLE.COM YARN-SITE.XML_yarn.timeline-service.keytab=/etc/security/keytabs/jhs.keytab -YARN_SITE.XML_yarn.log.server.url=http://jhs:8188/applicationhistory/logs/ +YARN-SITE.XML_yarn.log.server.url=http://jhs:8188/applicationhistory/logs/ YARN-SITE.XML_yarn.nodemanager.principal=nm/_HOST@EXAMPLE.COM YARN-SITE.XML_yarn.nodemanager.keytab=/etc/security/keytabs/nm.keytab @@ -94,15 +93,17 @@ YARN-SITE.XML_yarn.nodemanager.disk-health-checker.enable=false YARN-SITE.XML_yarn.resourcemanager.hostname=rm YARN-SITE.XML_yarn.resourcemanager.keytab=/etc/security/keytabs/rm.keytab YARN-SITE.XML_yarn.resourcemanager.principal=rm/rm@EXAMPLE.COM -YARN_SITE_XML_yarn.resourcemanager.system.metrics.publisher.enabled=true +YARN-SITE.XML_yarn.resourcemanager.system.metrics.publisher.enabled=true YARN-SITE.XML_yarn.log-aggregation-enable=true -YARN-SITE.yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds=3600 -YARN-SITE.yarn.nodemanager.delete.debug-delay-sec=600 +YARN-SITE.XML_yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds=3600 +YARN-SITE.XML_yarn.nodemanager.delete.debug-delay-sec=600 -YARN-SITE.yarn.nodemanager.container-executor.class=org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor -YARN-SITE.yarn.nodemanager.linux-container-executor.path=/opt/hadoop/bin/container-executor -YARN-SITE.yarn.nodemanager.linux-container-executor.group=hadoop +# Yarn LinuxContainer requires the /opt/hadoop/etc/hadoop to be owned by root and not modifiable by other users, +# which prevents start.sh from changing the configurations based on docker-config +# YARN-SITE.XML_yarn.nodemanager.container-executor.class=org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor +# YARN-SITE.XML_yarn.nodemanager.linux-container-executor.path=/opt/hadoop/bin/container-executor +# YARN-SITE.XML_yarn.nodemanager.linux-container-executor.group=hadoop CAPACITY-SCHEDULER.XML_yarn.scheduler.capacity.maximum-applications=10000 CAPACITY-SCHEDULER.XML_yarn.scheduler.capacity.maximum-am-resource-percent=0.1 @@ -175,4 +176,4 @@ KERBEROS_SERVER=kdc JAVA_HOME=/usr/lib/jvm/jre JSVC_HOME=/usr/bin SLEEP_SECONDS=5 -KERBEROS_ENABLED=true \ No newline at end of file +KERBEROS_ENABLED=true diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure/.env b/hadoop-ozone/dist/src/main/compose/ozonesecure/.env index c4253faa341ed..37227ac42bf60 100644 --- a/hadoop-ozone/dist/src/main/compose/ozonesecure/.env +++ b/hadoop-ozone/dist/src/main/compose/ozonesecure/.env @@ -16,4 +16,4 @@ HDDS_VERSION=${hdds.version} HADOOP_VERSION=3 -HADOOP_RUNNER_VERSION=${docker.ozone-runner.version} \ No newline at end of file +OZONE_RUNNER_VERSION=${docker.ozone-runner.version} diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-compose.yaml index d202717f9d7e6..de60a411116c3 100644 --- a/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-compose.yaml @@ -35,7 +35,7 @@ services: command: ["hadoop", "kms"] datanode: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} volumes: - ../..:/opt/hadoop ports: @@ -44,7 +44,7 @@ services: env_file: - docker-config om: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: om volumes: - ../..:/opt/hadoop @@ -56,7 +56,7 @@ services: - docker-config command: ["/opt/hadoop/bin/ozone","om"] s3g: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: s3g volumes: - ../..:/opt/hadoop @@ -66,7 +66,7 @@ services: - ./docker-config command: ["/opt/hadoop/bin/ozone","s3g"] recon: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: recon volumes: - ../..:/opt/hadoop @@ -78,7 +78,7 @@ services: WAITFOR: om:9874 command: ["/opt/hadoop/bin/ozone","recon"] scm: - image: apache/ozone-runner:${HADOOP_RUNNER_VERSION} + image: apache/ozone-runner:${OZONE_RUNNER_VERSION} hostname: scm volumes: - ../..:/opt/hadoop diff --git a/hadoop-ozone/dist/src/main/compose/test-all.sh b/hadoop-ozone/dist/src/main/compose/test-all.sh index 7883e878e7d3d..afa5d56c6f110 100755 --- a/hadoop-ozone/dist/src/main/compose/test-all.sh +++ b/hadoop-ozone/dist/src/main/compose/test-all.sh @@ -29,7 +29,7 @@ rm "$ALL_RESULT_DIR/*" RESULT=0 IFS=$'\n' # shellcheck disable=SC2044 -for test in $(find "$SCRIPT_DIR" -name test.sh); do +for test in $(find "$SCRIPT_DIR" -name test.sh | sort); do echo "Executing test in $(dirname "$test")" #required to read the .env file from the right location @@ -41,7 +41,7 @@ for test in $(find "$SCRIPT_DIR" -name test.sh); do echo "ERROR: Test execution of $(dirname "$test") is FAILED!!!!" fi RESULT_DIR="$(dirname "$test")/result" - cp "$RESULT_DIR"/robot-*.xml "$ALL_RESULT_DIR" + cp "$RESULT_DIR"/robot-*.xml "$RESULT_DIR"/docker-*.log "$ALL_RESULT_DIR"/ done rebot -N "smoketests" -d "$SCRIPT_DIR/result" "$SCRIPT_DIR/result/robot-*.xml" diff --git a/hadoop-ozone/dist/src/main/compose/test-single.sh b/hadoop-ozone/dist/src/main/compose/test-single.sh index f1203d3077551..629a9bc07a653 100755 --- a/hadoop-ozone/dist/src/main/compose/test-single.sh +++ b/hadoop-ozone/dist/src/main/compose/test-single.sh @@ -48,6 +48,8 @@ fi # shellcheck source=testlib.sh source "$COMPOSE_DIR/../testlib.sh" +create_results_dir + execute_robot_test "$1" "$2" generate_report diff --git a/hadoop-ozone/dist/src/main/compose/testlib.sh b/hadoop-ozone/dist/src/main/compose/testlib.sh index f659c0ed5a737..b20dca894ae25 100755 --- a/hadoop-ozone/dist/src/main/compose/testlib.sh +++ b/hadoop-ozone/dist/src/main/compose/testlib.sh @@ -77,6 +77,7 @@ wait_for_datanodes(){ sleep 2 done echo "WARNING! Datanodes are not started successfully. Please check the docker-compose files" + return 1 } ## @description Starts a docker-compose based test environment @@ -86,13 +87,14 @@ start_docker_env(){ create_results_dir - docker-compose -f "$COMPOSE_FILE" down - docker-compose -f "$COMPOSE_FILE" up -d --scale datanode="${datanode_count}" \ + docker-compose -f "$COMPOSE_FILE" --no-ansi down + docker-compose -f "$COMPOSE_FILE" --no-ansi up -d --scale datanode="${datanode_count}" \ && wait_for_datanodes "$COMPOSE_FILE" "${datanode_count}" \ && sleep 10 if [[ $? -gt 0 ]]; then - docker-compose -f "$COMPOSE_FILE" down + OUTPUT_NAME="$COMPOSE_ENV_NAME" + stop_docker_env return 1 fi } @@ -136,9 +138,9 @@ execute_command_in_container(){ ## @description Stops a docker-compose based test environment (with saving the logs) stop_docker_env(){ - docker-compose -f "$COMPOSE_FILE" logs > "$RESULT_DIR/docker-$OUTPUT_NAME.log" + docker-compose -f "$COMPOSE_FILE" --no-ansi logs > "$RESULT_DIR/docker-$OUTPUT_NAME.log" if [ "${KEEP_RUNNING:-false}" = false ]; then - docker-compose -f "$COMPOSE_FILE" down + docker-compose -f "$COMPOSE_FILE" --no-ansi down fi } diff --git a/hadoop-ozone/dist/src/main/smoketest/gdpr/gdpr.robot b/hadoop-ozone/dist/src/main/smoketest/gdpr/gdpr.robot new file mode 100644 index 0000000000000..f4705eb0a3d0b --- /dev/null +++ b/hadoop-ozone/dist/src/main/smoketest/gdpr/gdpr.robot @@ -0,0 +1,89 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +*** Settings *** +Documentation Smoketest Ozone GDPR Feature +Library OperatingSystem +Library BuiltIn +Library String +Resource ../commonlib.robot +Suite Setup Generate volume + +*** Variables *** +${volume} generated + +*** Keywords *** +Generate volume + ${random} = Generate Random String 5 [LOWER] + Set Suite Variable ${volume} ${random} + +*** Test Cases *** +Test GDPR disabled + Test GDPR(disabled) without explicit options ${volume} + +Test GDPR --enforcegdpr=true + Test GDPR with --enforcegdpr=true ${volume} + +Test GDPR -g=true + Test GDPR with -g=true ${volume} + +Test GDPR -g=false + Test GDPR with -g=false ${volume} + +*** Keywords *** +Test GDPR(disabled) without explicit options + [arguments] ${volume} + Execute ozone sh volume create /${volume} --quota 100TB + Execute ozone sh bucket create /${volume}/mybucket1 + ${result} = Execute ozone sh bucket info /${volume}/mybucket1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="mybucket1") | .metadata | .gdprEnabled' + Should Be Equal ${result} null + Execute ozone sh key put /${volume}/mybucket1/mykey /opt/hadoop/NOTICE.txt + Execute rm -f NOTICE.txt.1 + ${result} = Execute ozone sh key info /${volume}/mybucket1/mykey | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="mykey") | .metadata | .gdprEnabled' + Should Be Equal ${result} null + Execute ozone sh key delete /${volume}/mybucket1/mykey + +Test GDPR with --enforcegdpr=true + [arguments] ${volume} + Execute ozone sh bucket create --enforcegdpr=true /${volume}/mybucket2 + ${result} = Execute ozone sh bucket info /${volume}/mybucket2 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="mybucket2") | .metadata | .gdprEnabled' + Should Be Equal ${result} true + Execute ozone sh key put /${volume}/mybucket2/mykey /opt/hadoop/NOTICE.txt + Execute rm -f NOTICE.txt.1 + ${result} = Execute ozone sh key info /${volume}/mybucket2/mykey | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="mykey") | .metadata | .gdprEnabled' + Should Be Equal ${result} true + Execute ozone sh key delete /${volume}/mybucket2/mykey + +Test GDPR with -g=true + [arguments] ${volume} + Execute ozone sh bucket create -g=true /${volume}/mybucket3 + ${result} = Execute ozone sh bucket info /${volume}/mybucket3 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="mybucket3") | .metadata | .gdprEnabled' + Should Be Equal ${result} true + Execute ozone sh key put /${volume}/mybucket3/mykey /opt/hadoop/NOTICE.txt + Execute rm -f NOTICE.txt.1 + ${result} = Execute ozone sh key info /${volume}/mybucket3/mykey | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="mykey") | .metadata | .gdprEnabled' + Should Be Equal ${result} true + Execute ozone sh key delete /${volume}/mybucket3/mykey + +Test GDPR with -g=false + [arguments] ${volume} + Execute ozone sh bucket create /${volume}/mybucket4 + ${result} = Execute ozone sh bucket info /${volume}/mybucket4 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="mybucket4") | .metadata | .gdprEnabled' + Should Be Equal ${result} null + Execute ozone sh key put /${volume}/mybucket4/mykey /opt/hadoop/NOTICE.txt + Execute rm -f NOTICE.txt.1 + ${result} = Execute ozone sh key info /${volume}/mybucket4/mykey | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="mykey") | .metadata | .gdprEnabled' + Should Be Equal ${result} null + Execute ozone sh key delete /${volume}/mybucket4/mykey diff --git a/hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/ozone/cluster.py b/hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/ozone/cluster.py index 143426614df1d..1616083377b40 100644 --- a/hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/ozone/cluster.py +++ b/hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/ozone/cluster.py @@ -151,8 +151,8 @@ def start(self): # check if docker is up. - if "HADOOP_RUNNER_VERSION" not in os.environ: - self.__logger__.error("HADOOP_RUNNER_VERSION is not set.") + if "OZONE_RUNNER_VERSION" not in os.environ: + self.__logger__.error("OZONE_RUNNER_VERSION is not set.") sys.exit(1) if "HDDS_VERSION" not in os.environ: diff --git a/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/BaseInsightSubCommand.java b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/BaseInsightSubCommand.java index 95cda4168c5c3..4c3875c3ac0be 100644 --- a/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/BaseInsightSubCommand.java +++ b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/BaseInsightSubCommand.java @@ -31,6 +31,8 @@ import org.apache.hadoop.ozone.insight.scm.NodeManagerInsight; import org.apache.hadoop.ozone.insight.scm.ReplicaManagerInsight; import org.apache.hadoop.ozone.insight.scm.ScmProtocolBlockLocationInsight; +import org.apache.hadoop.ozone.insight.scm.ScmProtocolContainerLocationInsight; +import org.apache.hadoop.ozone.insight.scm.ScmProtocolSecurityInsight; import org.apache.hadoop.ozone.om.OMConfigKeys; import picocli.CommandLine; @@ -88,7 +90,10 @@ public Map createInsightPoints( insights.put("scm.event-queue", new EventQueueInsight()); insights.put("scm.protocol.block-location", new ScmProtocolBlockLocationInsight()); - + insights.put("scm.protocol.container-location", + new ScmProtocolContainerLocationInsight()); + insights.put("scm.protocol.security", + new ScmProtocolSecurityInsight()); insights.put("om.key-manager", new KeyManagerInsight()); insights.put("om.protocol.client", new OmProtocolInsight()); diff --git a/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolBlockLocationInsight.java b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolBlockLocationInsight.java index 5ca0945238be8..f67f64194b3a7 100644 --- a/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolBlockLocationInsight.java +++ b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolBlockLocationInsight.java @@ -42,9 +42,9 @@ public List getRelatedLoggers(boolean verbose) { new LoggerSource(Type.SCM, ScmBlockLocationProtocolServerSideTranslatorPB.class, defaultLevel(verbose))); - new LoggerSource(Type.SCM, + loggers.add(new LoggerSource(Type.SCM, SCMBlockProtocolServer.class, - defaultLevel(verbose)); + defaultLevel(verbose))); return loggers; } diff --git a/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolContainerLocationInsight.java b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolContainerLocationInsight.java new file mode 100644 index 0000000000000..d6db589ed824a --- /dev/null +++ b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolContainerLocationInsight.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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.ozone.insight.scm; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StorageContainerLocationProtocolService; +import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB; +import org.apache.hadoop.ozone.insight.BaseInsightPoint; +import org.apache.hadoop.ozone.insight.Component.Type; +import org.apache.hadoop.ozone.insight.LoggerSource; +import org.apache.hadoop.ozone.insight.MetricGroupDisplay; + +/** + * Insight metric to check the SCM block location protocol behaviour. + */ +public class ScmProtocolContainerLocationInsight extends BaseInsightPoint { + + @Override + public List getRelatedLoggers(boolean verbose) { + List loggers = new ArrayList<>(); + loggers.add( + new LoggerSource(Type.SCM, + StorageContainerLocationProtocolServerSideTranslatorPB.class, + defaultLevel(verbose))); + new LoggerSource(Type.SCM, + StorageContainerLocationProtocolService.class, + defaultLevel(verbose)); + return loggers; + } + + @Override + public List getMetrics() { + List metrics = new ArrayList<>(); + + Map filter = new HashMap<>(); + filter.put("servername", "StorageContainerLocationProtocolService"); + + addRpcMetrics(metrics, Type.SCM, filter); + + addProtocolMessageMetrics(metrics, "scm_container_location_protocol", + Type.SCM, StorageContainerLocationProtocolProtos.Type.values()); + + return metrics; + } + + @Override + public String getDescription() { + return "SCM Container location protocol endpoint"; + } + +} diff --git a/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolDatanodeInsight.java b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolDatanodeInsight.java new file mode 100644 index 0000000000000..289af89a7c569 --- /dev/null +++ b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolDatanodeInsight.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.ozone.insight.scm; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer; +import org.apache.hadoop.ozone.insight.BaseInsightPoint; +import org.apache.hadoop.ozone.insight.Component.Type; +import org.apache.hadoop.ozone.insight.LoggerSource; +import org.apache.hadoop.ozone.insight.MetricGroupDisplay; +import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolServerSideTranslatorPB; + +/** + * Insight metric to check the SCM datanode protocol behaviour. + */ +public class ScmProtocolDatanodeInsight extends BaseInsightPoint { + + @Override + public List getRelatedLoggers(boolean verbose) { + List loggers = new ArrayList<>(); + loggers.add( + new LoggerSource(Type.SCM, + SCMDatanodeProtocolServer.class, + defaultLevel(verbose))); + loggers.add( + new LoggerSource(Type.SCM, + StorageContainerDatanodeProtocolServerSideTranslatorPB.class, + defaultLevel(verbose))); + return loggers; + } + + @Override + public List getMetrics() { + List metrics = new ArrayList<>(); + + Map filter = new HashMap<>(); + filter.put("servername", "StorageContainerDatanodeProtocolService"); + + addRpcMetrics(metrics, Type.SCM, filter); + + addProtocolMessageMetrics(metrics, "scm_datanode_protocol", + Type.SCM, StorageContainerDatanodeProtocolProtos.Type.values()); + + return metrics; + } + + @Override + public String getDescription() { + return "SCM Datanode protocol endpoint"; + } + +} diff --git a/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolSecurityInsight.java b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolSecurityInsight.java new file mode 100644 index 0000000000000..734da34f8bbda --- /dev/null +++ b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/scm/ScmProtocolSecurityInsight.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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.ozone.insight.scm; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos; +import org.apache.hadoop.hdds.scm.protocol.SCMSecurityProtocolServerSideTranslatorPB; +import org.apache.hadoop.hdds.scm.server.SCMSecurityProtocolServer; +import org.apache.hadoop.ozone.insight.BaseInsightPoint; +import org.apache.hadoop.ozone.insight.Component.Type; +import org.apache.hadoop.ozone.insight.LoggerSource; +import org.apache.hadoop.ozone.insight.MetricGroupDisplay; + +/** + * Insight metric to check the SCM block location protocol behaviour. + */ +public class ScmProtocolSecurityInsight extends BaseInsightPoint { + + @Override + public List getRelatedLoggers(boolean verbose) { + List loggers = new ArrayList<>(); + loggers.add( + new LoggerSource(Type.SCM, + SCMSecurityProtocolServerSideTranslatorPB.class, + defaultLevel(verbose))); + new LoggerSource(Type.SCM, + SCMSecurityProtocolServer.class, + defaultLevel(verbose)); + return loggers; + } + + @Override + public List getMetrics() { + List metrics = new ArrayList<>(); + + Map filter = new HashMap<>(); + filter.put("servername", "SCMSecurityProtocolService"); + + addRpcMetrics(metrics, Type.SCM, filter); + + addProtocolMessageMetrics(metrics, "scm_security_protocol", + Type.SCM, SCMSecurityProtocolProtos.Type.values()); + + return metrics; + } + + @Override + public String getDescription() { + return "SCM Block location protocol endpoint"; + } + +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java index 7b908151f487f..9ac45b8811682 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java @@ -353,6 +353,71 @@ public void testApplyTransactionFailure() throws Exception { Assert.assertTrue(snapshot.getPath().equals(latestSnapshot.getPath())); } + @Test + public void testApplyTransactionIdempotencyWithClosedContainer() + throws Exception { + OzoneOutputStream key = + objectStore.getVolume(volumeName).getBucket(bucketName) + .createKey("ratis", 1024, ReplicationType.RATIS, + ReplicationFactor.ONE, new HashMap<>()); + // First write and flush creates a container in the datanode + key.write("ratis".getBytes()); + key.flush(); + key.write("ratis".getBytes()); + KeyOutputStream groupOutputStream = (KeyOutputStream) key.getOutputStream(); + List locationInfoList = + groupOutputStream.getLocationInfoList(); + Assert.assertEquals(1, locationInfoList.size()); + OmKeyLocationInfo omKeyLocationInfo = locationInfoList.get(0); + ContainerData containerData = + cluster.getHddsDatanodes().get(0).getDatanodeStateMachine() + .getContainer().getContainerSet() + .getContainer(omKeyLocationInfo.getContainerID()) + .getContainerData(); + Assert.assertTrue(containerData instanceof KeyValueContainerData); + key.close(); + ContainerStateMachine stateMachine = + (ContainerStateMachine) ContainerTestHelper.getStateMachine(cluster); + SimpleStateMachineStorage storage = + (SimpleStateMachineStorage) stateMachine.getStateMachineStorage(); + Path parentPath = storage.findLatestSnapshot().getFile().getPath(); + // Since the snapshot threshold is set to 1, since there are + // applyTransactions, we should see snapshots + Assert.assertTrue(parentPath.getParent().toFile().listFiles().length > 0); + FileInfo snapshot = storage.findLatestSnapshot().getFile(); + Assert.assertNotNull(snapshot); + long containerID = omKeyLocationInfo.getContainerID(); + Pipeline pipeline = cluster.getStorageContainerLocationClient() + .getContainerWithPipeline(containerID).getPipeline(); + XceiverClientSpi xceiverClient = + xceiverClientManager.acquireClient(pipeline); + ContainerProtos.ContainerCommandRequestProto.Builder request = + ContainerProtos.ContainerCommandRequestProto.newBuilder(); + request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); + request.setCmdType(ContainerProtos.Type.CloseContainer); + request.setContainerID(containerID); + request.setCloseContainer( + ContainerProtos.CloseContainerRequestProto.getDefaultInstance()); + try { + xceiverClient.sendCommand(request.build()); + } catch (IOException e) { + Assert.fail("Exception should not be thrown"); + } + Assert.assertTrue( + cluster.getHddsDatanodes().get(0).getDatanodeStateMachine() + .getContainer().getContainerSet().getContainer(containerID) + .getContainerState() + == ContainerProtos.ContainerDataProto.State.CLOSED); + Assert.assertTrue(stateMachine.isStateMachineHealthy()); + try { + stateMachine.takeSnapshot(); + } catch (IOException ioe) { + Assert.fail("Exception should not be thrown"); + } + FileInfo latestSnapshot = storage.findLatestSnapshot().getFile(); + Assert.assertFalse(snapshot.getPath().equals(latestSnapshot.getPath())); + } + @Test public void testValidateBCSIDOnDnRestart() throws Exception { OzoneOutputStream key = diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java index 75dd880d2306c..3cba9b3effaed 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServlet.java @@ -154,8 +154,6 @@ public void write(int b) throws IOException { Assert.assertTrue( omMetrics.getLastCheckpointCreationTimeTaken() == 0); - Assert.assertTrue( - omMetrics.getLastCheckpointTarOperationTimeTaken() == 0); Assert.assertTrue( omMetrics.getLastCheckpointStreamingTimeTaken() == 0); @@ -164,8 +162,6 @@ public void write(int b) throws IOException { Assert.assertTrue(tempFile.length() > 0); Assert.assertTrue( omMetrics.getLastCheckpointCreationTimeTaken() > 0); - Assert.assertTrue( - omMetrics.getLastCheckpointTarOperationTimeTaken() > 0); Assert.assertTrue( omMetrics.getLastCheckpointStreamingTimeTaken() > 0); } finally { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerConfiguration.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerConfiguration.java index 77f0dfc2305a0..2716d51f07e95 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerConfiguration.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerConfiguration.java @@ -119,10 +119,13 @@ public void testDefaultPortIfNotSpecified() throws Exception { String omNode1Id = "omNode1"; String omNode2Id = "omNode2"; String omNodesKeyValue = omNode1Id + "," + omNode2Id; - conf.set(OMConfigKeys.OZONE_OM_NODES_KEY, omNodesKeyValue); + String serviceID = "service1"; + conf.set(OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY, serviceID); + conf.set(OMConfigKeys.OZONE_OM_NODES_KEY + "." + serviceID, + omNodesKeyValue); - String omNode1RpcAddrKey = getOMAddrKeyWithSuffix(null, omNode1Id); - String omNode2RpcAddrKey = getOMAddrKeyWithSuffix(null, omNode2Id); + String omNode1RpcAddrKey = getOMAddrKeyWithSuffix(serviceID, omNode1Id); + String omNode2RpcAddrKey = getOMAddrKeyWithSuffix(serviceID, omNode2Id); conf.set(omNode1RpcAddrKey, "0.0.0.0"); conf.set(omNode2RpcAddrKey, "122.0.0.122"); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerRocksDBLogging.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerRocksDBLogging.java new file mode 100644 index 0000000000000..5ca2eea57054a --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerRocksDBLogging.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.ozone.om; + +import java.util.UUID; +import java.util.concurrent.TimeoutException; + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.utils.db.DBStoreBuilder; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +/** + * Test RocksDB logging for Ozone Manager. + */ +public class TestOzoneManagerRocksDBLogging { + private MiniOzoneCluster cluster = null; + private OzoneConfiguration conf; + private String clusterId; + private String scmId; + private String omId; + + @Rule + public Timeout timeout = new Timeout(60000); + + @Before + public void init() throws Exception { + conf = new OzoneConfiguration(); + conf.set("hadoop.hdds.db.rocksdb.logging.enabled", "true"); + clusterId = UUID.randomUUID().toString(); + scmId = UUID.randomUUID().toString(); + omId = UUID.randomUUID().toString(); + cluster = MiniOzoneCluster.newBuilder(conf) + .setClusterId(clusterId) + .setScmId(scmId) + .setOmId(omId) + .build(); + cluster.waitForClusterToBeReady(); + } + + /** + * Shutdown MiniDFSCluster. + */ + @After + public void shutdown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testOMRocksDBLoggingEnabled() throws Exception { + + GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer + .captureLogs(DBStoreBuilder.ROCKS_DB_LOGGER); + cluster.restartOzoneManager(); + GenericTestUtils.waitFor(() -> logCapturer.getOutput() + .contains("db_impl.cc"), + 1000, 10000); + + cluster.getConf().set("hadoop.hdds.db.rocksdb.logging.enabled", "false"); + cluster.restartOzoneManager(); + logCapturer.clearOutput(); + try { + GenericTestUtils.waitFor(() -> logCapturer.getOutput() + .contains("db_impl.cc"), + 1000, 10000); + Assert.fail(); + } catch (TimeoutException ex) { + Assert.assertTrue(ex.getMessage().contains("Timed out")); + } + } + +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java index bae71bf51131c..354c9075e3e97 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java @@ -193,7 +193,6 @@ private KeyManagerImpl(OzoneManager om, ScmClient scmClient, this.secretManager = secretManager; this.kmsProvider = kmsProvider; - start(conf); } @Override diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java index dbbb065a3b85d..81031838b2395 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java @@ -23,9 +23,8 @@ import static org.apache.hadoop.ozone.OzoneConsts. OZONE_DB_CHECKPOINT_REQUEST_FLUSH; -import java.io.File; -import java.io.FileInputStream; import java.io.IOException; +import java.nio.file.Path; import java.time.Duration; import java.time.Instant; @@ -34,12 +33,9 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdfs.server.namenode.TransferFsImage; import org.apache.hadoop.hdfs.util.DataTransferThrottler; -import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.hdds.utils.db.DBStore; @@ -102,8 +98,7 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) { return; } - FileInputStream checkpointFileInputStream = null; - File checkPointTarFile = null; + DBCheckpoint checkpoint = null; try { boolean flush = false; @@ -131,8 +126,8 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) { ratisSnapshotIndex = om.getRatisSnapshotIndex(); } - DBCheckpoint checkpoint = omDbStore.getCheckpoint(flush); - if (checkpoint == null) { + checkpoint = omDbStore.getCheckpoint(flush); + if (checkpoint == null || checkpoint.getCheckpointLocation() == null) { LOG.error("Unable to process metadata snapshot request. " + "Checkpoint request returned null."); response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); @@ -141,49 +136,41 @@ public void doGet(HttpServletRequest request, HttpServletResponse response) { omMetrics.setLastCheckpointCreationTimeTaken( checkpoint.checkpointCreationTimeTaken()); - Instant start = Instant.now(); - checkPointTarFile = OmUtils.createTarFile( - checkpoint.getCheckpointLocation()); - Instant end = Instant.now(); - - long duration = Duration.between(start, end).toMillis(); - LOG.debug("Time taken to archive the checkpoint : " + duration + - " milliseconds"); - LOG.info("Checkpoint Tar location = " + - checkPointTarFile.getAbsolutePath()); - omMetrics.setLastCheckpointTarOperationTimeTaken(duration); - + Path file = checkpoint.getCheckpointLocation().getFileName(); + if (file == null) { + return; + } response.setContentType("application/x-tgz"); response.setHeader("Content-Disposition", "attachment; filename=\"" + - checkPointTarFile.getName() + "\""); + file.toString() + ".tgz\""); // Ratis snapshot index used when downloading DB checkpoint to OM follower response.setHeader(OM_RATIS_SNAPSHOT_INDEX, String.valueOf(ratisSnapshotIndex)); - checkpointFileInputStream = new FileInputStream(checkPointTarFile); - start = Instant.now(); - TransferFsImage.copyFileToStream(response.getOutputStream(), - checkPointTarFile, - checkpointFileInputStream, - throttler); - end = Instant.now(); + Instant start = Instant.now(); + OmUtils.writeOmDBCheckpointToStream(checkpoint, + response.getOutputStream()); + Instant end = Instant.now(); - duration = Duration.between(start, end).toMillis(); - LOG.debug("Time taken to write the checkpoint to response output " + + long duration = Duration.between(start, end).toMillis(); + LOG.info("Time taken to write the checkpoint to response output " + "stream: " + duration + " milliseconds"); omMetrics.setLastCheckpointStreamingTimeTaken(duration); - checkpoint.cleanupCheckpoint(); - } catch (IOException e) { + } catch (Exception e) { LOG.error( "Unable to process metadata snapshot request. ", e); response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); } finally { - if (checkPointTarFile != null) { - FileUtils.deleteQuietly(checkPointTarFile); + if (checkpoint != null) { + try { + checkpoint.cleanupCheckpoint(); + } catch (IOException e) { + LOG.error("Error trying to clean checkpoint at {} .", + checkpoint.getCheckpointLocation().toString()); + } } - IOUtils.closeStream(checkpointFileInputStream); } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java index de42be01705cb..2d1ae30648db5 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java @@ -123,7 +123,6 @@ public class OMMetrics { // Metrics to track checkpointing statistics from last run. private @Metric MutableGaugeLong lastCheckpointCreationTimeTaken; - private @Metric MutableGaugeLong lastCheckpointTarOperationTimeTaken; private @Metric MutableGaugeLong lastCheckpointStreamingTimeTaken; private @Metric MutableCounterLong numBucketS3Creates; @@ -511,10 +510,6 @@ public void setLastCheckpointCreationTimeTaken(long val) { this.lastCheckpointCreationTimeTaken.set(val); } - public void setLastCheckpointTarOperationTimeTaken(long val) { - this.lastCheckpointTarOperationTimeTaken.set(val); - } - public void setLastCheckpointStreamingTimeTaken(long val) { this.lastCheckpointStreamingTimeTaken.set(val); } @@ -756,11 +751,6 @@ public long getLastCheckpointCreationTimeTaken() { return lastCheckpointCreationTimeTaken.value(); } - @VisibleForTesting - public long getLastCheckpointTarOperationTimeTaken() { - return lastCheckpointTarOperationTimeTaken.value(); - } - @VisibleForTesting public long getLastCheckpointStreamingTimeTaken() { return lastCheckpointStreamingTimeTaken.value(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java index 59158e23a491d..6c085911e11b3 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java @@ -47,7 +47,7 @@ import org.apache.hadoop.ozone.om.codec.RepeatedOmKeyInfoCodec; import org.apache.hadoop.ozone.om.codec.S3SecretValueCodec; import org.apache.hadoop.ozone.om.codec.TokenIdentifierCodec; -import org.apache.hadoop.ozone.om.codec.VolumeListCodec; +import org.apache.hadoop.ozone.om.codec.UserVolumeInfoCodec; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; @@ -61,7 +61,8 @@ import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.om.helpers.S3SecretValue; import org.apache.hadoop.ozone.om.lock.OzoneManagerLock; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList; +import org.apache.hadoop.ozone.protocol.proto + .OzoneManagerProtocolProtos.UserVolumeInfo; import org.apache.hadoop.ozone.security.OzoneTokenIdentifier; import com.google.common.annotations.VisibleForTesting; @@ -92,7 +93,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager { * |----------------------------------------------------------------------| * | Column Family | VALUE | * |----------------------------------------------------------------------| - * | userTable | user->VolumeList | + * | userTable | /user->UserVolumeInfo | * |----------------------------------------------------------------------| * | volumeTable | /volume->VolumeInfo | * |----------------------------------------------------------------------| @@ -170,7 +171,7 @@ protected OmMetadataManagerImpl() { } @Override - public Table getUserTable() { + public Table getUserTable() { return userTable; } @@ -266,7 +267,7 @@ protected DBStoreBuilder addOMTablesAndCodecs(DBStoreBuilder builder) { .addCodec(RepeatedOmKeyInfo.class, new RepeatedOmKeyInfoCodec()) .addCodec(OmBucketInfo.class, new OmBucketInfoCodec()) .addCodec(OmVolumeArgs.class, new OmVolumeArgsCodec()) - .addCodec(VolumeList.class, new VolumeListCodec()) + .addCodec(UserVolumeInfo.class, new UserVolumeInfoCodec()) .addCodec(OmMultipartKeyInfo.class, new OmMultipartKeyInfoCodec()) .addCodec(S3SecretValue.class, new S3SecretValueCodec()) .addCodec(OmPrefixInfo.class, new OmPrefixInfoCodec()); @@ -279,7 +280,7 @@ protected DBStoreBuilder addOMTablesAndCodecs(DBStoreBuilder builder) { */ protected void initializeOmTables() throws IOException { userTable = - this.store.getTable(USER_TABLE, String.class, VolumeList.class); + this.store.getTable(USER_TABLE, String.class, UserVolumeInfo.class); checkTableStatus(userTable, USER_TABLE); TableCacheImpl.CacheCleanupPolicy cleanupPolicy = @@ -706,7 +707,7 @@ public List listKeys(String volumeName, String bucketName, public List listVolumes(String userName, String prefix, String startKey, int maxKeys) throws IOException { List result = Lists.newArrayList(); - VolumeList volumes; + UserVolumeInfo volumes; if (StringUtil.isBlank(userName)) { throw new OMException("User name is required to list Volumes.", ResultCodes.USER_NOT_FOUND); @@ -747,15 +748,15 @@ public List listVolumes(String userName, return result; } - private VolumeList getVolumesByUser(String userNameKey) + private UserVolumeInfo getVolumesByUser(String userNameKey) throws OMException { try { - VolumeList volumeList = getUserTable().get(userNameKey); - if (volumeList == null) { + UserVolumeInfo userVolInfo = getUserTable().get(userNameKey); + if (userVolInfo == null) { // No volume found for this user, return an empty list - return VolumeList.newBuilder().build(); + return UserVolumeInfo.newBuilder().build(); } else { - return volumeList; + return userVolInfo; } } catch (IOException e) { throw new OMException("Unable to get volumes info by the given user, " diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index d2d7256fb8939..a6503d73140a3 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -42,7 +42,6 @@ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -74,9 +73,10 @@ import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.OzoneConfigKeys; -import org.apache.hadoop.ozone.OzoneIllegalArgumentException; import org.apache.hadoop.ozone.OzoneSecurityUtil; import org.apache.hadoop.ozone.om.ha.OMFailoverProxyProvider; +import org.apache.hadoop.ozone.om.ha.OMHANodeDetails; +import org.apache.hadoop.ozone.om.ha.OMNodeDetails; import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList; import org.apache.hadoop.ozone.om.helpers.S3SecretValue; import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol; @@ -207,10 +207,6 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METRICS_SAVE_INTERVAL; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METRICS_SAVE_INTERVAL_DEFAULT; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_NODE_ID_KEY; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_PORT_DEFAULT; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_PORT_KEY; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_USER_MAX_VOLUME; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_USER_MAX_VOLUME_DEFAULT; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_AUTH_METHOD; @@ -310,12 +306,32 @@ private OzoneManager(OzoneConfiguration conf) throws IOException, super(OzoneVersionInfo.OZONE_VERSION_INFO); Preconditions.checkNotNull(conf); configuration = conf; + // Load HA related configurations + OMHANodeDetails omhaNodeDetails = + OMHANodeDetails.loadOMHAConfig(configuration); + + this.peerNodes = omhaNodeDetails.getPeerNodeDetails(); + this.omNodeDetails = omhaNodeDetails.getLocalNodeDetails(); + + omStorage = new OMStorage(conf); + omId = omStorage.getOmId(); + + // In case of single OM Node Service there will be no OM Node ID + // specified, set it to value from om storage + if (this.omNodeDetails.getOMNodeId() == null) { + this.omNodeDetails = + OMHANodeDetails.getOMNodeDetails(conf, omNodeDetails.getOMServiceId(), + omStorage.getOmId(), omNodeDetails.getRpcAddress(), + omNodeDetails.getRatisPort()); + } + + loginOMUserIfSecurityEnabled(conf); + this.maxUserVolumeCount = conf.getInt(OZONE_OM_USER_MAX_VOLUME, OZONE_OM_USER_MAX_VOLUME_DEFAULT); Preconditions.checkArgument(this.maxUserVolumeCount > 0, OZONE_OM_USER_MAX_VOLUME + " value should be greater than zero"); - omStorage = new OMStorage(conf); - omId = omStorage.getOmId(); + if (omStorage.getState() != StorageState.INITIALIZED) { throw new OMException("OM not initialized.", ResultCodes.OM_NOT_INITIALIZED); @@ -342,8 +358,7 @@ private OzoneManager(OzoneConfiguration conf) throws IOException, OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY, OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT); - // Load HA related configurations - loadOMHAConfigs(configuration); + InetSocketAddress omNodeRpcAddr = omNodeDetails.getRpcAddress(); omRpcAddressTxt = new Text(omNodeDetails.getRpcAddressString()); @@ -420,7 +435,7 @@ private OzoneManager(OzoneConfiguration conf) throws IOException, OzoneManagerProtocolProtos.Type.values()); // Start Om Rpc Server. - omRpcServer = getRpcServer(conf); + omRpcServer = getRpcServer(configuration); omRpcAddress = updateRPCListenAddress(configuration, OZONE_OM_ADDRESS_KEY, omNodeRpcAddr, omRpcServer); @@ -513,195 +528,6 @@ public boolean isGrpcBlockTokenEnabled() { return grpcBlockTokenEnabled; } - /** - * Inspects and loads OM node configurations. - * - * If {@link OMConfigKeys#OZONE_OM_SERVICE_IDS_KEY} is configured with - * multiple ids and/ or if {@link OMConfigKeys#OZONE_OM_NODE_ID_KEY} is not - * specifically configured , this method determines the omServiceId - * and omNodeId by matching the node's address with the configured - * addresses. When a match is found, it sets the omServicId and omNodeId from - * the corresponding configuration key. This method also finds the OM peers - * nodes belonging to the same OM service. - * - * @param conf - */ - private void loadOMHAConfigs(Configuration conf) { - InetSocketAddress localRpcAddress = null; - String localOMServiceId = null; - String localOMNodeId = null; - int localRatisPort = 0; - Collection omServiceIds = conf.getTrimmedStringCollection( - OZONE_OM_SERVICE_IDS_KEY); - - String knownOMNodeId = conf.get(OZONE_OM_NODE_ID_KEY); - int found = 0; - boolean isOMAddressSet = false; - - for (String serviceId : OmUtils.emptyAsSingletonNull(omServiceIds)) { - Collection omNodeIds = OmUtils.getOMNodeIds(conf, serviceId); - - List peerNodesList = new ArrayList<>(); - boolean isPeer = false; - for (String nodeId : OmUtils.emptyAsSingletonNull(omNodeIds)) { - if (knownOMNodeId != null && !knownOMNodeId.equals(nodeId)) { - isPeer = true; - } else { - isPeer = false; - } - String rpcAddrKey = OmUtils.addKeySuffixes(OZONE_OM_ADDRESS_KEY, - serviceId, nodeId); - String rpcAddrStr = OmUtils.getOmRpcAddress(conf, rpcAddrKey); - if (rpcAddrStr == null) { - continue; - } - - // If OM address is set for any node id, we will not fallback to the - // default - isOMAddressSet = true; - - String ratisPortKey = OmUtils.addKeySuffixes(OZONE_OM_RATIS_PORT_KEY, - serviceId, nodeId); - int ratisPort = conf.getInt(ratisPortKey, OZONE_OM_RATIS_PORT_DEFAULT); - - InetSocketAddress addr = null; - try { - addr = NetUtils.createSocketAddr(rpcAddrStr); - } catch (Exception e) { - LOG.warn("Exception in creating socket address " + addr, e); - continue; - } - if (!addr.isUnresolved()) { - if (!isPeer && OmUtils.isAddressLocal(addr)) { - localRpcAddress = addr; - localOMServiceId = serviceId; - localOMNodeId = nodeId; - localRatisPort = ratisPort; - found++; - } else { - // This OMNode belongs to same OM service as the current OMNode. - // Add it to peerNodes list. - String httpAddr = OmUtils.getHttpAddressForOMPeerNode(conf, - serviceId, nodeId, addr.getHostName()); - String httpsAddr = OmUtils.getHttpsAddressForOMPeerNode(conf, - serviceId, nodeId, addr.getHostName()); - OMNodeDetails peerNodeInfo = new OMNodeDetails.Builder() - .setOMServiceId(serviceId) - .setOMNodeId(nodeId) - .setRpcAddress(addr) - .setRatisPort(ratisPort) - .setHttpAddress(httpAddr) - .setHttpsAddress(httpsAddr) - .build(); - peerNodesList.add(peerNodeInfo); - } - } - } - if (found == 1) { - LOG.debug("Found one matching OM address with service ID: {} and node" + - " ID: {}", localOMServiceId, localOMNodeId); - - setOMNodeDetails(localOMServiceId, localOMNodeId, localRpcAddress, - localRatisPort); - - this.peerNodes = peerNodesList; - - LOG.info("Found matching OM address with OMServiceId: {}, " + - "OMNodeId: {}, RPC Address: {} and Ratis port: {}", - localOMServiceId, localOMNodeId, - NetUtils.getHostPortString(localRpcAddress), localRatisPort); - return; - } else if (found > 1) { - String msg = "Configuration has multiple " + OZONE_OM_ADDRESS_KEY + - " addresses that match local node's address. Please configure the" + - " system with " + OZONE_OM_SERVICE_IDS_KEY + " and " + - OZONE_OM_ADDRESS_KEY; - throw new OzoneIllegalArgumentException(msg); - } - } - - if (!isOMAddressSet) { - // No OM address is set. Fallback to default - InetSocketAddress omAddress = OmUtils.getOmAddress(conf); - int ratisPort = conf.getInt(OZONE_OM_RATIS_PORT_KEY, - OZONE_OM_RATIS_PORT_DEFAULT); - - LOG.info("Configuration either no {} set. Falling back to the default " + - "OM address {}", OZONE_OM_ADDRESS_KEY, omAddress); - - setOMNodeDetails(null, null, omAddress, ratisPort); - - } else { - String msg = "Configuration has no " + OZONE_OM_ADDRESS_KEY + " " + - "address that matches local node's address. Please configure the " + - "system with " + OZONE_OM_ADDRESS_KEY; - LOG.info(msg); - throw new OzoneIllegalArgumentException(msg); - } - } - - /** - * Builds and sets OMNodeDetails object. - */ - private void setOMNodeDetails(String serviceId, String nodeId, - InetSocketAddress rpcAddress, int ratisPort) { - - if (serviceId == null) { - // If no serviceId is set, take the default serviceID om-service - serviceId = OzoneConsts.OM_SERVICE_ID_DEFAULT; - LOG.info("OM Service ID is not set. Setting it to the default ID: {}", - serviceId); - } - if (nodeId == null) { - // If no nodeId is set, take the omId from omStorage as the nodeID - nodeId = omId; - LOG.info("OM Node ID is not set. Setting it to the OmStorage's " + - "OmID: {}", nodeId); - } - - this.omNodeDetails = new OMNodeDetails.Builder() - .setOMServiceId(serviceId) - .setOMNodeId(nodeId) - .setRpcAddress(rpcAddress) - .setRatisPort(ratisPort) - .build(); - - // Set this nodes OZONE_OM_ADDRESS_KEY to the discovered address. - configuration.set(OZONE_OM_ADDRESS_KEY, - NetUtils.getHostPortString(rpcAddress)); - - // Get and set Http(s) address of local node. If base config keys are - // not set, check for keys suffixed with OM serivce ID and node ID. - setOMNodeSpecificConfigs(serviceId, nodeId); - } - - /** - * Check if any of the following configuration keys have been set using OM - * Node ID suffixed to the key. If yes, then set the base key with the - * configured valued. - * 1. {@link OMConfigKeys#OZONE_OM_HTTP_ADDRESS_KEY} - * 2. {@link OMConfigKeys#OZONE_OM_HTTPS_ADDRESS_KEY} - * 3. {@link OMConfigKeys#OZONE_OM_HTTP_BIND_HOST_KEY} - * 4. {@link OMConfigKeys#OZONE_OM_HTTPS_BIND_HOST_KEY} - */ - private void setOMNodeSpecificConfigs(String omServiceId, String omNodeId) { - String[] confKeys = new String[] { - OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY, - OMConfigKeys.OZONE_OM_HTTPS_ADDRESS_KEY, - OMConfigKeys.OZONE_OM_HTTP_BIND_HOST_KEY, - OMConfigKeys.OZONE_OM_HTTPS_BIND_HOST_KEY}; - - for (String confKey : confKeys) { - String confValue = OmUtils.getConfSuffixedWithOMNodeId( - configuration, confKey, omServiceId, omNodeId); - if (confValue != null) { - LOG.info("Setting configuration key {} with value of key {}: {}", - confKey, OmUtils.addKeySuffixes(confKey, omNodeId), confValue); - configuration.set(confKey, confValue); - } - } - } - private KeyProviderCryptoExtension createKeyProviderExt( OzoneConfiguration conf) throws IOException { KeyProvider keyProvider = KMSUtil.createKeyProvider(conf, @@ -801,7 +627,7 @@ private OzoneDelegationTokenSecretManager createDelegationTokenSecretManager( return new OzoneDelegationTokenSecretManager(conf, tokenMaxLifetime, tokenRenewInterval, tokenRemoverScanInterval, omRpcAddressTxt, - s3SecretManager); + s3SecretManager, certClient); } private OzoneBlockTokenSecretManager createBlockTokenSecretManager( @@ -1024,7 +850,6 @@ private static boolean isOzoneSecurityEnabled() { */ public static OzoneManager createOm(OzoneConfiguration conf) throws IOException, AuthenticationException { - loginOMUserIfSecurityEnabled(conf); return new OzoneManager(conf); } @@ -1053,6 +878,7 @@ private static void loginOMUserIfSecurityEnabled(OzoneConfiguration conf) @VisibleForTesting public static boolean omInit(OzoneConfiguration conf) throws IOException, AuthenticationException { + OMHANodeDetails.loadOMHAConfig(conf); loginOMUserIfSecurityEnabled(conf); OMStorage omStorage = new OMStorage(conf); StorageState state = omStorage.getState(); @@ -1361,7 +1187,7 @@ private RPC.Server getRpcServer(OzoneConfiguration conf) throws IOException { return omRpcServer; } - InetSocketAddress omNodeRpcAddr = OmUtils.getOmAddress(configuration); + InetSocketAddress omNodeRpcAddr = OmUtils.getOmAddress(conf); final int handlerCount = conf.getInt(OZONE_OM_HANDLER_COUNT_KEY, OZONE_OM_HANDLER_COUNT_DEFAULT); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java index 675895dacff35..4ea8529e2d319 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java @@ -28,8 +28,10 @@ import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList; +import org.apache.hadoop.ozone.protocol.proto + .OzoneManagerProtocolProtos.OzoneAclInfo; +import org.apache.hadoop.ozone.protocol.proto + .OzoneManagerProtocolProtos.UserVolumeInfo; import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; import org.apache.hadoop.ozone.security.acl.OzoneObj; import org.apache.hadoop.ozone.security.acl.RequestContext; @@ -73,11 +75,11 @@ public VolumeManagerImpl(OMMetadataManager metadataManager, } // Helpers to add and delete volume from user list - private VolumeList addVolumeToOwnerList(String volume, String owner) + private UserVolumeInfo addVolumeToOwnerList(String volume, String owner) throws IOException { // Get the volume list String dbUserKey = metadataManager.getUserKey(owner); - VolumeList volumeList = metadataManager.getUserTable().get(dbUserKey); + UserVolumeInfo volumeList = metadataManager.getUserTable().get(dbUserKey); List prevVolList = new ArrayList<>(); if (volumeList != null) { prevVolList.addAll(volumeList.getVolumeNamesList()); @@ -92,16 +94,16 @@ private VolumeList addVolumeToOwnerList(String volume, String owner) // Add the new volume to the list prevVolList.add(volume); - VolumeList newVolList = VolumeList.newBuilder() + UserVolumeInfo newVolList = UserVolumeInfo.newBuilder() .addAllVolumeNames(prevVolList).build(); return newVolList; } - private VolumeList delVolumeFromOwnerList(String volume, String owner) + private UserVolumeInfo delVolumeFromOwnerList(String volume, String owner) throws IOException { // Get the volume list - VolumeList volumeList = metadataManager.getUserTable().get(owner); + UserVolumeInfo volumeList = metadataManager.getUserTable().get(owner); List prevVolList = new ArrayList<>(); if (volumeList != null) { prevVolList.addAll(volumeList.getVolumeNamesList()); @@ -112,7 +114,7 @@ private VolumeList delVolumeFromOwnerList(String volume, String owner) // Remove the volume from the list prevVolList.remove(volume); - VolumeList newVolList = VolumeList.newBuilder() + UserVolumeInfo newVolList = UserVolumeInfo.newBuilder() .addAllVolumeNames(prevVolList).build(); return newVolList; } @@ -144,7 +146,7 @@ public void createVolume(OmVolumeArgs omVolumeArgs) throws IOException { throw new OMException(ResultCodes.VOLUME_ALREADY_EXISTS); } - VolumeList volumeList = addVolumeToOwnerList(omVolumeArgs.getVolume(), + UserVolumeInfo volumeList = addVolumeToOwnerList(omVolumeArgs.getVolume(), omVolumeArgs.getOwnerName()); // Set creation time @@ -173,7 +175,7 @@ public void createVolume(OmVolumeArgs omVolumeArgs) throws IOException { } private void createVolumeCommitToDB(OmVolumeArgs omVolumeArgs, - VolumeList volumeList, String dbVolumeKey, String dbUserKey) + UserVolumeInfo volumeList, String dbVolumeKey, String dbUserKey) throws IOException { try (BatchOperation batch = metadataManager.getStore() .initBatchOperation()) { @@ -222,11 +224,12 @@ public void setOwner(String volume, String owner) acquiredUsersLock = metadataManager.getLock().acquireMultiUserLock(owner, originalOwner); - VolumeList oldOwnerVolumeList = delVolumeFromOwnerList(volume, + UserVolumeInfo oldOwnerVolumeList = delVolumeFromOwnerList(volume, originalOwner); String newOwner = metadataManager.getUserKey(owner); - VolumeList newOwnerVolumeList = addVolumeToOwnerList(volume, newOwner); + UserVolumeInfo newOwnerVolumeList = addVolumeToOwnerList(volume, + newOwner); volumeArgs.setOwnerName(owner); setOwnerCommitToDB(oldOwnerVolumeList, newOwnerVolumeList, @@ -246,8 +249,8 @@ public void setOwner(String volume, String owner) } - private void setOwnerCommitToDB(VolumeList oldOwnerVolumeList, - VolumeList newOwnerVolumeList, OmVolumeArgs newOwnerVolumeArgs, + private void setOwnerCommitToDB(UserVolumeInfo oldOwnerVolumeList, + UserVolumeInfo newOwnerVolumeList, OmVolumeArgs newOwnerVolumeArgs, String oldOwner) throws IOException { try (BatchOperation batch = metadataManager.getStore() .initBatchOperation()) { @@ -370,7 +373,7 @@ public void deleteVolume(String volume) throws IOException { Preconditions.checkState(volume.equals(volumeArgs.getVolume())); // delete the volume from the owner list // as well as delete the volume entry - VolumeList newVolumeList = delVolumeFromOwnerList(volume, + UserVolumeInfo newVolumeList = delVolumeFromOwnerList(volume, volumeArgs.getOwnerName()); @@ -390,7 +393,7 @@ public void deleteVolume(String volume) throws IOException { } - private void deleteVolumeCommitToDB(VolumeList newVolumeList, + private void deleteVolumeCommitToDB(UserVolumeInfo newVolumeList, String volume, String owner) throws IOException { try (BatchOperation batch = metadataManager.getStore() .initBatchOperation()) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMHANodeDetails.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMHANodeDetails.java new file mode 100644 index 0000000000000..8d9e70977a3d5 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMHANodeDetails.java @@ -0,0 +1,306 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.ozone.om.ha; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.ozone.OmUtils; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.OzoneIllegalArgumentException; +import org.apache.hadoop.ozone.om.OMConfigKeys; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_NODES_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_NODE_ID_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_PORT_DEFAULT; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_PORT_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY; + +/** + * Class which maintains peer information and it's own OM node information. + */ +public class OMHANodeDetails { + + private static String[] genericConfigKeys = new String[] { + OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY, + OMConfigKeys.OZONE_OM_HTTPS_ADDRESS_KEY, + OMConfigKeys.OZONE_OM_HTTP_BIND_HOST_KEY, + OMConfigKeys.OZONE_OM_HTTPS_BIND_HOST_KEY, + OMConfigKeys.OZONE_OM_DB_DIRS, + OMConfigKeys.OZONE_OM_ADDRESS_KEY, + }; + + public static final Logger LOG = + LoggerFactory.getLogger(OMHANodeDetails.class); + private final OMNodeDetails localNodeDetails; + private final List peerNodeDetails; + + public OMHANodeDetails(OMNodeDetails localNodeDetails, + List peerNodeDetails) { + this.localNodeDetails = localNodeDetails; + this.peerNodeDetails = peerNodeDetails; + } + + public OMNodeDetails getLocalNodeDetails() { + return localNodeDetails; + } + + public List< OMNodeDetails > getPeerNodeDetails() { + return peerNodeDetails; + } + + + /** + * Inspects and loads OM node configurations. + * + * If {@link OMConfigKeys#OZONE_OM_SERVICE_IDS_KEY} is configured with + * multiple ids and/ or if {@link OMConfigKeys#OZONE_OM_NODE_ID_KEY} is not + * specifically configured , this method determines the omServiceId + * and omNodeId by matching the node's address with the configured + * addresses. When a match is found, it sets the omServicId and omNodeId from + * the corresponding configuration key. This method also finds the OM peers + * nodes belonging to the same OM service. + * + * @param conf + */ + public static OMHANodeDetails loadOMHAConfig(OzoneConfiguration conf) { + InetSocketAddress localRpcAddress = null; + String localOMServiceId = null; + String localOMNodeId = null; + int localRatisPort = 0; + Collection omServiceIds = conf.getTrimmedStringCollection( + OZONE_OM_SERVICE_IDS_KEY); + + String knownOMNodeId = conf.get(OZONE_OM_NODE_ID_KEY); + int found = 0; + boolean isOMAddressSet = false; + + for (String serviceId : omServiceIds) { + Collection omNodeIds = OmUtils.getOMNodeIds(conf, serviceId); + + if (omNodeIds.size() == 0) { + String msg = "Configuration does not have any value set for " + + OZONE_OM_NODES_KEY + " for service ID " + serviceId + ". List of " + + "OM Node ID's should be specified for the service ID"; + throw new OzoneIllegalArgumentException(msg); + } + + List peerNodesList = new ArrayList<>(); + boolean isPeer; + for (String nodeId : omNodeIds) { + if (knownOMNodeId != null && !knownOMNodeId.equals(nodeId)) { + isPeer = true; + } else { + isPeer = false; + } + String rpcAddrKey = OmUtils.addKeySuffixes(OZONE_OM_ADDRESS_KEY, + serviceId, nodeId); + String rpcAddrStr = OmUtils.getOmRpcAddress(conf, rpcAddrKey); + if (rpcAddrStr == null || rpcAddrStr.isEmpty()) { + String msg = "Configuration does not have any value set for " + + rpcAddrKey + "." + "OM Rpc Address should be set for all node " + + "IDs for a service ID."; + throw new OzoneIllegalArgumentException(msg); + } + + // If OM address is set for any node id, we will not fallback to the + // default + isOMAddressSet = true; + + String ratisPortKey = OmUtils.addKeySuffixes(OZONE_OM_RATIS_PORT_KEY, + serviceId, nodeId); + int ratisPort = conf.getInt(ratisPortKey, OZONE_OM_RATIS_PORT_DEFAULT); + + InetSocketAddress addr = null; + try { + addr = NetUtils.createSocketAddr(rpcAddrStr); + } catch (Exception e) { + LOG.warn("Exception in creating socket address " + addr, e); + continue; + } + if (!addr.isUnresolved()) { + if (!isPeer && OmUtils.isAddressLocal(addr)) { + localRpcAddress = addr; + localOMServiceId = serviceId; + localOMNodeId = nodeId; + localRatisPort = ratisPort; + found++; + } else { + // This OMNode belongs to same OM service as the current OMNode. + // Add it to peerNodes list. + // This OMNode belongs to same OM service as the current OMNode. + // Add it to peerNodes list. + peerNodesList.add(getHAOMNodeDetails(conf, serviceId, + nodeId, addr, ratisPort)); + } + } + } + if (found == 1) { + LOG.debug("Found one matching OM address with service ID: {} and node" + + " ID: {}", localOMServiceId, localOMNodeId); + + LOG.info("Found matching OM address with OMServiceId: {}, " + + "OMNodeId: {}, RPC Address: {} and Ratis port: {}", + localOMServiceId, localOMNodeId, + NetUtils.getHostPortString(localRpcAddress), localRatisPort); + + + setOMNodeSpecificConfigs(conf, localOMServiceId, localOMNodeId); + return new OMHANodeDetails(getHAOMNodeDetails(conf, localOMServiceId, + localOMNodeId, localRpcAddress, localRatisPort), peerNodesList); + + } else if (found > 1) { + String msg = "Configuration has multiple " + OZONE_OM_ADDRESS_KEY + + " addresses that match local node's address. Please configure the" + + " system with " + OZONE_OM_SERVICE_IDS_KEY + " and " + + OZONE_OM_ADDRESS_KEY; + throw new OzoneIllegalArgumentException(msg); + } + } + + if (!isOMAddressSet) { + // No OM address is set. Fallback to default + InetSocketAddress omAddress = OmUtils.getOmAddress(conf); + int ratisPort = conf.getInt(OZONE_OM_RATIS_PORT_KEY, + OZONE_OM_RATIS_PORT_DEFAULT); + + LOG.info("Configuration either no {} set. Falling back to the default " + + "OM address {}", OZONE_OM_ADDRESS_KEY, omAddress); + + return new OMHANodeDetails(getOMNodeDetails(conf, null, + null, omAddress, ratisPort), new ArrayList<>()); + + } else { + String msg = "Configuration has no " + OZONE_OM_ADDRESS_KEY + " " + + "address that matches local node's address. Please configure the " + + "system with " + OZONE_OM_ADDRESS_KEY; + LOG.info(msg); + throw new OzoneIllegalArgumentException(msg); + } + } + + /** + * Create Local OM Node Details. + * @param serviceId - Service ID this OM belongs to, + * @param nodeId - Node ID of this OM. + * @param rpcAddress - Rpc Address of the OM. + * @param ratisPort - Ratis port of the OM. + * @return OMNodeDetails + */ + public static OMNodeDetails getOMNodeDetails(OzoneConfiguration conf, + String serviceId, String nodeId, InetSocketAddress rpcAddress, + int ratisPort) { + + if (serviceId == null) { + // If no serviceId is set, take the default serviceID om-service + serviceId = OzoneConsts.OM_SERVICE_ID_DEFAULT; + LOG.info("OM Service ID is not set. Setting it to the default ID: {}", + serviceId); + } + + + // We need to pass null for serviceID and nodeID as this is set for + // non-HA cluster. This means one node OM cluster. + String httpAddr = OmUtils.getHttpAddressForOMPeerNode(conf, + null, null, rpcAddress.getHostName()); + String httpsAddr = OmUtils.getHttpsAddressForOMPeerNode(conf, + null, null, rpcAddress.getHostName()); + + return new OMNodeDetails.Builder() + .setOMServiceId(serviceId) + .setOMNodeId(nodeId) + .setRpcAddress(rpcAddress) + .setRatisPort(ratisPort) + .setHttpAddress(httpAddr) + .setHttpsAddress(httpsAddr) + .build(); + + } + + + /** + * Create Local OM Node Details. + * @param serviceId - Service ID this OM belongs to, + * @param nodeId - Node ID of this OM. + * @param rpcAddress - Rpc Address of the OM. + * @param ratisPort - Ratis port of the OM. + * @return OMNodeDetails + */ + public static OMNodeDetails getHAOMNodeDetails(OzoneConfiguration conf, + String serviceId, String nodeId, InetSocketAddress rpcAddress, + int ratisPort) { + Preconditions.checkNotNull(serviceId); + Preconditions.checkNotNull(nodeId); + + String httpAddr = OmUtils.getHttpAddressForOMPeerNode(conf, + serviceId, nodeId, rpcAddress.getHostName()); + String httpsAddr = OmUtils.getHttpsAddressForOMPeerNode(conf, + serviceId, nodeId, rpcAddress.getHostName()); + + return new OMNodeDetails.Builder() + .setOMServiceId(serviceId) + .setOMNodeId(nodeId) + .setRpcAddress(rpcAddress) + .setRatisPort(ratisPort) + .setHttpAddress(httpAddr) + .setHttpsAddress(httpsAddr) + .build(); + + } + + + /** + * Check if any of the following configuration keys have been set using OM + * Node ID suffixed to the key. If yes, then set the base key with the + * configured valued. + * 1. {@link OMConfigKeys#OZONE_OM_HTTP_ADDRESS_KEY} + * 2. {@link OMConfigKeys#OZONE_OM_HTTPS_ADDRESS_KEY} + * 3. {@link OMConfigKeys#OZONE_OM_HTTP_BIND_HOST_KEY} + * 4. {@link OMConfigKeys#OZONE_OM_HTTPS_BIND_HOST_KEY}\ + * 5. {@link OMConfigKeys#OZONE_OM_HTTP_KERBEROS_KEYTAB_FILE} + * 6. {@link OMConfigKeys#OZONE_OM_HTTP_KERBEROS_PRINCIPAL_KEY} + * 7. {@link OMConfigKeys#OZONE_OM_KERBEROS_KEYTAB_FILE_KEY} + * 8. {@link OMConfigKeys#OZONE_OM_KERBEROS_PRINCIPAL_KEY} + * 9. {@link OMConfigKeys#OZONE_OM_DB_DIRS} + * 10. {@link OMConfigKeys#OZONE_OM_ADDRESS_KEY} + */ + private static void setOMNodeSpecificConfigs( + OzoneConfiguration ozoneConfiguration, String omServiceId, + String omNodeId) { + + for (String confKey : genericConfigKeys) { + String confValue = OmUtils.getConfSuffixedWithOMNodeId( + ozoneConfiguration, confKey, omServiceId, omNodeId); + if (confValue != null) { + LOG.info("Setting configuration key {} with value of key {}: {}", + confKey, OmUtils.addKeySuffixes(confKey, omNodeId), confValue); + ozoneConfiguration.set(confKey, confValue); + } + } + } + + +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMNodeDetails.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMNodeDetails.java similarity index 99% rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMNodeDetails.java rename to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMNodeDetails.java index fc8c818b1b5bf..7d69b933e8298 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMNodeDetails.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMNodeDetails.java @@ -15,7 +15,7 @@ * the License. */ -package org.apache.hadoop.ozone.om; +package org.apache.hadoop.ozone.om.ha; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.http.HttpConfig; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/package-info.java new file mode 100644 index 0000000000000..3c40c88b9545a --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/package-info.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.om.ha; + +/** + * This package contains classes related to OM HA. + */ diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java index c341d30525ac8..69a7ae93a81aa 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java @@ -40,7 +40,7 @@ import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.ozone.om.OMConfigKeys; -import org.apache.hadoop.ozone.om.OMNodeDetails; +import org.apache.hadoop.ozone.om.ha.OMNodeDetails; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.helpers.OMRatisHelper; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketCreateRequest.java index 73772982a73f2..7a7091d1a141e 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketCreateRequest.java @@ -57,8 +57,7 @@ .S3CreateBucketResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .S3CreateVolumeInfo; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .VolumeList; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserVolumeInfo; import org.apache.hadoop.util.Time; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; @@ -176,7 +175,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, OmVolumeArgs omVolumeArgs = createOmVolumeArgs(volumeName, userName, s3CreateBucketRequest.getS3CreateVolumeInfo() .getCreationTime()); - VolumeList volumeList = omMetadataManager.getUserTable().get( + UserVolumeInfo volumeList = omMetadataManager.getUserTable().get( omMetadataManager.getUserKey(userName)); volumeList = addVolumeToOwnerList(volumeList, volumeName, userName, ozoneManager.getMaxUserVolumeCount(), diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeCreateRequest.java index 3568b3ba23a4a..c06069c2567ac 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeCreateRequest.java @@ -47,8 +47,7 @@ .OMResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .VolumeInfo; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .VolumeList; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserVolumeInfo; import org.apache.hadoop.util.Time; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS_WILDCARD; @@ -133,7 +132,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, } } - VolumeList volumeList = null; + UserVolumeInfo volumeList = null; // acquire lock. acquiredVolumeLock = omMetadataManager.getLock().acquireLock(VOLUME_LOCK, diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java index 9bd9a1d5aed42..485536fc32870 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java @@ -95,7 +95,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, } OmVolumeArgs omVolumeArgs = null; - OzoneManagerProtocolProtos.VolumeList newVolumeList = null; + OzoneManagerProtocolProtos.UserVolumeInfo newVolumeList = null; acquiredVolumeLock = omMetadataManager.getLock().acquireLock(VOLUME_LOCK, volume); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeRequest.java index a5e7645b45574..7c38c41320d61 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeRequest.java @@ -26,7 +26,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .OMRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .VolumeList; + .UserVolumeInfo; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; @@ -50,10 +50,10 @@ public OMVolumeRequest(OMRequest omRequest) { * @param volume - volume which needs to deleted from the volume list. * @param owner - Name of the Owner. * @param txID - The transaction ID that is updating this value. - * @return VolumeList - updated volume list for the user. + * @return UserVolumeInfo - updated UserVolumeInfo. * @throws IOException */ - protected VolumeList delVolumeFromOwnerList(VolumeList volumeList, + protected UserVolumeInfo delVolumeFromOwnerList(UserVolumeInfo volumeList, String volume, String owner, long txID) throws IOException { List prevVolList = new ArrayList<>(); @@ -68,7 +68,7 @@ protected VolumeList delVolumeFromOwnerList(VolumeList volumeList, // Remove the volume from the list prevVolList.remove(volume); - VolumeList newVolList = VolumeList.newBuilder() + UserVolumeInfo newVolList = UserVolumeInfo.newBuilder() .addAllVolumeNames(prevVolList) .setObjectID(volumeList.getObjectID()) .setUpdateID(txID) @@ -88,7 +88,7 @@ protected VolumeList delVolumeFromOwnerList(VolumeList volumeList, * @throws OMException - if user has volumes greater than * maxUserVolumeCount, an exception is thrown. */ - protected VolumeList addVolumeToOwnerList(VolumeList volumeList, + protected UserVolumeInfo addVolumeToOwnerList(UserVolumeInfo volumeList, String volume, String owner, long maxUserVolumeCount, long txID) throws IOException { @@ -109,7 +109,7 @@ protected VolumeList addVolumeToOwnerList(VolumeList volumeList, // Add the new volume to the list prevVolList.add(volume); - VolumeList newVolList = VolumeList.newBuilder() + UserVolumeInfo newVolList = UserVolumeInfo.newBuilder() .setObjectID(objectID) .setUpdateID(txID) .addAllVolumeNames(prevVolList).build(); @@ -129,7 +129,7 @@ protected VolumeList addVolumeToOwnerList(VolumeList volumeList, * @throws IOException */ protected void createVolume(final OMMetadataManager omMetadataManager, - OmVolumeArgs omVolumeArgs, VolumeList volumeList, String dbVolumeKey, + OmVolumeArgs omVolumeArgs, UserVolumeInfo volumeList, String dbVolumeKey, String dbUserKey, long transactionLogIndex) { // Update cache: Update user and volume cache. omMetadataManager.getUserTable().addCacheEntry(new CacheKey<>(dbUserKey), diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java index 0dd0ff7bf5780..129b2f9fdd8e6 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java @@ -117,8 +117,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, String dbVolumeKey = omMetadataManager.getVolumeKey(volume); - OzoneManagerProtocolProtos.VolumeList oldOwnerVolumeList = null; - OzoneManagerProtocolProtos.VolumeList newOwnerVolumeList = null; + OzoneManagerProtocolProtos.UserVolumeInfo oldOwnerVolumeList = null; + OzoneManagerProtocolProtos.UserVolumeInfo newOwnerVolumeList = null; OmVolumeArgs omVolumeArgs = null; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeCreateResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeCreateResponse.java index db2ded4874e06..1bd3e4fd8c701 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeCreateResponse.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeCreateResponse.java @@ -27,8 +27,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .OMResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .VolumeList; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserVolumeInfo; import org.apache.hadoop.hdds.utils.db.BatchOperation; @@ -39,14 +38,14 @@ */ public class OMVolumeCreateResponse extends OMClientResponse { - private VolumeList volumeList; + private UserVolumeInfo userVolumeInfo; private OmVolumeArgs omVolumeArgs; public OMVolumeCreateResponse(OmVolumeArgs omVolumeArgs, - VolumeList volumeList, @Nonnull OMResponse omResponse) { + UserVolumeInfo userVolumeInfo, @Nonnull OMResponse omResponse) { super(omResponse); this.omVolumeArgs = omVolumeArgs; - this.volumeList = volumeList; + this.userVolumeInfo = userVolumeInfo; } @Override public void addToDBBatch(OMMetadataManager omMetadataManager, @@ -63,7 +62,7 @@ public void addToDBBatch(OMMetadataManager omMetadataManager, omMetadataManager.getVolumeTable().putWithBatch(batchOperation, dbVolumeKey, omVolumeArgs); omMetadataManager.getUserTable().putWithBatch(batchOperation, dbUserKey, - volumeList); + userVolumeInfo); } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeDeleteResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeDeleteResponse.java index 873747ed7c800..6718ce5be6c8c 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeDeleteResponse.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeDeleteResponse.java @@ -26,7 +26,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .OMResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .VolumeList; + .UserVolumeInfo; import org.apache.hadoop.hdds.utils.db.BatchOperation; import javax.annotation.Nonnull; @@ -37,10 +37,10 @@ public class OMVolumeDeleteResponse extends OMClientResponse { private String volume; private String owner; - private VolumeList updatedVolumeList; + private UserVolumeInfo updatedVolumeList; public OMVolumeDeleteResponse(String volume, String owner, - VolumeList updatedVolumeList, @Nonnull OMResponse omResponse) { + UserVolumeInfo updatedVolumeList, @Nonnull OMResponse omResponse) { super(omResponse); this.volume = volume; this.owner = owner; @@ -55,7 +55,7 @@ public void addToDBBatch(OMMetadataManager omMetadataManager, // not called in failure scenario in OM code. if (getOMResponse().getStatus() == OzoneManagerProtocolProtos.Status.OK) { String dbUserKey = omMetadataManager.getUserKey(owner); - VolumeList volumeList = updatedVolumeList; + UserVolumeInfo volumeList = updatedVolumeList; if (updatedVolumeList.getVolumeNamesList().size() == 0) { omMetadataManager.getUserTable().deleteWithBatch(batchOperation, dbUserKey); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeSetOwnerResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeSetOwnerResponse.java index ddf1512d94222..8e0270215f264 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeSetOwnerResponse.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/volume/OMVolumeSetOwnerResponse.java @@ -26,7 +26,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .VolumeList; + .UserVolumeInfo; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .OMResponse; import org.apache.hadoop.hdds.utils.db.BatchOperation; @@ -39,12 +39,12 @@ public class OMVolumeSetOwnerResponse extends OMClientResponse { private String oldOwner; - private VolumeList oldOwnerVolumeList; - private VolumeList newOwnerVolumeList; + private UserVolumeInfo oldOwnerVolumeList; + private UserVolumeInfo newOwnerVolumeList; private OmVolumeArgs newOwnerVolumeArgs; public OMVolumeSetOwnerResponse(String oldOwner, - VolumeList oldOwnerVolumeList, VolumeList newOwnerVolumeList, + UserVolumeInfo oldOwnerVolumeList, UserVolumeInfo newOwnerVolumeList, OmVolumeArgs newOwnerVolumeArgs, @Nonnull OMResponse omResponse) { super(omResponse); this.oldOwner = oldOwner; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/OzoneManagerSnapshotProvider.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/OzoneManagerSnapshotProvider.java index d77536ae68432..5bca52dee4060 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/OzoneManagerSnapshotProvider.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/OzoneManagerSnapshotProvider.java @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.http.HttpConfig; -import org.apache.hadoop.ozone.om.OMNodeDetails; +import org.apache.hadoop.ozone.om.ha.OMNodeDetails; import org.apache.hadoop.hdds.utils.db.DBCheckpoint; import org.apache.hadoop.hdds.utils.db.RocksDBCheckpoint; import org.apache.http.Header; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/ObjectPrinter.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/ObjectPrinter.java index 064d30ad1c20f..2a17275a56670 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/ObjectPrinter.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/ObjectPrinter.java @@ -29,8 +29,7 @@ private ObjectPrinter() { } public static String getObjectAsJson(Object o) throws IOException { - return JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString(o)); + return JsonUtils.toJsonStringWithDefaultPrettyPrinter(o); } public static void printObjectAsJson(Object o) throws IOException { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/AddAclBucketHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/AddAclBucketHandler.java index 6b32f6400bd18..112e8f38079d8 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/AddAclBucketHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/AddAclBucketHandler.java @@ -24,7 +24,6 @@ import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.OzoneAddress; import org.apache.hadoop.ozone.web.ozShell.Shell; -import org.apache.hadoop.ozone.web.utils.JsonUtils; import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Parameters; @@ -92,8 +91,8 @@ public Void call() throws Exception { boolean result = client.getObjectStore().addAcl(obj, OzoneAcl.parseAcl(acl)); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString("Acl set successfully: " + result))); + System.out.printf("%s%n", "Acl added successfully: " + result); + client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/GetAclBucketHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/GetAclBucketHandler.java index 0bb967c62f551..ccb5d46fcb128 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/GetAclBucketHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/GetAclBucketHandler.java @@ -75,8 +75,8 @@ public Void call() throws Exception { List result = client.getObjectStore().getAcl(obj); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString(result))); + System.out.printf("%s%n", + JsonUtils.toJsonStringWithDefaultPrettyPrinter(result)); client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/RemoveAclBucketHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/RemoveAclBucketHandler.java index 635c34bd66f55..216f66c56293e 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/RemoveAclBucketHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/RemoveAclBucketHandler.java @@ -24,7 +24,6 @@ import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.OzoneAddress; import org.apache.hadoop.ozone.web.ozShell.Shell; -import org.apache.hadoop.ozone.web.utils.JsonUtils; import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Parameters; @@ -68,7 +67,7 @@ public class RemoveAclBucketHandler extends Handler { */ @Override public Void call() throws Exception { - Objects.requireNonNull(acl, "New acl to be added not specified."); + Objects.requireNonNull(acl, "ACL to be removed not specified."); OzoneAddress address = new OzoneAddress(uri); address.ensureBucketAddress(); OzoneClient client = address.createClient(createOzoneConfiguration()); @@ -92,8 +91,8 @@ public Void call() throws Exception { boolean result = client.getObjectStore().removeAcl(obj, OzoneAcl.parseAcl(acl)); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString("Acl removed successfully: " + result))); + System.out.printf("%s%n", "Acl removed successfully: " + result); + client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/SetAclBucketHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/SetAclBucketHandler.java index 2fc43f9bd0255..e603068198a03 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/SetAclBucketHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/SetAclBucketHandler.java @@ -24,7 +24,6 @@ import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.OzoneAddress; import org.apache.hadoop.ozone.web.ozShell.Shell; -import org.apache.hadoop.ozone.web.utils.JsonUtils; import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Parameters; @@ -92,8 +91,8 @@ public Void call() throws Exception { boolean result = client.getObjectStore().setAcl(obj, OzoneAcl.parseAcls(acls)); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString("Acl set successfully: " + result))); + System.out.printf("%s%n", "Acl set successfully: " + result); + client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/AddAclKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/AddAclKeyHandler.java index 13298dceb526f..b4e81345b849f 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/AddAclKeyHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/AddAclKeyHandler.java @@ -24,7 +24,6 @@ import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.OzoneAddress; import org.apache.hadoop.ozone.web.ozShell.Shell; -import org.apache.hadoop.ozone.web.utils.JsonUtils; import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Parameters; @@ -95,8 +94,8 @@ public Void call() throws Exception { boolean result = client.getObjectStore().addAcl(obj, OzoneAcl.parseAcl(acl)); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString("Acl set successfully: " + result))); + System.out.printf("%s%n", "Acl added successfully: " + result); + client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetAclKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetAclKeyHandler.java index edfa66aa30948..6423dbbb6e468 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetAclKeyHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetAclKeyHandler.java @@ -78,8 +78,8 @@ public Void call() throws Exception { List result = client.getObjectStore().getAcl(obj); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString(result))); + System.out.printf("%s%n", + JsonUtils.toJsonStringWithDefaultPrettyPrinter(result)); client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/RemoveAclKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/RemoveAclKeyHandler.java index 1359721642310..f561aa2aeb6d1 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/RemoveAclKeyHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/RemoveAclKeyHandler.java @@ -24,7 +24,6 @@ import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.OzoneAddress; import org.apache.hadoop.ozone.web.ozShell.Shell; -import org.apache.hadoop.ozone.web.utils.JsonUtils; import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Parameters; @@ -68,7 +67,7 @@ public class RemoveAclKeyHandler extends Handler { */ @Override public Void call() throws Exception { - Objects.requireNonNull(acl, "New acl to be added not specified."); + Objects.requireNonNull(acl, "ACL to be removed not specified."); OzoneAddress address = new OzoneAddress(uri); address.ensureKeyAddress(); OzoneClient client = address.createClient(createOzoneConfiguration()); @@ -95,8 +94,8 @@ public Void call() throws Exception { boolean result = client.getObjectStore().removeAcl(obj, OzoneAcl.parseAcl(acl)); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString("Acl set successfully: " + result))); + System.out.printf("%s%n", "Acl removed successfully: " + result); + client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/SetAclKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/SetAclKeyHandler.java index 397330591ea3a..a6a4872f95324 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/SetAclKeyHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/SetAclKeyHandler.java @@ -24,7 +24,6 @@ import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.OzoneAddress; import org.apache.hadoop.ozone.web.ozShell.Shell; -import org.apache.hadoop.ozone.web.utils.JsonUtils; import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Parameters; @@ -94,8 +93,8 @@ public Void call() throws Exception { boolean result = client.getObjectStore().setAcl(obj, OzoneAcl.parseAcls(acls)); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString("Acl set successfully: " + result))); + System.out.printf("%s%n", "Acl set successfully: " + result); + client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/GetTokenHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/GetTokenHandler.java index 7626b6598f951..6d1777c7d3bb4 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/GetTokenHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/GetTokenHandler.java @@ -71,7 +71,7 @@ public Void call() throws Exception { } System.out.printf("%s", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString(token.encodeToUrlString()))); + token.encodeToUrlString())); return null; } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/PrintTokenHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/PrintTokenHandler.java index 93e4c24752705..24f910081127c 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/PrintTokenHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/PrintTokenHandler.java @@ -65,7 +65,7 @@ public Void call() throws Exception { token.decodeFromUrlString(encodedToken); System.out.printf("%s", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString(token.toString()))); + token.toString())); return null; } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/AddAclVolumeHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/AddAclVolumeHandler.java index acce64860dac5..b9d57436287d6 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/AddAclVolumeHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/AddAclVolumeHandler.java @@ -24,7 +24,6 @@ import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.OzoneAddress; import org.apache.hadoop.ozone.web.ozShell.Shell; -import org.apache.hadoop.ozone.web.utils.JsonUtils; import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Parameters; @@ -89,8 +88,8 @@ public Void call() throws Exception { boolean result = client.getObjectStore().addAcl(obj, OzoneAcl.parseAcl(acl)); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString("Acl set successfully: " + result))); + System.out.printf("%s%n", "Acl added successfully: " + result); + client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/GetAclVolumeHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/GetAclVolumeHandler.java index b4be3f8249d40..6c0bb207dd7db 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/GetAclVolumeHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/GetAclVolumeHandler.java @@ -69,8 +69,8 @@ public Void call() throws Exception { OzoneObj.StoreType.valueOf(storeType)) .build(); List result = client.getObjectStore().getAcl(obj); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString(result))); + System.out.printf("%s%n", + JsonUtils.toJsonStringWithDefaultPrettyPrinter(result)); client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/RemoveAclVolumeHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/RemoveAclVolumeHandler.java index 9b3420b3f3a6e..d984f4891f538 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/RemoveAclVolumeHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/RemoveAclVolumeHandler.java @@ -24,7 +24,6 @@ import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.OzoneAddress; import org.apache.hadoop.ozone.web.ozShell.Shell; -import org.apache.hadoop.ozone.web.utils.JsonUtils; import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Parameters; @@ -68,7 +67,7 @@ public class RemoveAclVolumeHandler extends Handler { */ @Override public Void call() throws Exception { - Objects.requireNonNull(acl, "New acl to be added not specified."); + Objects.requireNonNull(acl, "ACL to be removed not specified."); OzoneAddress address = new OzoneAddress(uri); address.ensureVolumeAddress(); OzoneClient client = address.createClient(createOzoneConfiguration()); @@ -89,8 +88,8 @@ public Void call() throws Exception { boolean result = client.getObjectStore().removeAcl(obj, OzoneAcl.parseAcl(acl)); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString("Acl removed successfully: " + result))); + System.out.printf("%s%n", "Acl removed successfully: " + result); + client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/SetAclVolumeHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/SetAclVolumeHandler.java index e3299e35946fc..185f862e2942b 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/SetAclVolumeHandler.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/SetAclVolumeHandler.java @@ -24,7 +24,6 @@ import org.apache.hadoop.ozone.web.ozShell.Handler; import org.apache.hadoop.ozone.web.ozShell.OzoneAddress; import org.apache.hadoop.ozone.web.ozShell.Shell; -import org.apache.hadoop.ozone.web.utils.JsonUtils; import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Parameters; @@ -92,8 +91,8 @@ public Void call() throws Exception { boolean result = client.getObjectStore().setAcl(obj, OzoneAcl.parseAcls(acls)); - System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter( - JsonUtils.toJsonString("Acl set successfully: " + result))); + System.out.printf("%s%n", "Acl set successfully: " + result); + client.close(); return null; } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyDeletingService.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyDeletingService.java index f2e992b90395e..3c707ba1e18bd 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyDeletingService.java @@ -95,6 +95,7 @@ public void checkIfDeleteServiceisDeletingKeys() new KeyManagerImpl( new ScmBlockLocationTestingClient(null, null, 0), metaMgr, conf, UUID.randomUUID().toString(), null); + keyManager.start(conf); final int keyCount = 100; createAndDeleteKeys(keyManager, keyCount, 1); KeyDeletingService keyDeletingService = @@ -117,6 +118,7 @@ public void checkIfDeleteServiceWithFailingSCM() new KeyManagerImpl( new ScmBlockLocationTestingClient(null, null, 1), metaMgr, conf, UUID.randomUUID().toString(), null); + keyManager.start(conf); final int keyCount = 100; createAndDeleteKeys(keyManager, keyCount, 1); KeyDeletingService keyDeletingService = @@ -144,6 +146,7 @@ public void checkDeletionForEmptyKey() new KeyManagerImpl( new ScmBlockLocationTestingClient(null, null, 1), metaMgr, conf, UUID.randomUUID().toString(), null); + keyManager.start(conf); final int keyCount = 100; createAndDeleteKeys(keyManager, keyCount, 0); KeyDeletingService keyDeletingService = diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java index 68649eb24457b..c04fba24e19f9 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java @@ -32,7 +32,7 @@ import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.OMMetadataManager; -import org.apache.hadoop.ozone.om.OMNodeDetails; +import org.apache.hadoop.ozone.om.ha.OMNodeDetails; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java index 2a0e835691f4f..88848f8b2a8fc 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java @@ -266,15 +266,15 @@ public static List< HddsProtos.KeyValue> getMetadataList() { */ public static void addUserToDB(String volumeName, String ownerName, OMMetadataManager omMetadataManager) throws Exception { - OzoneManagerProtocolProtos.VolumeList volumeList = - OzoneManagerProtocolProtos.VolumeList + OzoneManagerProtocolProtos.UserVolumeInfo userVolumeInfo = + OzoneManagerProtocolProtos.UserVolumeInfo .newBuilder() .addVolumeNames(volumeName) .setObjectID(1) .setUpdateID(1) .build(); omMetadataManager.getUserTable().put( - omMetadataManager.getUserKey(ownerName), volumeList); + omMetadataManager.getUserKey(ownerName), userVolumeInfo); } /** diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/package-info.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/package-info.java new file mode 100644 index 0000000000000..0bdab7d655aec --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +/** + * Tests for OM request. + */ +package org.apache.hadoop.ozone.om.request; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeCreateRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeCreateRequest.java index 57f721ab51050..b685711416a3f 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeCreateRequest.java @@ -140,10 +140,10 @@ public void testValidateAndUpdateCacheSuccess() throws Exception { Assert.assertEquals(volumeInfo.getCreationTime(), omVolumeArgs.getCreationTime()); - OzoneManagerProtocolProtos.VolumeList volumeList = omMetadataManager + OzoneManagerProtocolProtos.UserVolumeInfo userVolumeInfo = omMetadataManager .getUserTable().get(ownerKey); - Assert.assertNotNull(volumeList); - Assert.assertEquals(volumeName, volumeList.getVolumeNames(0)); + Assert.assertNotNull(userVolumeInfo); + Assert.assertEquals(volumeName, userVolumeInfo.getVolumeNames(0)); // Create another volume for the user. originalRequest = createVolumeRequest("vol1", adminName, diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeSetOwnerRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeSetOwnerRequest.java index d67ac08c78f43..af38ba03875a2 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeSetOwnerRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeSetOwnerRequest.java @@ -90,14 +90,14 @@ public void testValidateAndUpdateCacheSuccess() throws Exception { Assert.assertEquals(newOwner, fromDBOwner); - OzoneManagerProtocolProtos.VolumeList newOwnerVolumeList = + OzoneManagerProtocolProtos.UserVolumeInfo newOwnerVolumeList = omMetadataManager.getUserTable().get(newOwnerKey); Assert.assertNotNull(newOwnerVolumeList); Assert.assertEquals(volumeName, newOwnerVolumeList.getVolumeNamesList().get(0)); - OzoneManagerProtocolProtos.VolumeList oldOwnerVolumeList = + OzoneManagerProtocolProtos.UserVolumeInfo oldOwnerVolumeList = omMetadataManager.getUserTable().get( omMetadataManager.getUserKey(ownerKey)); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestOMResponseUtils.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestOMResponseUtils.java index 2f4dbb94a9801..5e41d2d513467 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestOMResponseUtils.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestOMResponseUtils.java @@ -56,8 +56,8 @@ public static S3BucketCreateResponse createS3BucketResponse(String userName, .getDefaultInstance()) .build(); - OzoneManagerProtocolProtos.VolumeList volumeList = - OzoneManagerProtocolProtos.VolumeList.newBuilder() + OzoneManagerProtocolProtos.UserVolumeInfo userVolumeInfo = + OzoneManagerProtocolProtos.UserVolumeInfo.newBuilder() .setObjectID(1) .setUpdateID(1) .addVolumeNames(volumeName).build(); @@ -67,7 +67,7 @@ public static S3BucketCreateResponse createS3BucketResponse(String userName, .setVolume(volumeName).setCreationTime(Time.now()).build(); OMVolumeCreateResponse omVolumeCreateResponse = - new OMVolumeCreateResponse(omVolumeArgs, volumeList, omResponse); + new OMVolumeCreateResponse(omVolumeArgs, userVolumeInfo, omResponse); OmBucketInfo omBucketInfo = TestOMResponseUtils.createBucket( diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/package-info.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/package-info.java new file mode 100644 index 0000000000000..fd48e14db9780 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +/** + * Tests for OM Response. + */ +package org.apache.hadoop.ozone.om.response; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java index f6d18883bcabe..b69d8b7b07ef7 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java @@ -26,10 +26,10 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .CreateVolumeResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .VolumeList; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .OMResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .UserVolumeInfo; import org.apache.hadoop.util.Time; import org.apache.hadoop.hdds.utils.db.BatchOperation; import org.junit.Assert; @@ -68,7 +68,7 @@ public void testAddToDBBatch() throws Exception { String volumeName = UUID.randomUUID().toString(); String userName = "user1"; - VolumeList volumeList = VolumeList.newBuilder() + UserVolumeInfo volumeList = UserVolumeInfo.newBuilder() .setObjectID(1).setUpdateID(1) .addVolumeNames(volumeName).build(); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java index 12e7388f83023..5d6b48127ee27 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java @@ -26,10 +26,10 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .CreateVolumeResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .VolumeList; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .OMResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .UserVolumeInfo; import org.apache.hadoop.util.Time; import org.apache.hadoop.hdds.utils.db.BatchOperation; import org.junit.Assert; @@ -68,7 +68,7 @@ public void testAddToDBBatch() throws Exception { String volumeName = UUID.randomUUID().toString(); String userName = "user1"; - VolumeList volumeList = VolumeList.newBuilder() + UserVolumeInfo volumeList = UserVolumeInfo.newBuilder() .setObjectID(1) .setUpdateID(1) .addVolumeNames(volumeName).build(); @@ -87,7 +87,7 @@ public void testAddToDBBatch() throws Exception { new OMVolumeCreateResponse(omVolumeArgs, volumeList, omResponse); // As we are deleting updated volume list should be empty. - VolumeList updatedVolumeList = VolumeList.newBuilder() + UserVolumeInfo updatedVolumeList = UserVolumeInfo.newBuilder() .setObjectID(1).setUpdateID(1).build(); OMVolumeDeleteResponse omVolumeDeleteResponse = new OMVolumeDeleteResponse(volumeName, userName, updatedVolumeList, diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java index 38255bac357f5..0951c062994b9 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java @@ -26,10 +26,10 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .CreateVolumeResponse; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos - .VolumeList; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos .OMResponse; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos + .UserVolumeInfo; import org.apache.hadoop.util.Time; import org.apache.hadoop.hdds.utils.db.BatchOperation; import org.apache.hadoop.hdds.utils.db.Table; @@ -69,7 +69,7 @@ public void testAddToDBBatch() throws Exception { String volumeName = UUID.randomUUID().toString(); String oldOwner = "user1"; - VolumeList volumeList = VolumeList.newBuilder() + UserVolumeInfo volumeList = UserVolumeInfo.newBuilder() .setObjectID(1) .setUpdateID(1) .addVolumeNames(volumeName).build(); @@ -90,11 +90,11 @@ public void testAddToDBBatch() throws Exception { String newOwner = "user2"; - VolumeList newOwnerVolumeList = VolumeList.newBuilder() + UserVolumeInfo newOwnerVolumeList = UserVolumeInfo.newBuilder() .setObjectID(1) .setUpdateID(1) .addVolumeNames(volumeName).build(); - VolumeList oldOwnerVolumeList = VolumeList.newBuilder() + UserVolumeInfo oldOwnerVolumeList = UserVolumeInfo.newBuilder() .setObjectID(2) .setUpdateID(2) .build(); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/package-info.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/package-info.java new file mode 100644 index 0000000000000..98788cd722b3b --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/package-info.java @@ -0,0 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Test Volume functions. + */ +package org.apache.hadoop.ozone.om.response.volume; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/TestOzoneDelegationTokenSecretManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/TestOzoneDelegationTokenSecretManager.java index f05a1e80be6de..874252d171faf 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/TestOzoneDelegationTokenSecretManager.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/TestOzoneDelegationTokenSecretManager.java @@ -169,8 +169,15 @@ public void testCreateToken() throws Exception { validateHash(token.getPassword(), token.getIdentifier()); } - @Test - public void testRenewTokenSuccess() throws Exception { + private void restartSecretManager() throws IOException { + secretManager.stop(); + secretManager = null; + secretManager = createSecretManager(conf, tokenMaxLifetime, + expiryTime, tokenRemoverScanInterval); + } + + private void testRenewTokenSuccessHelper(boolean restartSecretManager) + throws Exception { secretManager = createSecretManager(conf, tokenMaxLifetime, expiryTime, tokenRemoverScanInterval); secretManager.start(certificateClient); @@ -178,10 +185,25 @@ public void testRenewTokenSuccess() throws Exception { TEST_USER, TEST_USER); Thread.sleep(10 * 5); + + if (restartSecretManager) { + restartSecretManager(); + } + long renewalTime = secretManager.renewToken(token, TEST_USER.toString()); Assert.assertTrue(renewalTime > 0); } + @Test + public void testReloadAndRenewToken() throws Exception { + testRenewTokenSuccessHelper(true); + } + + @Test + public void testRenewTokenSuccess() throws Exception { + testRenewTokenSuccessHelper(false); + } + /** * Tests failure for mismatch in renewer. */ @@ -375,6 +397,7 @@ private void validateHash(byte[] hash, byte[] identifier) throws Exception { createSecretManager(OzoneConfiguration config, long tokenMaxLife, long expiry, long tokenRemoverScanTime) throws IOException { return new OzoneDelegationTokenSecretManager(config, tokenMaxLife, - expiry, tokenRemoverScanTime, serviceRpcAdd, s3SecretManager); + expiry, tokenRemoverScanTime, serviceRpcAdd, s3SecretManager, + certificateClient); } } \ No newline at end of file diff --git a/hadoop-ozone/ozonefs-lib-current/pom.xml b/hadoop-ozone/ozonefs-lib-current/pom.xml index 5953acb62ef73..1645ccc82b240 100644 --- a/hadoop-ozone/ozonefs-lib-current/pom.xml +++ b/hadoop-ozone/ozonefs-lib-current/pom.xml @@ -85,6 +85,9 @@ + + ozone-default-generated.xml + diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java index 5e1a50d31c232..9ea03b545f306 100644 --- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java +++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java @@ -91,9 +91,11 @@ private static OzoneConfiguration createConf() { ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); Thread.currentThread().setContextClassLoader(null); - OzoneConfiguration conf = new OzoneConfiguration(); - Thread.currentThread().setContextClassLoader(contextClassLoader); - return conf; + try { + return new OzoneConfiguration(); + } finally { + Thread.currentThread().setContextClassLoader(contextClassLoader); + } } public BasicOzoneClientAdapterImpl(OzoneConfiguration conf, String volumeStr, @@ -109,38 +111,39 @@ public BasicOzoneClientAdapterImpl(String omHost, int omPort, ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); Thread.currentThread().setContextClassLoader(null); - OzoneConfiguration conf = OzoneConfiguration.of(hadoopConf); - if (omHost == null && OmUtils.isServiceIdsDefined(conf)) { - // When the host name or service id isn't given - // but ozone.om.service.ids is defined, declare failure. + try { + OzoneConfiguration conf = OzoneConfiguration.of(hadoopConf); - // This is a safety precaution that prevents the client from - // accidentally failing over to an unintended OM. - throw new IllegalArgumentException("Service ID or host name must not" - + " be omitted when ozone.om.service.ids is defined."); - } + if (omHost == null && OmUtils.isServiceIdsDefined(conf)) { + // When the host name or service id isn't given + // but ozone.om.service.ids is defined, declare failure. - if (omPort != -1) { - // When the port number is specified, perform the following check - if (OmUtils.isOmHAServiceId(conf, omHost)) { - // If omHost is a service id, it shouldn't use a port - throw new IllegalArgumentException("Port " + omPort + - " specified in URI but host '" + omHost + "' is " - + "a logical (HA) OzoneManager and does not use port information."); + // This is a safety precaution that prevents the client from + // accidentally failing over to an unintended OM. + throw new IllegalArgumentException("Service ID or host name must not" + + " be omitted when ozone.om.service.ids is defined."); } - } else { - // When port number is not specified, read it from config - omPort = OmUtils.getOmRpcPort(conf); - } - SecurityConfig secConfig = new SecurityConfig(conf); + if (omPort != -1) { + // When the port number is specified, perform the following check + if (OmUtils.isOmHAServiceId(conf, omHost)) { + // If omHost is a service id, it shouldn't use a port + throw new IllegalArgumentException("Port " + omPort + + " specified in URI but host '" + omHost + "' is a " + + "logical (HA) OzoneManager and does not use port information."); + } + } else { + // When port number is not specified, read it from config + omPort = OmUtils.getOmRpcPort(conf); + } - if (secConfig.isSecurityEnabled()) { - this.securityEnabled = true; - } + SecurityConfig secConfig = new SecurityConfig(conf); + + if (secConfig.isSecurityEnabled()) { + this.securityEnabled = true; + } - try { String replicationTypeConf = conf.get(OzoneConfigKeys.OZONE_REPLICATION_TYPE, OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT); diff --git a/hadoop-ozone/pom.xml b/hadoop-ozone/pom.xml index 3a8edf7fa0285..825e65ccf937c 100644 --- a/hadoop-ozone/pom.xml +++ b/hadoop-ozone/pom.xml @@ -291,7 +291,7 @@ **/dependency-reduced-pom.xml **/node_modules/** **/yarn.lock - **/recon-web/build/** + **/ozone-recon-web/build/** src/main/license/** @@ -343,7 +343,7 @@ **/node_modules/* - **/recon-web/** + **/ozone-recon-web/** diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java index 95e6f9b00a344..2d29d3f940442 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java @@ -32,9 +32,11 @@ import java.io.InputStream; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.zip.GZIPOutputStream; import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.HddsConfigKeys; @@ -83,6 +85,65 @@ public File getReconDbDir(Configuration conf, String dirConfigKey) { return getOzoneMetaDirPath(conf); } + /** + * Given a source directory, create a tar.gz file from it. + * + * @param sourcePath the path to the directory to be archived. + * @return tar.gz file + * @throws IOException + */ + public static File createTarFile(Path sourcePath) throws IOException { + TarArchiveOutputStream tarOs = null; + try { + String sourceDir = sourcePath.toString(); + String fileName = sourceDir.concat(".tar.gz"); + FileOutputStream fileOutputStream = new FileOutputStream(fileName); + GZIPOutputStream gzipOutputStream = + new GZIPOutputStream(new BufferedOutputStream(fileOutputStream)); + tarOs = new TarArchiveOutputStream(gzipOutputStream); + File folder = new File(sourceDir); + File[] filesInDir = folder.listFiles(); + if (filesInDir != null) { + for (File file : filesInDir) { + addFilesToArchive(file.getName(), file, tarOs); + } + } + return new File(fileName); + } finally { + try { + org.apache.hadoop.io.IOUtils.closeStream(tarOs); + } catch (Exception e) { + LOG.error("Exception encountered when closing " + + "TAR file output stream: " + e); + } + } + } + + private static void addFilesToArchive(String source, File file, + TarArchiveOutputStream + tarFileOutputStream) + throws IOException { + tarFileOutputStream.putArchiveEntry(new TarArchiveEntry(file, source)); + if (file.isFile()) { + FileInputStream fileInputStream = new FileInputStream(file); + BufferedInputStream bufferedInputStream = + new BufferedInputStream(fileInputStream); + org.apache.commons.compress.utils.IOUtils.copy(bufferedInputStream, + tarFileOutputStream); + tarFileOutputStream.closeArchiveEntry(); + fileInputStream.close(); + } else if (file.isDirectory()) { + tarFileOutputStream.closeArchiveEntry(); + File[] filesInDir = file.listFiles(); + if (filesInDir != null) { + for (File cFile : filesInDir) { + addFilesToArchive(cFile.getAbsolutePath(), cFile, + tarFileOutputStream); + } + } + } + } + /** * Untar DB snapshot tar file to recon OM snapshot directory. * @@ -175,4 +236,39 @@ public InputStream makeHttpCall(CloseableHttpClient httpClient, } } + /** + * Load last known DB in Recon. + * @param reconDbDir + * @param fileNamePrefix + * @return + */ + public File getLastKnownDB(File reconDbDir, String fileNamePrefix) { + String lastKnownSnapshotFileName = null; + long lastKnonwnSnapshotTs = Long.MIN_VALUE; + if (reconDbDir != null) { + File[] snapshotFiles = reconDbDir.listFiles((dir, name) -> + name.startsWith(fileNamePrefix)); + if (snapshotFiles != null) { + for (File snapshotFile : snapshotFiles) { + String fileName = snapshotFile.getName(); + try { + String[] fileNameSplits = fileName.split("_"); + if (fileNameSplits.length <= 1) { + continue; + } + long snapshotTimestamp = Long.parseLong(fileNameSplits[1]); + if (lastKnonwnSnapshotTs < snapshotTimestamp) { + lastKnonwnSnapshotTs = snapshotTimestamp; + lastKnownSnapshotFileName = fileName; + } + } catch (NumberFormatException nfEx) { + LOG.warn("Unknown file found in Recon DB dir : {}", fileName); + } + } + } + } + return lastKnownSnapshotFileName == null ? null : + new File(reconDbDir.getPath(), lastKnownSnapshotFileName); + } + } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/recovery/ReconOmMetadataManagerImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/recovery/ReconOmMetadataManagerImpl.java index e554b25e0ba6d..3d55c999c9046 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/recovery/ReconOmMetadataManagerImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/recovery/ReconOmMetadataManagerImpl.java @@ -18,6 +18,9 @@ package org.apache.hadoop.ozone.recon.recovery; +import static org.apache.hadoop.ozone.recon.ReconConstants.RECON_OM_SNAPSHOT_DB; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SNAPSHOT_DB_DIR; + import java.io.File; import java.io.IOException; @@ -28,6 +31,7 @@ import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.hdds.utils.db.DBStoreBuilder; +import org.apache.hadoop.ozone.recon.ReconUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,17 +46,28 @@ public class ReconOmMetadataManagerImpl extends OmMetadataManagerImpl private static final Logger LOG = LoggerFactory.getLogger(ReconOmMetadataManagerImpl.class); - @Inject private OzoneConfiguration ozoneConfiguration; + private ReconUtils reconUtils; @Inject - public ReconOmMetadataManagerImpl(OzoneConfiguration configuration) { + public ReconOmMetadataManagerImpl(OzoneConfiguration configuration, + ReconUtils reconUtils) { + this.reconUtils = reconUtils; this.ozoneConfiguration = configuration; } @Override public void start(OzoneConfiguration configuration) throws IOException { LOG.info("Starting ReconOMMetadataManagerImpl"); + File reconDbDir = + reconUtils.getReconDbDir(configuration, OZONE_RECON_OM_SNAPSHOT_DB_DIR); + File lastKnownOMSnapshot = + reconUtils.getLastKnownDB(reconDbDir, RECON_OM_SNAPSHOT_DB); + if (lastKnownOMSnapshot != null) { + LOG.info("Last known snapshot for OM : {}", + lastKnownOMSnapshot.getAbsolutePath()); + initializeNewRdbStore(lastKnownOMSnapshot); + } } /** @@ -69,7 +84,7 @@ private void initializeNewRdbStore(File dbFile) throws IOException { addOMTablesAndCodecs(dbStoreBuilder); DBStore newStore = dbStoreBuilder.build(); setStore(newStore); - LOG.info("Created new OM DB snapshot at {}.", + LOG.info("Created OM DB snapshot at {}.", dbFile.getAbsolutePath()); } catch (IOException ioEx) { LOG.error("Unable to initialize Recon OM DB snapshot store.", diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ContainerDBServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ContainerDBServiceProviderImpl.java index 4e32e1a317ea0..85edb7e94d026 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ContainerDBServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ContainerDBServiceProviderImpl.java @@ -81,14 +81,7 @@ public class ContainerDBServiceProviderImpl public ContainerDBServiceProviderImpl(DBStore dbStore, Configuration sqlConfiguration) { globalStatsDao = new GlobalStatsDao(sqlConfiguration); - try { - this.containerKeyTable = dbStore.getTable(CONTAINER_KEY_TABLE, - ContainerKeyPrefix.class, Integer.class); - this.containerKeyCountTable = dbStore.getTable(CONTAINER_KEY_COUNT_TABLE, - Long.class, Long.class); - } catch (IOException e) { - LOG.error("Unable to create Container Key tables." + e); - } + initializeTables(dbStore); } /** @@ -107,8 +100,9 @@ public void initNewContainerDB(Map File oldDBLocation = containerDbStore.getDbLocation(); containerDbStore = ReconContainerDBProvider .getNewDBStore(configuration, reconUtils); - containerKeyTable = containerDbStore.getTable(CONTAINER_KEY_TABLE, - ContainerKeyPrefix.class, Integer.class); + LOG.info("Creating new Recon Container DB at {}", + containerDbStore.getDbLocation().getAbsolutePath()); + initializeTables(containerDbStore); if (oldDBLocation.exists()) { LOG.info("Cleaning up old Recon Container DB at {}.", @@ -127,6 +121,20 @@ public void initNewContainerDB(Map storeContainerCount(0L); } + /** + * Initialize the container DB tables. + * @param dbStore + */ + private void initializeTables(DBStore dbStore) { + try { + this.containerKeyTable = dbStore.getTable(CONTAINER_KEY_TABLE, + ContainerKeyPrefix.class, Integer.class); + this.containerKeyCountTable = dbStore.getTable(CONTAINER_KEY_COUNT_TABLE, + Long.class, Long.class); + } catch (IOException e) { + LOG.error("Unable to create Container Key tables." + e); + } + } /** * Concatenate the containerID and Key Prefix using a delimiter and store the * count into the container DB store. diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index a5e5e4df5e50d..789b30168c823 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -178,6 +178,11 @@ public OMMetadataManager getOMMetadataManagerInstance() { @Override public void start() { + try { + omMetadataManager.start(configuration); + } catch (IOException ioEx) { + LOG.error("Error staring Recon OM Metadata Manager.", ioEx); + } long initialDelay = configuration.getTimeDuration( RECON_OM_SNAPSHOT_TASK_INITIAL_DELAY, RECON_OM_SNAPSHOT_TASK_INITIAL_DELAY_DEFAULT, @@ -235,20 +240,24 @@ DBCheckpoint getOzoneManagerDBSnapshot() { * @throws IOException */ @VisibleForTesting - void updateReconOmDBWithNewSnapshot() throws IOException { + boolean updateReconOmDBWithNewSnapshot() throws IOException { // Obtain the current DB snapshot from OM and // update the in house OM metadata managed DB instance. DBCheckpoint dbSnapshot = getOzoneManagerDBSnapshot(); if (dbSnapshot != null && dbSnapshot.getCheckpointLocation() != null) { + LOG.info("Got new checkpoint from OM : " + + dbSnapshot.getCheckpointLocation()); try { omMetadataManager.updateOmDB(dbSnapshot.getCheckpointLocation() .toFile()); + return true; } catch (IOException e) { LOG.error("Unable to refresh Recon OM DB Snapshot. ", e); } } else { LOG.error("Null snapshot location got from OM."); } + return false; } /** @@ -287,6 +296,7 @@ void getAndApplyDeltaUpdatesFromOM( */ @VisibleForTesting void syncDataFromOM() { + LOG.info("Syncing data from Ozone Manager."); long currentSequenceNumber = getCurrentOMDBSequenceNumber(); boolean fullSnapshot = false; @@ -296,6 +306,7 @@ void syncDataFromOM() { OMDBUpdatesHandler omdbUpdatesHandler = new OMDBUpdatesHandler(omMetadataManager); try { + LOG.info("Obtaining delta updates from Ozone Manager"); // Get updates from OM and apply to local Recon OM DB. getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); @@ -315,16 +326,20 @@ void syncDataFromOM() { if (fullSnapshot) { try { + LOG.info("Obtaining full snapshot from Ozone Manager"); // Update local Recon OM DB to new snapshot. - updateReconOmDBWithNewSnapshot(); + boolean success = updateReconOmDBWithNewSnapshot(); // Update timestamp of successful delta updates query. - ReconTaskStatus reconTaskStatusRecord = - new ReconTaskStatus( - OmSnapshotTaskName.OM_DB_FULL_SNAPSHOT.name(), - System.currentTimeMillis(), getCurrentOMDBSequenceNumber()); - reconTaskStatusDao.update(reconTaskStatusRecord); - // Reinitialize tasks that are listening. - reconTaskController.reInitializeTasks(omMetadataManager); + if (success) { + ReconTaskStatus reconTaskStatusRecord = + new ReconTaskStatus( + OmSnapshotTaskName.OM_DB_FULL_SNAPSHOT.name(), + System.currentTimeMillis(), getCurrentOMDBSequenceNumber()); + reconTaskStatusDao.update(reconTaskStatusRecord); + // Reinitialize tasks that are listening. + LOG.info("Calling reprocess on Recon tasks."); + reconTaskController.reInitializeTasks(omMetadataManager); + } } catch (IOException | InterruptedException e) { LOG.error("Unable to update Recon's OM DB with new snapshot ", e); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ReconContainerDBProvider.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ReconContainerDBProvider.java index f2de1297e3306..9c3e987f67298 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ReconContainerDBProvider.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/ReconContainerDBProvider.java @@ -23,6 +23,7 @@ import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_KEY_TABLE; import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR; +import java.io.File; import java.nio.file.Path; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -49,15 +50,29 @@ public class ReconContainerDBProvider implements Provider { private static final Logger LOG = LoggerFactory.getLogger(ReconContainerDBProvider.class); - @Inject private OzoneConfiguration configuration; + private ReconUtils reconUtils; @Inject - private ReconUtils reconUtils; + public ReconContainerDBProvider(OzoneConfiguration configuration, + ReconUtils reconUtils) { + this.configuration = configuration; + this.reconUtils = reconUtils; + } @Override public DBStore get() { - DBStore dbStore = getNewDBStore(configuration, reconUtils); + DBStore dbStore; + File reconDbDir = + reconUtils.getReconDbDir(configuration, OZONE_RECON_DB_DIR); + File lastKnownOMSnapshot = + reconUtils.getLastKnownDB(reconDbDir, RECON_CONTAINER_DB); + if (lastKnownOMSnapshot != null) { + dbStore = getDBStore(configuration, reconUtils, + lastKnownOMSnapshot.getName()); + } else { + dbStore = getNewDBStore(configuration, reconUtils); + } if (dbStore == null) { throw new ProvisionException("Unable to provide instance of DBStore " + "store."); @@ -65,10 +80,9 @@ public DBStore get() { return dbStore; } - public static DBStore getNewDBStore(OzoneConfiguration configuration, - ReconUtils reconUtils) { + private static DBStore getDBStore(OzoneConfiguration configuration, + ReconUtils reconUtils, String dbName) { DBStore dbStore = null; - String dbName = RECON_CONTAINER_DB + "_" + System.currentTimeMillis(); try { Path metaDir = reconUtils.getReconDbDir( configuration, OZONE_RECON_DB_DIR).toPath(); @@ -86,4 +100,10 @@ public static DBStore getNewDBStore(OzoneConfiguration configuration, } return dbStore; } + + static DBStore getNewDBStore(OzoneConfiguration configuration, + ReconUtils reconUtils) { + String dbName = RECON_CONTAINER_DB + "_" + System.currentTimeMillis(); + return getDBStore(configuration, reconUtils, dbName); + } } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/AbstractOMMetadataManagerTest.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/AbstractOMMetadataManagerTest.java index 649ec02ed3d45..ff3765af12035 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/AbstractOMMetadataManagerTest.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/AbstractOMMetadataManagerTest.java @@ -120,7 +120,7 @@ protected ReconOMMetadataManager getTestMetadataManager( .getAbsolutePath()); ReconOMMetadataManager reconOMMetaMgr = - new ReconOmMetadataManagerImpl(configuration); + new ReconOmMetadataManagerImpl(configuration, new ReconUtils()); reconOMMetaMgr.start(configuration); reconOMMetaMgr.updateOmDB( diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconUtils.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconUtils.java index ad048374218c7..6bb8993decb98 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconUtils.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconUtils.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.recon; +import static org.apache.hadoop.ozone.recon.ReconUtils.createTarFile; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -27,15 +28,16 @@ import java.io.BufferedWriter; import java.io.File; import java.io.FileInputStream; +import java.io.FileOutputStream; import java.io.FileWriter; import java.io.IOException; import java.io.InputStream; import java.nio.charset.Charset; import java.nio.file.Paths; +import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.ozone.OmUtils; import org.apache.http.HttpEntity; import org.apache.http.StatusLine; import org.apache.http.client.methods.CloseableHttpResponse; @@ -66,6 +68,44 @@ public void testGetReconDbDir() throws Exception { Assert.assertEquals(filePath, file.getAbsolutePath()); } + @Test + public void testCreateTarFile() throws Exception { + + File tempSnapshotDir = null; + FileInputStream fis = null; + FileOutputStream fos = null; + File tarFile = null; + + try { + String testDirName = System.getProperty("java.io.tmpdir"); + if (!testDirName.endsWith("/")) { + testDirName += "/"; + } + testDirName += "TestCreateTarFile_Dir" + System.currentTimeMillis(); + tempSnapshotDir = new File(testDirName); + tempSnapshotDir.mkdirs(); + + File file = new File(testDirName + "/temp1.txt"); + FileWriter writer = new FileWriter(file); + writer.write("Test data 1"); + writer.close(); + + file = new File(testDirName + "/temp2.txt"); + writer = new FileWriter(file); + writer.write("Test data 2"); + writer.close(); + + tarFile = createTarFile(Paths.get(testDirName)); + Assert.assertNotNull(tarFile); + + } finally { + org.apache.hadoop.io.IOUtils.closeStream(fis); + org.apache.hadoop.io.IOUtils.closeStream(fos); + FileUtils.deleteDirectory(tempSnapshotDir); + FileUtils.deleteQuietly(tarFile); + } + } + @Test public void testUntarCheckpointFile() throws Exception { @@ -87,7 +127,7 @@ public void testUntarCheckpointFile() throws Exception { writer.close(); //Create test tar file. - File tarFile = OmUtils.createTarFile(newDir.toPath()); + File tarFile = createTarFile(newDir.toPath()); File outputDir = folder.newFolder(); new ReconUtils().untarCheckpointFile(tarFile, outputDir.toPath()); @@ -133,4 +173,35 @@ public int read() throws IOException { assertEquals("File 1 Contents", contents); } + @Test + public void testGetLastKnownDB() throws IOException { + File newDir = folder.newFolder(); + + File file1 = Paths.get(newDir.getAbsolutePath(), "valid_1") + .toFile(); + String str = "File1 Contents"; + BufferedWriter writer = new BufferedWriter(new FileWriter( + file1.getAbsolutePath())); + writer.write(str); + writer.close(); + + File file2 = Paths.get(newDir.getAbsolutePath(), "valid_2") + .toFile(); + str = "File2 Contents"; + writer = new BufferedWriter(new FileWriter(file2.getAbsolutePath())); + writer.write(str); + writer.close(); + + + File file3 = Paths.get(newDir.getAbsolutePath(), "invalid_3") + .toFile(); + str = "File3 Contents"; + writer = new BufferedWriter(new FileWriter(file3.getAbsolutePath())); + writer.write(str); + writer.close(); + + ReconUtils reconUtils = new ReconUtils(); + File latestValidFile = reconUtils.getLastKnownDB(newDir, "valid"); + assertTrue(latestValidFile.getName().equals("valid_2")); + } } \ No newline at end of file diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/recovery/TestReconOmMetadataManagerImpl.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/recovery/TestReconOmMetadataManagerImpl.java index 64fb8d8dc5316..a9e6aea42c7bf 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/recovery/TestReconOmMetadataManagerImpl.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/recovery/TestReconOmMetadataManagerImpl.java @@ -22,7 +22,9 @@ import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SNAPSHOT_DB_DIR; import java.io.File; +import java.io.IOException; +import org.apache.commons.io.FileUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -31,6 +33,7 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; import org.apache.hadoop.hdds.utils.db.DBCheckpoint; +import org.apache.hadoop.ozone.recon.ReconUtils; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -44,9 +47,95 @@ public class TestReconOmMetadataManagerImpl { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Test + public void testStart() throws Exception { + + OMMetadataManager omMetadataManager = getOMMetadataManager(); + + //Take checkpoint of the above OM DB. + DBCheckpoint checkpoint = omMetadataManager.getStore() + .getCheckpoint(true); + File snapshotFile = new File( + checkpoint.getCheckpointLocation().getParent() + "/" + + "om.snapshot.db_" + System.currentTimeMillis()); + checkpoint.getCheckpointLocation().toFile().renameTo(snapshotFile); + + //Create new Recon OM Metadata manager instance. + File reconOmDbDir = temporaryFolder.newFolder(); + OzoneConfiguration configuration = new OzoneConfiguration(); + configuration.set(OZONE_RECON_OM_SNAPSHOT_DB_DIR, reconOmDbDir + .getAbsolutePath()); + FileUtils.copyDirectory(snapshotFile.getParentFile(), reconOmDbDir); + + ReconOMMetadataManager reconOMMetadataManager = + new ReconOmMetadataManagerImpl(configuration, new ReconUtils()); + reconOMMetadataManager.start(configuration); + + Assert.assertNotNull(reconOMMetadataManager.getBucketTable()); + Assert.assertNotNull(reconOMMetadataManager.getVolumeTable() + .get("/sampleVol")); + Assert.assertNotNull(reconOMMetadataManager.getBucketTable() + .get("/sampleVol/bucketOne")); + Assert.assertNotNull(reconOMMetadataManager.getKeyTable() + .get("/sampleVol/bucketOne/key_one")); + Assert.assertNotNull(reconOMMetadataManager.getKeyTable() + .get("/sampleVol/bucketOne/key_two")); + } + @Test public void testUpdateOmDB() throws Exception { + OMMetadataManager omMetadataManager = getOMMetadataManager(); + //Make sure OM Metadata reflects the keys that were inserted. + Assert.assertNotNull(omMetadataManager.getKeyTable() + .get("/sampleVol/bucketOne/key_one")); + Assert.assertNotNull(omMetadataManager.getKeyTable() + .get("/sampleVol/bucketOne/key_two")); + + //Take checkpoint of OM DB. + DBCheckpoint checkpoint = omMetadataManager.getStore() + .getCheckpoint(true); + Assert.assertNotNull(checkpoint.getCheckpointLocation()); + + //Create new Recon OM Metadata manager instance. + File reconOmDbDir = temporaryFolder.newFolder(); + OzoneConfiguration configuration = new OzoneConfiguration(); + configuration.set(OZONE_RECON_OM_SNAPSHOT_DB_DIR, reconOmDbDir + .getAbsolutePath()); + ReconOMMetadataManager reconOMMetadataManager = + new ReconOmMetadataManagerImpl(configuration, new ReconUtils()); + reconOMMetadataManager.start(configuration); + + //Before accepting a snapshot, the metadata should have null tables. + Assert.assertNull(reconOMMetadataManager.getBucketTable()); + + //Update Recon OM DB with the OM DB checkpoint location. + reconOMMetadataManager.updateOmDB( + checkpoint.getCheckpointLocation().toFile()); + + //Now, the tables should have been initialized. + Assert.assertNotNull(reconOMMetadataManager.getBucketTable()); + + // Check volume and bucket entries. + Assert.assertNotNull(reconOMMetadataManager.getVolumeTable() + .get("/sampleVol")); + Assert.assertNotNull(reconOMMetadataManager.getBucketTable() + .get("/sampleVol/bucketOne")); + + //Verify Keys inserted in OM DB are available in Recon OM DB. + Assert.assertNotNull(reconOMMetadataManager.getKeyTable() + .get("/sampleVol/bucketOne/key_one")); + Assert.assertNotNull(reconOMMetadataManager.getKeyTable() + .get("/sampleVol/bucketOne/key_two")); + + } + + /** + * Get test OM metadata manager. + * @return OMMetadataManager instance + * @throws IOException + */ + private OMMetadataManager getOMMetadataManager() throws IOException { //Create a new OM Metadata Manager instance + DB. File omDbDir = temporaryFolder.newFolder(); OzoneConfiguration omConfiguration = new OzoneConfiguration(); @@ -93,48 +182,6 @@ public void testUpdateOmDB() throws Exception { .setReplicationType(HddsProtos.ReplicationType.STAND_ALONE) .build()); - //Make sure OM Metadata reflects the keys that were inserted. - Assert.assertNotNull(omMetadataManager.getKeyTable() - .get("/sampleVol/bucketOne/key_one")); - Assert.assertNotNull(omMetadataManager.getKeyTable() - .get("/sampleVol/bucketOne/key_two")); - - //Take checkpoint of OM DB. - DBCheckpoint checkpoint = omMetadataManager.getStore() - .getCheckpoint(true); - Assert.assertNotNull(checkpoint.getCheckpointLocation()); - - //Create new Recon OM Metadata manager instance. - File reconOmDbDir = temporaryFolder.newFolder(); - OzoneConfiguration configuration = new OzoneConfiguration(); - configuration.set(OZONE_RECON_OM_SNAPSHOT_DB_DIR, reconOmDbDir - .getAbsolutePath()); - ReconOMMetadataManager reconOMMetadataManager = - new ReconOmMetadataManagerImpl(configuration); - reconOMMetadataManager.start(configuration); - - //Before accepting a snapshot, the metadata should have null tables. - Assert.assertNull(reconOMMetadataManager.getBucketTable()); - - //Update Recon OM DB with the OM DB checkpoint location. - reconOMMetadataManager.updateOmDB( - checkpoint.getCheckpointLocation().toFile()); - - //Now, the tables should have been initialized. - Assert.assertNotNull(reconOMMetadataManager.getBucketTable()); - - // Check volume and bucket entries. - Assert.assertNotNull(reconOMMetadataManager.getVolumeTable() - .get(volumeKey)); - Assert.assertNotNull(reconOMMetadataManager.getBucketTable() - .get(bucketKey)); - - //Verify Keys inserted in OM DB are available in Recon OM DB. - Assert.assertNotNull(reconOMMetadataManager.getKeyTable() - .get("/sampleVol/bucketOne/key_one")); - Assert.assertNotNull(reconOMMetadataManager.getKeyTable() - .get("/sampleVol/bucketOne/key_two")); - + return omMetadataManager; } - } \ No newline at end of file diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java index f38cbc69b6bf5..63b414071d77a 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR; import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SNAPSHOT_DB_DIR; +import static org.apache.hadoop.ozone.recon.ReconUtils.createTarFile; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -41,7 +42,6 @@ import java.nio.file.Paths; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; @@ -96,7 +96,7 @@ public void testUpdateReconOmDBWithNewSnapshot() throws Exception { DBCheckpoint checkpoint = omMetadataManager.getStore() .getCheckpoint(true); - File tarFile = OmUtils.createTarFile(checkpoint.getCheckpointLocation()); + File tarFile = createTarFile(checkpoint.getCheckpointLocation()); InputStream inputStream = new FileInputStream(tarFile); ReconUtils reconUtilsMock = getMockReconUtils(); when(reconUtilsMock.makeHttpCall(any(), anyString())) @@ -114,7 +114,7 @@ public void testUpdateReconOmDBWithNewSnapshot() throws Exception { Assert.assertNull(reconOMMetadataManager.getKeyTable() .get("/sampleVol/bucketOne/key_two")); - ozoneManagerServiceProvider.updateReconOmDBWithNewSnapshot(); + assertTrue(ozoneManagerServiceProvider.updateReconOmDBWithNewSnapshot()); assertNotNull(reconOMMetadataManager.getKeyTable() .get("/sampleVol/bucketOne/key_one")); @@ -147,7 +147,7 @@ public void testGetOzoneManagerDBSnapshot() throws Exception { writer.close(); //Create test tar file. - File tarFile = OmUtils.createTarFile(checkpointDir.toPath()); + File tarFile = createTarFile(checkpointDir.toPath()); InputStream fileInputStream = new FileInputStream(tarFile); ReconUtils reconUtilsMock = getMockReconUtils(); when(reconUtilsMock.makeHttpCall(any(), anyString())) @@ -241,10 +241,10 @@ public void testSyncDataFromOMFullSnapshot() throws Exception { .reInitializeTasks(omMetadataManager); OzoneManagerServiceProviderImpl ozoneManagerServiceProvider = - new OzoneManagerServiceProviderImpl(configuration, omMetadataManager, + new MockOzoneServiceProvider(configuration, omMetadataManager, reconTaskControllerMock, new ReconUtils(), ozoneManagerProtocol); - //Should trigger full snapshot request. + // Should trigger full snapshot request. ozoneManagerServiceProvider.syncDataFromOM(); ArgumentCaptor captor = @@ -313,5 +313,26 @@ private OzoneManagerProtocol getMockOzoneManagerClient( .DBUpdatesRequest.class))).thenReturn(dbUpdatesWrapper); return ozoneManagerProtocolMock; } +} + +/** + * Mock OzoneManagerServiceProviderImpl which overrides + * updateReconOmDBWithNewSnapshot. + */ +class MockOzoneServiceProvider extends OzoneManagerServiceProviderImpl { + + MockOzoneServiceProvider(OzoneConfiguration configuration, + ReconOMMetadataManager omMetadataManager, + ReconTaskController reconTaskController, + ReconUtils reconUtils, + OzoneManagerProtocol ozoneManagerClient) + throws IOException { + super(configuration, omMetadataManager, reconTaskController, reconUtils, + ozoneManagerClient); + } -} \ No newline at end of file + @Override + public boolean updateReconOmDBWithNewSnapshot() { + return true; + } +} diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestReconContainerDBProvider.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestReconContainerDBProvider.java index cb4aa7d161786..ad1feeb68a328 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestReconContainerDBProvider.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestReconContainerDBProvider.java @@ -20,8 +20,6 @@ import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DB_DIR; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import java.io.File; import java.io.IOException; @@ -36,7 +34,6 @@ import com.google.inject.AbstractModule; import com.google.inject.Guice; import com.google.inject.Injector; -import com.google.inject.ProvisionException; import com.google.inject.Singleton; /** @@ -68,20 +65,10 @@ protected void configure() { @Test public void testGet() throws Exception { - ReconContainerDBProvider reconContainerDBProvider = injector.getInstance( ReconContainerDBProvider.class); DBStore dbStore = reconContainerDBProvider.get(); assertNotNull(dbStore); - - ReconContainerDBProvider reconContainerDBProviderNew = new - ReconContainerDBProvider(); - try { - reconContainerDBProviderNew.get(); - fail(); - } catch (Exception e) { - assertTrue(e instanceof ProvisionException); - } } } \ No newline at end of file diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java index d9afaf162a457..abaca03908240 100644 --- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java +++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java @@ -62,6 +62,9 @@ public class OzoneClientProducer { @Inject private Text omService; + @Inject + private String omServiceID; + @Produces public OzoneClient createClient() throws IOException { @@ -105,7 +108,13 @@ private OzoneClient getClient(OzoneConfiguration config) throws IOException { } catch (Exception e) { LOG.error("Error: ", e); } - return OzoneClientFactory.getClient(ozoneConfiguration); + + if (omServiceID == null) { + return OzoneClientFactory.getClient(ozoneConfiguration); + } else { + // As in HA case, we need to pass om service ID. + return OzoneClientFactory.getRpcClient(omServiceID, ozoneConfiguration); + } } @VisibleForTesting diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneServiceProvider.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneServiceProvider.java index f4342f62f4820..b98426c20149e 100644 --- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneServiceProvider.java +++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneServiceProvider.java @@ -20,33 +20,75 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.io.Text; import org.apache.hadoop.ozone.OmUtils; +import org.apache.hadoop.ozone.s3.util.OzoneS3Util; import org.apache.hadoop.security.SecurityUtil; import javax.annotation.PostConstruct; import javax.enterprise.context.ApplicationScoped; import javax.enterprise.inject.Produces; import javax.inject.Inject; + +import java.util.Arrays; +import java.util.Collection; + +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_NODES_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY; + /** * This class creates the OM service . */ @ApplicationScoped public class OzoneServiceProvider { - private Text omServiceAdd; + private Text omServiceAddr; + + private String omserviceID; @Inject private OzoneConfiguration conf; @PostConstruct public void init() { - omServiceAdd = SecurityUtil.buildTokenService(OmUtils. - getOmAddressForClients(conf)); + Collection serviceIdList = + conf.getTrimmedStringCollection(OZONE_OM_SERVICE_IDS_KEY); + if (serviceIdList.size() == 0) { + // Non-HA cluster + omServiceAddr = SecurityUtil.buildTokenService(OmUtils. + getOmAddressForClients(conf)); + } else { + // HA cluster. + //For now if multiple service id's are configured we throw exception. + // As if multiple service id's are configured, S3Gateway will not be + // knowing which one to talk to. In future, if OM federation is supported + // we can resolve this by having another property like + // ozone.om.internal.service.id. + // TODO: Revisit this later. + if (serviceIdList.size() > 1) { + throw new IllegalArgumentException("Multiple serviceIds are " + + "configured. " + Arrays.toString(serviceIdList.toArray())); + } else { + String serviceId = serviceIdList.iterator().next(); + Collection omNodeIds = OmUtils.getOMNodeIds(conf, serviceId); + if (omNodeIds.size() == 0) { + throw new IllegalArgumentException(OZONE_OM_NODES_KEY + + "." + serviceId + " is not defined"); + } + omServiceAddr = new Text(OzoneS3Util.buildServiceNameForToken(conf, + serviceId, omNodeIds)); + omserviceID = serviceId; + } + } } @Produces public Text getService() { - return omServiceAdd; + return omServiceAddr; + } + + @Produces + public String getOmServiceID() { + return omserviceID; } } diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/OzoneS3Util.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/OzoneS3Util.java index 129ea2d293d5e..ce7d4f2876ab8 100644 --- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/OzoneS3Util.java +++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/OzoneS3Util.java @@ -19,8 +19,17 @@ package org.apache.hadoop.ozone.s3.util; import org.apache.commons.codec.digest.DigestUtils; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.ozone.OmUtils; +import org.apache.hadoop.security.SecurityUtil; + +import javax.annotation.Nonnull; +import java.util.Collection; import java.util.Objects; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; + /** * Ozone util for S3 related operations. */ @@ -33,4 +42,39 @@ public static String getVolumeName(String userName) { Objects.requireNonNull(userName); return DigestUtils.md5Hex(userName); } + + /** + * Generate service Name for token. + * @param configuration + * @param serviceId - ozone manager service ID + * @param omNodeIds - list of node ids for the given OM service. + * @return service Name. + */ + public static String buildServiceNameForToken( + @Nonnull OzoneConfiguration configuration, @Nonnull String serviceId, + @Nonnull Collection omNodeIds) { + StringBuilder rpcAddress = new StringBuilder(); + + int nodesLength = omNodeIds.size(); + int counter = 0; + for (String nodeId : omNodeIds) { + counter++; + String rpcAddrKey = OmUtils.addKeySuffixes(OZONE_OM_ADDRESS_KEY, + serviceId, nodeId); + String rpcAddrStr = OmUtils.getOmRpcAddress(configuration, rpcAddrKey); + if (rpcAddrStr == null || rpcAddrStr.isEmpty()) { + throw new IllegalArgumentException("Could not find rpcAddress for " + + OZONE_OM_ADDRESS_KEY + "." + serviceId + "." + nodeId); + } + + if (counter != nodesLength) { + rpcAddress.append(SecurityUtil.buildTokenService( + NetUtils.createSocketAddr(rpcAddrStr)) + ","); + } else { + rpcAddress.append(SecurityUtil.buildTokenService( + NetUtils.createSocketAddr(rpcAddrStr))); + } + } + return rpcAddress.toString(); + } } diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestOzoneS3Util.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestOzoneS3Util.java new file mode 100644 index 0000000000000..8892a9784dbf6 --- /dev/null +++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestOzoneS3Util.java @@ -0,0 +1,130 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.s3.util; + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.OmUtils; +import org.apache.hadoop.ozone.om.OMConfigKeys; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collection; + +import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_TOKEN_SERVICE_USE_IP; +import static org.junit.Assert.fail; + +/** + * Class used to test OzoneS3Util. + */ +public class TestOzoneS3Util { + + + private OzoneConfiguration configuration; + private String serviceID = "omService"; + + @Before + public void setConf() { + configuration = new OzoneConfiguration(); + + String nodeIDs = "om1,om2,om3"; + configuration.set(OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY, serviceID); + configuration.set(OMConfigKeys.OZONE_OM_NODES_KEY + "." + serviceID, + nodeIDs); + configuration.setBoolean(HADOOP_SECURITY_TOKEN_SERVICE_USE_IP, false); + } + + @Test + public void testBuildServiceNameForToken() { + + Collection nodeIDList = OmUtils.getOMNodeIds(configuration, + serviceID); + + configuration.set(OmUtils.addKeySuffixes(OMConfigKeys.OZONE_OM_ADDRESS_KEY, + serviceID, "om1"), "om1:9862"); + configuration.set(OmUtils.addKeySuffixes(OMConfigKeys.OZONE_OM_ADDRESS_KEY, + serviceID, "om2"), "om2:9862"); + configuration.set(OmUtils.addKeySuffixes(OMConfigKeys.OZONE_OM_ADDRESS_KEY, + serviceID, "om3"), "om3:9862"); + + String expectedOmServiceAddress = buildServiceAddress(nodeIDList); + + SecurityUtil.setConfiguration(configuration); + String omserviceAddr = OzoneS3Util.buildServiceNameForToken(configuration, + serviceID, nodeIDList); + + Assert.assertEquals(expectedOmServiceAddress, omserviceAddr); + } + + + @Test + public void testBuildServiceNameForTokenIncorrectConfig() { + + Collection nodeIDList = OmUtils.getOMNodeIds(configuration, + serviceID); + + // Don't set om3 node rpc address. Here we are skipping setting of one of + // the OM address. So buildServiceNameForToken will fail. + configuration.set(OmUtils.addKeySuffixes(OMConfigKeys.OZONE_OM_ADDRESS_KEY, + serviceID, "om1"), "om1:9862"); + configuration.set(OmUtils.addKeySuffixes(OMConfigKeys.OZONE_OM_ADDRESS_KEY, + serviceID, "om2"), "om2:9862"); + + + SecurityUtil.setConfiguration(configuration); + + try { + OzoneS3Util.buildServiceNameForToken(configuration, + serviceID, nodeIDList); + fail("testBuildServiceNameForTokenIncorrectConfig failed"); + } catch (IllegalArgumentException ex) { + GenericTestUtils.assertExceptionContains("Could not find rpcAddress " + + "for", ex); + } + + + } + + /** + * Build serviceName from list of node ids. + * @param nodeIDList + * @return service name for token. + */ + private String buildServiceAddress(Collection nodeIDList) { + StringBuilder omServiceAddrBuilder = new StringBuilder(); + int nodesLength = nodeIDList.size(); + int counter = 0; + for (String nodeID : nodeIDList) { + counter++; + String addr = configuration.get(OmUtils.addKeySuffixes( + OMConfigKeys.OZONE_OM_ADDRESS_KEY, serviceID, nodeID)); + + if (counter != nodesLength) { + omServiceAddrBuilder.append(addr + ","); + } else { + omServiceAddrBuilder.append(addr); + } + } + + return omServiceAddrBuilder.toString(); + } + +} diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java index 7407f8b5b1372..cc316199afd00 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java @@ -34,7 +34,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserVolumeInfo; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.util.Tool; @@ -360,7 +360,7 @@ private void insertOMDB(Connection conn, KeyType type, String keyName, byte[] value) throws IOException, SQLException { switch (type) { case USER: - VolumeList volumeList = VolumeList.parseFrom(value); + UserVolumeInfo volumeList = UserVolumeInfo.parseFrom(value); for (String volumeName : volumeList.getVolumeNamesList()) { String insertVolumeList = String.format(INSERT_VOLUME_LIST, keyName, volumeName); diff --git a/hadoop-ozone/tools/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem b/hadoop-ozone/tools/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem new file mode 100644 index 0000000000000..03680027d539e --- /dev/null +++ b/hadoop-ozone/tools/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.hadoop.fs.ozone.OzoneFileSystem diff --git a/hadoop-tools/hadoop-archives/pom.xml b/hadoop-tools/hadoop-archives/pom.xml index a7fba81805166..47c1251046af5 100644 --- a/hadoop-tools/hadoop-archives/pom.xml +++ b/hadoop-tools/hadoop-archives/pom.xml @@ -92,6 +92,11 @@ test-jar test + + org.assertj + assertj-core + test + diff --git a/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java b/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java index c63bc42266b4f..b1755affa8833 100644 --- a/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java +++ b/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java @@ -49,6 +49,7 @@ import org.junit.Assert; import static org.junit.Assert.*; import static org.slf4j.LoggerFactory.getLogger; +import static org.assertj.core.api.Assertions.assertThat; import org.junit.Before; import org.junit.Test; @@ -400,7 +401,7 @@ public void testReadFileContent() throws Exception { readFileCount++; } } - assertEquals(fileList.size(), readFileCount); + assertThat(fileList.size()).isEqualTo(readFileCount); } finally { harFileSystem.close(); } diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index ff330e52dc01e..bd204b08a612f 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -188,8 +188,6 @@ **/ITestDynamoDBMetadataStoreScale.java **/ITestTerasort*.java - - **/ITest*CommitMRJob.java **/ITestS3GuardDDBRootOperations.java @@ -231,8 +229,6 @@ **/ITestTerasort*.java - - **/ITest*CommitMRJob.java **/ITestS3AContractRootDir.java **/ITestS3GuardDDBRootOperations.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 659e802a7c332..fdbdf37a26ab3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -352,26 +352,30 @@ private Constants() { /** * List of custom Signers. The signer class will be loaded, and the signer - * name will be associated with this signer class in the S3 SDK. e.g. Single - * CustomSigner {@literal ->} 'CustomSigner:org.apache...CustomSignerClass Multiple - * CustomSigners {@literal ->} 'CSigner1:CustomSignerClass1,CSigner2:CustomerSignerClass2 + * name will be associated with this signer class in the S3 SDK. + * Examples + * CustomSigner {@literal ->} 'CustomSigner:org.apache...CustomSignerClass' + * CustomSigners {@literal ->} 'CSigner1:CSigner1Class,CSigner2:CSigner2Class' + * Initializer {@literal ->} 'CSigner1:CSigner1Class:CSigner1InitializerClass' + * With Existing {@literal ->} 'AWS4Signer,CSigner1,CSigner2:CSigner2Class' */ public static final String CUSTOM_SIGNERS = "fs.s3a.custom.signers"; /** * There's 3 parameters that can be used to specify a non-default signing - * algorithm. fs.s3a.signing-algorithm - This property has existed for the - * longest time. If specified, without either of the other 2 properties being - * specified, this signing algorithm will be used for S3 and DDB (S3Guard). - * The other 2 properties override this value for S3 or DDB. + * algorithm.
+ * fs.s3a.signing-algorithm - This property has existed for the longest time. + * If specified, without either of the other 2 properties being specified, + * this signing algorithm will be used for S3 and DDB (S3Guard).
+ * The other 2 properties override this value for S3 or DDB.
* fs.s3a.s3.signing-algorithm - Allows overriding the S3 Signing algorithm. * This does not affect DDB. Specifying this property without specifying * fs.s3a.signing-algorithm will only update the signing algorithm for S3 - * requests, and the default will be used for DDB fs.s3a.ddb.signing-algorithm - * - Allows overriding the DDB Signing algorithm. This does not affect S3. - * Specifying this property without specifying fs.s3a.signing-algorithm will - * only update the signing algorithm for DDB requests, and the default will be - * used for S3 + * requests, and the default will be used for DDB.
+ * fs.s3a.ddb.signing-algorithm - Allows overriding the DDB Signing algorithm. + * This does not affect S3. Specifying this property without specifying + * fs.s3a.signing-algorithm will only update the signing algorithm for + * DDB requests, and the default will be used for S3. */ public static final String SIGNING_ALGORITHM = "fs.s3a.signing-algorithm"; @@ -833,4 +837,10 @@ private Constants() { public static final String AWS_SERVICE_IDENTIFIER_S3 = "S3"; public static final String AWS_SERVICE_IDENTIFIER_DDB = "DDB"; public static final String AWS_SERVICE_IDENTIFIER_STS = "STS"; + + /** + * How long to wait for the thread pool to terminate when cleaning up. + * Value: {@value} seconds. + */ + public static final int THREAD_POOL_SHUTDOWN_DELAY_SECONDS = 30; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 1a1d9b75e4f16..26f16a7b23271 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -95,6 +95,8 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Globber; +import org.apache.hadoop.fs.s3a.auth.SignerManager; +import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.CopyOutcome; @@ -108,6 +110,7 @@ import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.fs.FileAlreadyExistsException; @@ -151,6 +154,7 @@ import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.SemaphoredDelegatingExecutor; +import org.apache.hadoop.util.concurrent.HadoopExecutors; import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; @@ -182,7 +186,7 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class S3AFileSystem extends FileSystem implements StreamCapabilities, - AWSPolicyProvider { + AWSPolicyProvider, DelegationTokenProvider { /** * Default blocksize as used in blocksize and FS status queries. */ @@ -362,8 +366,8 @@ public void initialize(URI name, Configuration originalConf) } useListV1 = (listVersion == 1); - signerManager = new SignerManager(); - signerManager.initCustomSigners(conf); + signerManager = new SignerManager(bucket, this, conf, owner); + signerManager.initCustomSigners(); // creates the AWS client, including overriding auth chain if // the FS came with a DT @@ -1335,6 +1339,11 @@ private long innerRename(Path source, Path dest) return renameOperation.execute(); } + @Override public Token getFsDelegationToken() + throws IOException { + return getDelegationToken(null); + } + /** * The callbacks made by the rename and delete operations. * This separation allows the operation to be factored out and @@ -3054,6 +3063,12 @@ public void close() throws IOException { transfers.shutdownNow(true); transfers = null; } + HadoopExecutors.shutdown(boundedThreadPool, LOG, + THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); + boundedThreadPool = null; + HadoopExecutors.shutdown(unboundedThreadPool, LOG, + THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); + unboundedThreadPool = null; S3AUtils.closeAll(LOG, metadataStore, instrumentation); metadataStore = null; instrumentation = null; @@ -4056,7 +4071,7 @@ public List listMultipartUploads(String prefix) */ @Retries.OnceRaw void abortMultipartUpload(String destKey, String uploadId) { - LOG.info("Aborting multipart upload {} to {}", uploadId, destKey); + LOG.debug("Aborting multipart upload {} to {}", uploadId, destKey); getAmazonS3Client().abortMultipartUpload( new AbortMultipartUploadRequest(getBucket(), destKey, @@ -4076,7 +4091,7 @@ void abortMultipartUpload(MultipartUpload upload) { uploadId = upload.getUploadId(); if (LOG.isInfoEnabled()) { DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - LOG.info("Aborting multipart upload {} to {} initiated by {} on {}", + LOG.debug("Aborting multipart upload {} to {} initiated by {} on {}", uploadId, destKey, upload.getInitiator(), df.format(upload.getInitiated())); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index e9ed972c6a16e..15f73901cb873 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -611,11 +611,14 @@ public void getMetrics(MetricsCollector collector, boolean all) { public void close() { synchronized (metricsSystemLock) { + // it is critical to close each quantile, as they start a scheduled + // task in a shared thread pool. putLatencyQuantile.stop(); throttleRateQuantile.stop(); metricsSystem.unregisterSource(metricsSourceName); int activeSources = --metricsSourceActiveCounter; if (activeSources == 0) { + LOG.debug("Shutting down metrics publisher"); metricsSystem.publishMetricsNow(); metricsSystem.shutdown(); metricsSystem = null; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AwsSignerInitializer.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AwsSignerInitializer.java new file mode 100644 index 0000000000000..f02a3ed63d60e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AwsSignerInitializer.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.auth; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.security.UserGroupInformation; + +/** + * Interface which can be implemented to allow initialization of any custom + * signers which may be used by the {@link S3AFileSystem}. + */ +public interface AwsSignerInitializer { + + /** + * Register a store instance. + * + * @param bucketName the bucket name + * @param storeConf the store configuration + * @param dtProvider delegation token provider for the store + * @param storeUgi ugi under which the store is operating + */ + void registerStore(String bucketName, Configuration storeConf, + DelegationTokenProvider dtProvider, UserGroupInformation storeUgi); + + /** + * Unregister a store instance. + * + * @param bucketName the bucket name + * @param storeConf the store configuration + * @param dtProvider delegation token provider for the store + * @param storeUgi ugi under which the store is operating + */ + void unregisterStore(String bucketName, Configuration storeConf, + DelegationTokenProvider dtProvider, UserGroupInformation storeUgi); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SignerManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java similarity index 50% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SignerManager.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java index 5ca1482b8437c..cda769a789c30 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SignerManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java @@ -15,16 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.fs.s3a; +package org.apache.hadoop.fs.s3a.auth; -import com.amazonaws.auth.Signer; -import com.amazonaws.auth.SignerFactory; import java.io.Closeable; import java.io.IOException; +import java.util.LinkedList; +import java.util.List; + +import com.amazonaws.auth.Signer; +import com.amazonaws.auth.SignerFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.ReflectionUtils; import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS; @@ -36,17 +42,28 @@ public class SignerManager implements Closeable { private static final Logger LOG = LoggerFactory .getLogger(SignerManager.class); + private final List initializers = new LinkedList<>(); - public SignerManager() { + private final String bucketName; + private final DelegationTokenProvider delegationTokenProvider; + private final Configuration ownerConf; + private final UserGroupInformation ownerUgi; + + public SignerManager(String bucketName, + DelegationTokenProvider delegationTokenProvider, Configuration ownerConf, + UserGroupInformation ownerUgi) { + this.bucketName = bucketName; + this.delegationTokenProvider = delegationTokenProvider; + this.ownerConf = ownerConf; + this.ownerUgi = ownerUgi; } /** * Initialize custom signers and register them with the AWS SDK. * - * @param conf Hadoop configuration */ - public void initCustomSigners(Configuration conf) { - String[] customSigners = conf.getTrimmedStrings(CUSTOM_SIGNERS); + public void initCustomSigners() { + String[] customSigners = ownerConf.getTrimmedStrings(CUSTOM_SIGNERS); if (customSigners == null || customSigners.length == 0) { // No custom signers specified, nothing to do. LOG.debug("No custom signers specified"); @@ -55,15 +72,41 @@ public void initCustomSigners(Configuration conf) { for (String customSigner : customSigners) { String[] parts = customSigner.split(":"); - if (parts.length != 2) { - String message = - "Invalid format (Expected name:SignerClass) for CustomSigner: [" - + customSigner - + "]"; + if (!(parts.length == 1 || parts.length == 2 || parts.length == 3)) { + String message = "Invalid format (Expected name, name:SignerClass," + + " name:SignerClass:SignerInitializerClass)" + + " for CustomSigner: [" + customSigner + "]"; LOG.error(message); throw new IllegalArgumentException(message); } - maybeRegisterSigner(parts[0], parts[1], conf); + if (parts.length == 1) { + // Nothing to do. Trying to use a pre-defined Signer + } else { + // Register any custom Signer + maybeRegisterSigner(parts[0], parts[1], ownerConf); + + // If an initializer is specified, take care of instantiating it and + // setting it up + if (parts.length == 3) { + Class clazz = null; + try { + clazz = (Class) ownerConf + .getClassByName(parts[2]); + } catch (ClassNotFoundException e) { + throw new RuntimeException(String.format( + "SignerInitializer class" + " [%s] not found for signer [%s]", + parts[2], parts[0]), e); + } + LOG.debug("Creating signer initializer: [{}] for signer: [{}]", + parts[2], parts[0]); + AwsSignerInitializer signerInitializer = ReflectionUtils + .newInstance(clazz, null); + initializers.add(signerInitializer); + signerInitializer + .registerStore(bucketName, ownerConf, delegationTokenProvider, + ownerUgi); + } + } } } @@ -93,7 +136,12 @@ private static void maybeRegisterSigner(String signerName, } } - @Override - public void close() throws IOException { + @Override public void close() throws IOException { + LOG.debug("Unregistering fs from {} initializers", initializers.size()); + for (AwsSignerInitializer initializer : initializers) { + initializer + .unregisterStore(bucketName, ownerConf, delegationTokenProvider, + ownerUgi); + } } -} +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/DelegationTokenProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/DelegationTokenProvider.java new file mode 100644 index 0000000000000..56bd9aa2eb8bf --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/DelegationTokenProvider.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.auth.delegation; + +import java.io.IOException; + +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; + +/** + * Interface for S3A Delegation Token access. + */ +public interface DelegationTokenProvider { + Token getFsDelegationToken() throws IOException; +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java index a49ab52b1ffd2..e82fbda63dd0c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java @@ -18,18 +18,18 @@ package org.apache.hadoop.fs.s3a.commit; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.Date; import java.util.List; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import com.amazonaws.services.s3.model.MultipartUpload; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,7 +49,9 @@ import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.concurrent.HadoopExecutors; +import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS; import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; @@ -66,11 +68,28 @@ * to handle the creation of a committer when the destination is unknown. * * Requiring an output directory simplifies coding and testing. + * + * The original implementation loaded all .pendingset files + * before attempting any commit/abort operations. + * While straightforward and guaranteeing that no changes were made to the + * destination until all files had successfully been loaded -it didn't scale; + * the list grew until it exceeded heap size. + * + * The second iteration builds up an {@link ActiveCommit} class with the + * list of .pendingset files to load and then commit; that can be done + * incrementally and in parallel. + * As a side effect of this change, unless/until changed, + * the commit/abort/revert of all files uploaded by a single task will be + * serialized. This may slow down these operations if there are many files + * created by a few tasks, and the HTTP connection pool in the S3A + * committer was large enough for more all the parallel POST requests. */ public abstract class AbstractS3ACommitter extends PathOutputCommitter { private static final Logger LOG = LoggerFactory.getLogger(AbstractS3ACommitter.class); + public static final String THREAD_PREFIX = "s3a-committer-pool-"; + /** * Thread pool for task execution. */ @@ -349,16 +368,11 @@ public void recoverTask(TaskAttemptContext taskContext) throws IOException { * @throws IOException IO failure */ protected void maybeCreateSuccessMarkerFromCommits(JobContext context, - List pending) throws IOException { + ActiveCommit pending) throws IOException { List filenames = new ArrayList<>(pending.size()); - for (SinglePendingCommit commit : pending) { - String key = commit.getDestinationKey(); - if (!key.startsWith("/")) { - // fix up so that FS.makeQualified() sets up the path OK - key = "/" + key; - } - filenames.add(key); - } + // The list of committed objects in pending is size limited in + // ActiveCommit.uploadCommitted. + filenames.addAll(pending.committedObjects); maybeCreateSuccessMarker(context, filenames); } @@ -390,22 +404,25 @@ protected void maybeCreateSuccessMarker(JobContext context, } /** - * Base job setup deletes the success marker. - * TODO: Do we need this? + * Base job setup (optionally) deletes the success marker and + * always creates the destination directory. + * When objects are committed that dest dir marker will inevitably + * be deleted; creating it now ensures there is something at the end + * while the job is in progress -and if nothing is created, that + * it is still there. * @param context context * @throws IOException IO failure */ -/* @Override public void setupJob(JobContext context) throws IOException { - if (createJobMarker) { - try (DurationInfo d = new DurationInfo("Deleting _SUCCESS marker")) { + try (DurationInfo d = new DurationInfo(LOG, "preparing destination")) { + if (createJobMarker){ commitOperations.deleteSuccessMarker(getOutputPath()); } + getDestFS().mkdirs(getOutputPath()); } } -*/ @Override public void setupTask(TaskAttemptContext context) throws IOException { @@ -430,28 +447,152 @@ protected FileSystem getTaskAttemptFilesystem(TaskAttemptContext context) } /** - * Commit a list of pending uploads. + * Commit all the pending uploads. + * Each file listed in the ActiveCommit instance is queued for processing + * in a separate thread; its contents are loaded and then (sequentially) + * committed. + * On a failure or abort of a single file's commit, all its uploads are + * aborted. + * The revert operation lists the files already committed and deletes them. * @param context job context - * @param pending list of pending uploads + * @param pending pending uploads * @throws IOException on any failure */ - protected void commitPendingUploads(JobContext context, - List pending) throws IOException { + protected void commitPendingUploads( + final JobContext context, + final ActiveCommit pending) throws IOException { if (pending.isEmpty()) { LOG.warn("{}: No pending uploads to commit", getRole()); } - LOG.debug("{}: committing the output of {} task(s)", - getRole(), pending.size()); - try(CommitOperations.CommitContext commitContext + try (DurationInfo ignored = new DurationInfo(LOG, + "committing the output of %s task(s)", pending.size()); + CommitOperations.CommitContext commitContext = initiateCommitOperation()) { - Tasks.foreach(pending) + + Tasks.foreach(pending.getSourceFiles()) .stopOnFailure() + .suppressExceptions(false) .executeWith(buildThreadPool(context)) + .abortWith(path -> + loadAndAbort(commitContext, pending, path, true, false)) + .revertWith(path -> + loadAndRevert(commitContext, pending, path)) + .run(path -> + loadAndCommit(commitContext, pending, path)); + } + } + + /** + * Run a precommit check that all files are loadable. + * This check avoids the situation where the inability to read + * a file only surfaces partway through the job commit, so + * results in the destination being tainted. + * @param context job context + * @param pending the pending operations + * @throws IOException any failure + */ + protected void precommitCheckPendingFiles( + final JobContext context, + final ActiveCommit pending) throws IOException { + + FileSystem sourceFS = pending.getSourceFS(); + try (DurationInfo ignored = + new DurationInfo(LOG, "Preflight Load of pending files")) { + + Tasks.foreach(pending.getSourceFiles()) + .stopOnFailure() + .suppressExceptions(false) + .executeWith(buildThreadPool(context)) + .run(path -> PendingSet.load(sourceFS, path)); + } + } + + /** + * Load a pendingset file and commit all of its contents. + * @param commitContext context to commit through + * @param activeCommit commit state + * @param path path to load + * @throws IOException failure + */ + private void loadAndCommit( + final CommitOperations.CommitContext commitContext, + final ActiveCommit activeCommit, + final Path path) throws IOException { + + try (DurationInfo ignored = + new DurationInfo(LOG, false, "Committing %s", path)) { + PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), path); + Tasks.foreach(pendingSet.getCommits()) + .stopOnFailure() + .suppressExceptions(false) + .executeWith(singleCommitThreadPool()) .onFailure((commit, exception) -> commitContext.abortSingleCommit(commit)) .abortWith(commitContext::abortSingleCommit) .revertWith(commitContext::revertCommit) - .run(commitContext::commitOrFail); + .run(commit -> { + commitContext.commitOrFail(commit); + activeCommit.uploadCommitted( + commit.getDestinationKey(), commit.getLength()); + }); + } + } + + /** + * Load a pendingset file and revert all of its contents. + * @param commitContext context to commit through + * @param activeCommit commit state + * @param path path to load + * @throws IOException failure + */ + private void loadAndRevert( + final CommitOperations.CommitContext commitContext, + final ActiveCommit activeCommit, + final Path path) throws IOException { + + try (DurationInfo ignored = + new DurationInfo(LOG, false, "Committing %s", path)) { + PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), path); + Tasks.foreach(pendingSet.getCommits()) + .suppressExceptions(true) + .run(commitContext::revertCommit); + } + } + + /** + * Load a pendingset file and abort all of its contents. + * @param commitContext context to commit through + * @param activeCommit commit state + * @param path path to load + * @param deleteRemoteFiles should remote files be deleted? + * @throws IOException failure + */ + private void loadAndAbort( + final CommitOperations.CommitContext commitContext, + final ActiveCommit activeCommit, + final Path path, + final boolean suppressExceptions, + final boolean deleteRemoteFiles) throws IOException { + + try (DurationInfo ignored = + new DurationInfo(LOG, false, "Aborting %s", path)) { + PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), + path); + FileSystem fs = getDestFS(); + Tasks.foreach(pendingSet.getCommits()) + .executeWith(singleCommitThreadPool()) + .suppressExceptions(suppressExceptions) + .run(commit -> { + try { + commitContext.abortSingleCommit(commit); + } catch (FileNotFoundException e) { + // Commit ID was not known; file may exist. + // delete it if instructed to do so. + if (deleteRemoteFiles) { + fs.delete(commit.destinationPath(), false); + } + } + }); } } @@ -465,44 +606,15 @@ protected CommitOperations.CommitContext initiateCommitOperation() return getCommitOperations().initiateCommitOperation(getOutputPath()); } - /** - * Try to read every pendingset file and build a list of them/ - * In the case of a failure to read the file, exceptions are held until all - * reads have been attempted. - * @param context job context - * @param suppressExceptions whether to suppress exceptions. - * @param fs job attempt fs - * @param pendingCommitFiles list of files found in the listing scan - * @return the list of commits - * @throws IOException on a failure when suppressExceptions is false. - */ - protected List loadPendingsetFiles( - JobContext context, - boolean suppressExceptions, - FileSystem fs, - Iterable pendingCommitFiles) throws IOException { - - final List pending = Collections.synchronizedList( - Lists.newArrayList()); - Tasks.foreach(pendingCommitFiles) - .suppressExceptions(suppressExceptions) - .executeWith(buildThreadPool(context)) - .run(pendingCommitFile -> - pending.addAll( - PendingSet.load(fs, pendingCommitFile.getPath()).getCommits()) - ); - return pending; - } - /** * Internal Job commit operation: where the S3 requests are made * (potentially in parallel). * @param context job context - * @param pending pending request + * @param pending pending commits * @throws IOException any failure */ protected void commitJobInternal(JobContext context, - List pending) + ActiveCommit pending) throws IOException { commitPendingUploads(context, pending); @@ -523,6 +635,9 @@ public void abortJob(JobContext context, JobStatus.State state) * This must clean up operations; it is called when a commit fails, as * well as in an {@link #abortJob(JobContext, JobStatus.State)} call. * The base implementation calls {@link #cleanup(JobContext, boolean)} + * so cleans up the filesystems and destroys the thread pool. + * Subclasses must always invoke this superclass method after their + * own operations. * @param context job context * @param suppressExceptions should exceptions be suppressed? * @throws IOException any IO problem raised when suppressExceptions is false. @@ -536,13 +651,15 @@ protected void abortJobInternal(JobContext context, /** * Abort all pending uploads to the destination directory during * job cleanup operations. + * Note: this instantiates the thread pool if required -so + * {@link #destroyThreadPool()} must be called after this. * @param suppressExceptions should exceptions be suppressed * @throws IOException IO problem */ protected void abortPendingUploadsInCleanup( boolean suppressExceptions) throws IOException { Path dest = getOutputPath(); - try (DurationInfo d = + try (DurationInfo ignored = new DurationInfo(LOG, "Aborting all pending commits under %s", dest); CommitOperations.CommitContext commitContext @@ -565,13 +682,18 @@ protected void abortPendingUploadsInCleanup( } /** - * Subclass-specific pre commit actions. + * Subclass-specific pre-Job-commit actions. + * The staging committers all load the pending files to verify that + * they can be loaded. + * The Magic committer does not, because of the overhead of reading files + * from S3 makes it too expensive. * @param context job context * @param pending the pending operations * @throws IOException any failure */ - protected void preCommitJob(JobContext context, - List pending) throws IOException { + @VisibleForTesting + public void preCommitJob(JobContext context, + ActiveCommit pending) throws IOException { } /** @@ -584,7 +706,7 @@ protected void preCommitJob(JobContext context, *

* Commit internal: do the final commit sequence. *

- * The final commit action is to build the {@code __SUCCESS} file entry. + * The final commit action is to build the {@code _SUCCESS} file entry. *

* @param context job context * @throws IOException any failure @@ -594,7 +716,7 @@ public void commitJob(JobContext context) throws IOException { String id = jobIdString(context); try (DurationInfo d = new DurationInfo(LOG, "%s: commitJob(%s)", getRole(), id)) { - List pending + ActiveCommit pending = listPendingUploadsToCommit(context); preCommitJob(context, pending); commitJobInternal(context, pending); @@ -629,12 +751,13 @@ protected void jobCompleted(boolean success) { * @return a list of pending uploads. * @throws IOException Any IO failure */ - protected abstract List listPendingUploadsToCommit( + protected abstract ActiveCommit listPendingUploadsToCommit( JobContext context) throws IOException; /** - * Cleanup the job context, including aborting anything pending. + * Cleanup the job context, including aborting anything pending + * and destroying the thread pool. * @param context job context * @param suppressExceptions should exceptions be suppressed? * @throws IOException any failure if exceptions were not suppressed. @@ -645,6 +768,7 @@ protected void cleanup(JobContext context, "Cleanup job %s", jobIdString(context))) { abortPendingUploadsInCleanup(suppressExceptions); } finally { + destroyThreadPool(); cleanupStagingDirs(); } } @@ -715,7 +839,7 @@ protected String getRole() { /** * Returns an {@link ExecutorService} for parallel tasks. The number of - * threads in the thread-pool is set by s3.multipart.committer.num-threads. + * threads in the thread-pool is set by fs.s3a.committer.threads. * If num-threads is 0, this will return null; * * @param context the JobContext for this commit @@ -730,10 +854,10 @@ protected final synchronized ExecutorService buildThreadPool( DEFAULT_COMMITTER_THREADS); LOG.debug("{}: creating thread pool of size {}", getRole(), numThreads); if (numThreads > 0) { - threadPool = Executors.newFixedThreadPool(numThreads, + threadPool = HadoopExecutors.newFixedThreadPool(numThreads, new ThreadFactoryBuilder() .setDaemon(true) - .setNameFormat("s3-committer-pool-%d") + .setNameFormat(THREAD_PREFIX + context.getJobID() + "-%d") .build()); } else { return null; @@ -742,6 +866,40 @@ protected final synchronized ExecutorService buildThreadPool( return threadPool; } + /** + * Destroy any thread pools; wait for that to finish, + * but don't overreact if it doesn't finish in time. + */ + protected synchronized void destroyThreadPool() { + if (threadPool != null) { + LOG.debug("Destroying thread pool"); + HadoopExecutors.shutdown(threadPool, LOG, + THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); + threadPool = null; + } + } + + /** + * Get the thread pool for executing the single file commit/revert + * within the commit of all uploads of a single task. + * This is currently null; it is here to allow the Tasks class to + * provide the logic for execute/revert. + * Why not use the existing thread pool? Too much fear of deadlocking, + * and tasks are being committed in parallel anyway. + * @return null. always. + */ + protected final synchronized ExecutorService singleCommitThreadPool() { + return null; + } + + /** + * Does this committer have a thread pool? + * @return true if a thread pool exists. + */ + public synchronized boolean hasThreadPool() { + return threadPool != null; + } + /** * Delete the task attempt path without raising any errors. * @param context task context @@ -755,6 +913,8 @@ protected void deleteTaskAttemptPathQuietly(TaskAttemptContext context) { /** * Abort all pending uploads in the list. + * This operation is used by the magic committer as part of its + * rollback after a failure during task commit. * @param context job context * @param pending pending uploads * @param suppressExceptions should exceptions be suppressed @@ -779,4 +939,172 @@ protected void abortPendingUploads(JobContext context, } } + /** + * Abort all pending uploads in the list. + * @param context job context + * @param pending pending uploads + * @param suppressExceptions should exceptions be suppressed? + * @param deleteRemoteFiles should remote files be deleted? + * @throws IOException any exception raised + */ + protected void abortPendingUploads( + final JobContext context, + final ActiveCommit pending, + final boolean suppressExceptions, + final boolean deleteRemoteFiles) throws IOException { + + if (pending.isEmpty()) { + LOG.info("{}: no pending commits to abort", getRole()); + } else { + try (DurationInfo d = new DurationInfo(LOG, + "Aborting %s uploads", pending.size()); + CommitOperations.CommitContext commitContext + = initiateCommitOperation()) { + Tasks.foreach(pending.getSourceFiles()) + .executeWith(buildThreadPool(context)) + .suppressExceptions(suppressExceptions) + .run(path -> + loadAndAbort(commitContext, + pending, + path, + suppressExceptions, + deleteRemoteFiles)); + } + } + } + + /** + * State of the active commit operation. + * + * It contains a list of all pendingset files to load as the source + * of outstanding commits to complete/abort, + * and tracks the files uploaded. + * + * To avoid running out of heap by loading all the source files + * simultaneously: + *
    + *
  1. + * The list of files to load is passed round but + * the contents are only loaded on demand. + *
  2. + *
  3. + * The number of written files tracked for logging in + * the _SUCCESS file are limited to a small amount -enough + * for testing only. + *
  4. + *
+ */ + public static class ActiveCommit { + + private static final AbstractS3ACommitter.ActiveCommit EMPTY + = new ActiveCommit(null, new ArrayList<>()); + + /** All pendingset files to iterate through. */ + private final List sourceFiles; + + /** + * Filesystem for the source files. + */ + private final FileSystem sourceFS; + + /** + * List of committed objects; only built up until the commit limit is + * reached. + */ + private final List committedObjects = new ArrayList<>(); + + /** + * The total number of committed objects. + */ + private int committedObjectCount; + + /** + * Total number of bytes committed. + */ + private long committedBytes; + + /** + * Construct from a source FS and list of files. + * @param sourceFS filesystem containing the list of pending files + * @param sourceFiles .pendingset files to load and commit. + */ + public ActiveCommit( + final FileSystem sourceFS, + final List sourceFiles) { + this.sourceFiles = sourceFiles; + this.sourceFS = sourceFS; + } + + /** + * Create an active commit of the given pending files. + * @param pendingFS source filesystem. + * @param statuses list of file status or subclass to use. + * @return the commit + */ + public static ActiveCommit fromStatusList( + final FileSystem pendingFS, + final List statuses) { + return new ActiveCommit(pendingFS, + statuses.stream() + .map(FileStatus::getPath) + .collect(Collectors.toList())); + } + + /** + * Get the empty entry. + * @return an active commit with no pending files. + */ + public static ActiveCommit empty() { + return EMPTY; + } + + public List getSourceFiles() { + return sourceFiles; + } + + public FileSystem getSourceFS() { + return sourceFS; + } + + /** + * Note that a file was committed. + * Increase the counter of files and total size. + * If there is room in the committedFiles list, the file + * will be added to the list and so end up in the _SUCCESS file. + * @param key key of the committed object. + * @param size size in bytes. + */ + public synchronized void uploadCommitted(String key, long size) { + if (committedObjects.size() < SUCCESS_MARKER_FILE_LIMIT) { + committedObjects.add( + key.startsWith("/") ? key : ("/" + key)); + } + committedObjectCount++; + committedBytes += size; + } + + public synchronized List getCommittedObjects() { + return committedObjects; + } + + public synchronized int getCommittedFileCount() { + return committedObjectCount; + } + + public synchronized long getCommittedBytes() { + return committedBytes; + } + + public int size() { + return sourceFiles.size(); + } + + public boolean isEmpty() { + return sourceFiles.isEmpty(); + } + + public void add(Path path) { + sourceFiles.add(path); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitterFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitterFactory.java index b3bcca1b97ae5..6e7a99f50ef93 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitterFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitterFactory.java @@ -51,7 +51,7 @@ public PathOutputCommitter createOutputCommitter(Path outputPath, throw new PathCommitException(outputPath, "Filesystem not supported by this committer"); } - LOG.info("Using Commmitter {} for {}", + LOG.info("Using Committer {} for {}", outputCommitter, outputPath); return outputCommitter; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java index c9b0337bcb26a..3e28a5d2cf96f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java @@ -255,4 +255,10 @@ private CommitConstants() { public static final String FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS = "fs.s3a.committer.staging.abort.pending.uploads"; + /** + * The limit to the number of committed objects tracked during + * job commits and saved to the _SUCCESS file. + */ + public static final int SUCCESS_MARKER_FILE_LIMIT = 100; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java index cf84cb32eb8b7..e0273fa11a584 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java @@ -52,6 +52,12 @@ * Applications reading this data should use/check the {@link #name} field * to differentiate from any other JSON-based manifest and to identify * changes in the output format. + * + * Note: to deal with scale issues, the S3A committers do not include any + * more than the number of objects listed in + * {@link org.apache.hadoop.fs.s3a.commit.CommitConstants#SUCCESS_MARKER_FILE_LIMIT}. + * This is intended to suffice for basic integration tests. + * Larger tests should examine the generated files themselves. */ @SuppressWarnings("unused") @InterfaceAudience.Private diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java index 969286e4d8eb0..99121730644b1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java @@ -109,11 +109,11 @@ public void setupJob(JobContext context) throws IOException { * @return a list of pending commits. * @throws IOException Any IO failure */ - protected List listPendingUploadsToCommit( + protected ActiveCommit listPendingUploadsToCommit( JobContext context) throws IOException { FileSystem fs = getDestFS(); - return loadPendingsetFiles(context, false, fs, + return ActiveCommit.fromStatusList(fs, listAndFilter(fs, getJobAttemptPath(context), false, CommitOperations.PENDINGSET_FILTER)); } @@ -174,6 +174,7 @@ public void commitTask(TaskAttemptContext context) throws IOException { } finally { // delete the task attempt so there's no possibility of a second attempt deleteTaskAttemptPathQuietly(context); + destroyThreadPool(); } getCommitOperations().taskCompleted(true); } @@ -181,7 +182,7 @@ public void commitTask(TaskAttemptContext context) throws IOException { /** * Inner routine for committing a task. * The list of pending commits is loaded and then saved to the job attempt - * dir. + * dir in a single pendingset file. * Failure to load any file or save the final file triggers an abort of * all known pending commits. * @param context context @@ -250,6 +251,7 @@ public void abortTask(TaskAttemptContext context) throws IOException { deleteQuietly( attemptPath.getFileSystem(context.getConfiguration()), attemptPath, true); + destroyThreadPool(); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java index 32642c9e98fad..1a5a63c940f47 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java @@ -20,7 +20,6 @@ import java.io.FileNotFoundException; import java.io.IOException; -import java.util.List; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,7 +30,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathExistsException; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; -import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -66,7 +64,6 @@ public String getName() { @Override public void setupJob(JobContext context) throws IOException { - super.setupJob(context); Path outputPath = getOutputPath(); FileSystem fs = getDestFS(); ConflictResolution conflictResolution = getConflictResolutionMode( @@ -91,10 +88,10 @@ public void setupJob(JobContext context) throws IOException { } } catch (FileNotFoundException ignored) { // there is no destination path, hence, no conflict. - // make the parent directory, which also triggers a recursive directory - // creation operation - fs.mkdirs(outputPath); } + // make the parent directory, which also triggers a recursive directory + // creation operation + super.setupJob(context); } /** @@ -106,8 +103,12 @@ public void setupJob(JobContext context) throws IOException { * @throws IOException any failure */ @Override - protected void preCommitJob(JobContext context, - List pending) throws IOException { + public void preCommitJob( + final JobContext context, + final ActiveCommit pending) throws IOException { + + // see if the files can be loaded. + super.preCommitJob(context, pending); Path outputPath = getOutputPath(); FileSystem fs = getDestFS(); Configuration fsConf = fs.getConf(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java index b51bcb5f9c338..20aca3cf49ae0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java @@ -20,10 +20,11 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; -import com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,11 +33,14 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.PathCommitException; +import org.apache.hadoop.fs.s3a.commit.Tasks; +import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.DurationInfo; -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.COMMITTER_NAME_PARTITIONED; /** * Partitioned committer. @@ -52,6 +56,9 @@ *
  • REPLACE: delete the destination partition in the job commit * (i.e. after and only if all tasks have succeeded.
  • * + * To determine the paths, the precommit process actually has to read + * in all source files, independently of the final commit phase. + * This is inefficient, though some parallelization here helps. */ public class PartitionedStagingCommitter extends StagingCommitter { @@ -107,6 +114,7 @@ protected int commitTaskInternal(TaskAttemptContext context, } /** + * All * Job-side conflict resolution. * The partition path conflict resolution actions are: *
      @@ -119,13 +127,15 @@ protected int commitTaskInternal(TaskAttemptContext context, * @throws IOException any failure */ @Override - protected void preCommitJob(JobContext context, - List pending) throws IOException { + public void preCommitJob( + final JobContext context, + final ActiveCommit pending) throws IOException { FileSystem fs = getDestFS(); // enforce conflict resolution Configuration fsConf = fs.getConf(); + boolean shouldPrecheckPendingFiles = true; switch (getConflictResolutionMode(context, fsConf)) { case FAIL: // FAIL checking is done on the task side, so this does nothing @@ -134,21 +144,84 @@ protected void preCommitJob(JobContext context, // no check is needed because the output may exist for appending break; case REPLACE: - Set partitions = pending.stream() - .map(SinglePendingCommit::destinationPath) - .map(Path::getParent) - .collect(Collectors.toCollection(Sets::newLinkedHashSet)); - for (Path partitionPath : partitions) { - LOG.debug("{}: removing partition path to be replaced: " + - getRole(), partitionPath); - fs.delete(partitionPath, true); - } + // identify and replace the destination partitions + replacePartitions(context, pending); + // and so there is no need to do another check. + shouldPrecheckPendingFiles = false; break; default: throw new PathCommitException("", getRole() + ": unknown conflict resolution mode: " + getConflictResolutionMode(context, fsConf)); } + if (shouldPrecheckPendingFiles) { + precommitCheckPendingFiles(context, pending); + } + } + + /** + * Identify all partitions which need to be replaced and then delete them. + * The original implementation relied on all the pending commits to be + * loaded so could simply enumerate them. + * This iteration does not do that; it has to reload all the files + * to build the set, after which it initiates the delete process. + * This is done in parallel. + *
      +   *   Set partitions = pending.stream()
      +   *     .map(Path::getParent)
      +   *     .collect(Collectors.toCollection(Sets::newLinkedHashSet));
      +   *   for (Path partitionPath : partitions) {
      +   *     LOG.debug("{}: removing partition path to be replaced: " +
      +   *     getRole(), partitionPath);
      +   *     fs.delete(partitionPath, true);
      +   *   }
      +   * 
      + * + * @param context job context + * @param pending the pending operations + * @throws IOException any failure + */ + private void replacePartitions( + final JobContext context, + final ActiveCommit pending) throws IOException { + + Map partitions = new ConcurrentHashMap<>(); + FileSystem sourceFS = pending.getSourceFS(); + ExecutorService pool = buildThreadPool(context); + try (DurationInfo ignored = + new DurationInfo(LOG, "Replacing partitions")) { + + // the parent directories are saved to a concurrent hash map. + // for a marginal optimisation, the previous parent is tracked, so + // if a task writes many files to the same dir, the synchronized map + // is updated only once. + Tasks.foreach(pending.getSourceFiles()) + .stopOnFailure() + .suppressExceptions(false) + .executeWith(pool) + .run(path -> { + PendingSet pendingSet = PendingSet.load(sourceFS, path); + Path lastParent = null; + for (SinglePendingCommit commit : pendingSet.getCommits()) { + Path parent = commit.destinationPath().getParent(); + if (parent != null && !parent.equals(lastParent)) { + partitions.put(parent, ""); + lastParent = parent; + } + } + }); + } + // now do the deletes + FileSystem fs = getDestFS(); + Tasks.foreach(partitions.keySet()) + .stopOnFailure() + .suppressExceptions(false) + .executeWith(pool) + .run(partitionPath -> { + LOG.debug("{}: removing partition path to be replaced: " + + getRole(), partitionPath); + fs.delete(partitionPath, true); + }); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java index 7ec447863c150..6cc9e488523fb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java @@ -21,7 +21,6 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.Locale; import java.util.Queue; @@ -457,6 +456,7 @@ public void setupJob(JobContext context) throws IOException { context.getConfiguration().set( InternalCommitterConstants.FS_S3A_COMMITTER_STAGING_UUID, uuid); wrappedCommitter.setupJob(context); + super.setupJob(context); } /** @@ -466,7 +466,7 @@ public void setupJob(JobContext context) throws IOException { * @throws IOException Any IO failure */ @Override - protected List listPendingUploadsToCommit( + protected ActiveCommit listPendingUploadsToCommit( JobContext context) throws IOException { return listPendingUploads(context, false); @@ -480,7 +480,7 @@ protected List listPendingUploadsToCommit( * then this may not match the actual set of pending operations * @throws IOException shouldn't be raised, but retained for the compiler */ - protected List listPendingUploadsToAbort( + protected ActiveCommit listPendingUploadsToAbort( JobContext context) throws IOException { return listPendingUploads(context, true); } @@ -493,13 +493,14 @@ protected List listPendingUploadsToAbort( * then this may not match the actual set of pending operations * @throws IOException Any IO failure which wasn't swallowed. */ - protected List listPendingUploads( + protected ActiveCommit listPendingUploads( JobContext context, boolean suppressExceptions) throws IOException { - try { - Path wrappedJobAttemptPath = wrappedCommitter.getJobAttemptPath(context); + try (DurationInfo ignored = new DurationInfo(LOG, + "Listing pending uploads")) { + Path wrappedJobAttemptPath = getJobAttemptPath(context); final FileSystem attemptFS = wrappedJobAttemptPath.getFileSystem( context.getConfiguration()); - return loadPendingsetFiles(context, suppressExceptions, attemptFS, + return ActiveCommit.fromStatusList(attemptFS, listAndFilter(attemptFS, wrappedJobAttemptPath, false, HIDDEN_FILE_FILTER)); @@ -512,7 +513,7 @@ protected List listPendingUploads( maybeIgnore(suppressExceptions, "Listing pending uploads", e); } // reached iff an IOE was caught and swallowed - return new ArrayList<>(0); + return ActiveCommit.empty(); } @Override @@ -558,8 +559,8 @@ protected void abortJobInternal(JobContext context, boolean failed = false; try (DurationInfo d = new DurationInfo(LOG, "%s: aborting job in state %s ", r, jobIdString(context))) { - List pending = listPendingUploadsToAbort(context); - abortPendingUploads(context, pending, suppressExceptions); + ActiveCommit pending = listPendingUploadsToAbort(context); + abortPendingUploads(context, pending, suppressExceptions, true); } catch (FileNotFoundException e) { // nothing to list LOG.debug("No job directory to read uploads from"); @@ -571,6 +572,7 @@ protected void abortJobInternal(JobContext context, } } + /** * Delete the working paths of a job. *
        @@ -646,6 +648,8 @@ public void commitTask(TaskAttemptContext context) throws IOException { getRole(), context.getTaskAttemptID(), e); getCommitOperations().taskCompleted(false); throw e; + } finally { + destroyThreadPool(); } getCommitOperations().taskCompleted(true); } @@ -677,7 +681,8 @@ protected int commitTaskInternal(final TaskAttemptContext context, // we will try to abort the ones that had already succeeded. int commitCount = taskOutput.size(); final Queue commits = new ConcurrentLinkedQueue<>(); - LOG.info("{}: uploading from staging directory to S3", getRole()); + LOG.info("{}: uploading from staging directory to S3 {}", getRole(), + attemptPath); LOG.info("{}: Saving pending data information to {}", getRole(), commitsAttemptPath); if (taskOutput.isEmpty()) { @@ -693,6 +698,7 @@ protected int commitTaskInternal(final TaskAttemptContext context, try { Tasks.foreach(taskOutput) .stopOnFailure() + .suppressExceptions(false) .executeWith(buildThreadPool(context)) .run(stat -> { Path path = stat.getPath(); @@ -778,6 +784,8 @@ public void abortTask(TaskAttemptContext context) throws IOException { LOG.error("{}: exception when aborting task {}", getRole(), context.getTaskAttemptID(), e); throw e; + } finally { + destroyThreadPool(); } } @@ -900,4 +908,20 @@ public static String getConfictModeOption(JobContext context, defVal).toUpperCase(Locale.ENGLISH); } + /** + * Pre-commit actions for a job. + * Loads all the pending files to verify they can be loaded + * and parsed. + * @param context job context + * @param pending pending commits + * @throws IOException any failure + */ + @Override + public void preCommitJob( + final JobContext context, + final ActiveCommit pending) throws IOException { + + // see if the files can be loaded. + precommitCheckPendingFiles(context, pending); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitterConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitterConstants.java index c5fb967863953..c41715bd497d8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitterConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitterConstants.java @@ -34,7 +34,7 @@ private StagingCommitterConstants() { /** * The temporary path for staging data, if not explicitly set. * By using an unqualified path, this will be qualified to be relative - * to the users' home directory, so protectec from access for others. + * to the users' home directory, so protected from access for others. */ public static final String FILESYSTEM_TEMP_PATH = "tmp/staging"; diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index ea55f90cc159b..ca584df7cd59d 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1879,3 +1879,61 @@ To disable checksum verification in `distcp`, use the `-skipcrccheck` option: hadoop distcp -update -skipcrccheck -numListstatusThreads 40 /user/alice/datasets s3a://alice-backup/datasets ``` +### Advanced - Custom Signers + +AWS uees request signing to authenticate requests. In general, there should +be no need to override the signers, and the defaults work out of the box. +If, however, this is required - this section talks about how to configure +custom signers. There’s 2 broad config categories to be set - one for +registering a custom signer and another to specify usage. + +#### Registering Custom Signers +```xml + + fs.s3a.custom.signers + comma separated list of signers + + +``` +Acceptable value for each custom signer + +`SignerName`- this is used in case one of the default signers is being used. +(E.g `AWS4SignerType`, `QueryStringSignerType`, `AWSS3V4SignerType`). +If no custom signers are being used - this value does not need to be set. + +`SignerName:SignerClassName` - register a new signer with the specified name, +and the class for this signer. +The Signer Class must implement `com.amazonaws.auth.Signer`. + +`SignerName:SignerClassName:SignerInitializerClassName` - similar time above +except also allows for a custom SignerInitializer +(`org.apache.hadoop.fs.s3a.AwsSignerInitializer`) class to be specified. + +#### Usage of the Signers +Signers can be set at a per service level(S3, dynamodb, etc) or a common +signer for all services. + +```xml + + fs.s3a.s3.signing-algorithm + ${S3SignerName} + Specify the signer for S3 + + + + fs.s3a.ddb.signing-algorithm + ${DdbSignerName} + Specify the signer for DDB + + + + fs.s3a.signing-algorithm + ${SignerName} + +``` + +For a specific service, the service specific signer is looked up first. +If that is not specified, the common signer is looked up. If this is +not specified as well, SDK settings are used. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java index 3510a6486a260..13110201b7e37 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java @@ -28,12 +28,35 @@ */ public class S3AContract extends AbstractBondedFSContract { + /** + * Test resource with the contract bindings used in the standard + * contract tests: {@value}. + */ public static final String CONTRACT_XML = "contract/s3a.xml"; + /** + * Instantiate, adding the s3a.xml contract file. + * This may force a reload of the entire configuration, so interferes with + * any code which has removed bucket overrides. + * @param conf configuration. + */ public S3AContract(Configuration conf) { + this(conf, true); + } + + /** + * Instantiate, optionally adding the s3a.xml contract file. + * This may force a reload of the entire configuration, so interferes with + * any code which has removed bucket overrides. + * @param conf configuration. + * @param addContractResource should the s3a.xml file be added? + */ + public S3AContract(Configuration conf, boolean addContractResource) { super(conf); //insert the base features - addConfResource(CONTRACT_XML); + if (addContractResource) { + addConfResource(CONTRACT_XML); + } } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index b05bbb48bab02..a789eb5fa9f18 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.AbstractFSContract; import org.apache.hadoop.fs.contract.AbstractFSContractTestBase; @@ -46,12 +47,16 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase @Override protected AbstractFSContract createContract(Configuration conf) { - return new S3AContract(conf); + return new S3AContract(conf, false); } @Override public void setup() throws Exception { Thread.currentThread().setName("setup"); + // force load the local FS -not because we want the FS, but we need all + // filesystems which add default configuration resources to do it before + // our tests start adding/removing options. See HADOOP-16626. + FileSystem.getLocal(new Configuration()); super.setup(); } @@ -83,7 +88,8 @@ protected int getTestTimeoutMillis() { */ @Override protected Configuration createConfiguration() { - return S3ATestUtils.prepareTestConfiguration(super.createConfiguration()); + Configuration conf = super.createConfiguration(); + return S3ATestUtils.prepareTestConfiguration(conf); } protected Configuration getConfiguration() { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java index 6e81452e26035..79772ec9dadfe 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java @@ -19,17 +19,21 @@ package org.apache.hadoop.fs.s3a; import java.io.IOException; +import java.util.Set; +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; import org.junit.Test; import org.apache.hadoop.fs.Path; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getCurrentThreadNames; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.listInitialThreadsForLifecycleChecks; import static org.apache.hadoop.test.LambdaTestUtils.*; import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED; /** - * Tests of the S3A FileSystem which is closed; just make sure - * that that basic file Ops fail meaningfully. + * Tests of the S3A FileSystem which is closed. */ public class ITestS3AClosedFS extends AbstractS3ATestBase { @@ -47,6 +51,16 @@ public void teardown() { // no op, as the FS is closed } + private static final Set THREAD_SET = + listInitialThreadsForLifecycleChecks(); + + @AfterClass + public static void checkForThreadLeakage() { + Assertions.assertThat(getCurrentThreadNames()) + .describedAs("The threads at the end of the test run") + .isSubsetOf(THREAD_SET); + } + @Test public void testClosedGetFileStatus() throws Exception { intercept(IOException.class, E_FS_CLOSED, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 1889c05431066..592c4be907db1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -63,7 +63,10 @@ import java.text.SimpleDateFormat; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.Callable; +import java.util.stream.Collectors; import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CREDENTIAL_PROVIDER_PATH; @@ -440,8 +443,7 @@ public static Path createTestPath(Path defVal) { */ public static void assumeS3GuardState(boolean shouldBeEnabled, Configuration originalConf) throws URISyntaxException { - boolean isEnabled = getTestPropertyBool(originalConf, TEST_S3GUARD_ENABLED, - originalConf.getBoolean(TEST_S3GUARD_ENABLED, false)); + boolean isEnabled = isS3GuardTestPropertySet(originalConf); Assume.assumeThat("Unexpected S3Guard test state:" + " shouldBeEnabled=" + shouldBeEnabled + " and isEnabled=" + isEnabled, @@ -459,13 +461,22 @@ public static void assumeS3GuardState(boolean shouldBeEnabled, shouldBeEnabled, Is.is(!usingNullImpl)); } + /** + * Is the test option for S3Guard set? + * @param conf configuration to examine. + * @return true if the config or system property turns s3guard tests on + */ + public static boolean isS3GuardTestPropertySet(final Configuration conf) { + return getTestPropertyBool(conf, TEST_S3GUARD_ENABLED, + conf.getBoolean(TEST_S3GUARD_ENABLED, false)); + } + /** * Conditionally set the S3Guard options from test properties. * @param conf configuration */ public static void maybeEnableS3Guard(Configuration conf) { - if (getTestPropertyBool(conf, TEST_S3GUARD_ENABLED, - conf.getBoolean(TEST_S3GUARD_ENABLED, false))) { + if (isS3GuardTestPropertySet(conf)) { // S3Guard is enabled. boolean authoritative = getTestPropertyBool(conf, TEST_S3GUARD_AUTHORITATIVE, @@ -598,7 +609,7 @@ public static Configuration prepareTestConfiguration(final Configuration conf) { String tmpDir = conf.get(HADOOP_TMP_DIR, "target/build/test"); if (testUniqueForkId != null) { // patch temp dir for the specific branch - tmpDir = tmpDir + File.pathSeparatorChar + testUniqueForkId; + tmpDir = tmpDir + File.separator + testUniqueForkId; conf.set(HADOOP_TMP_DIR, tmpDir); } conf.set(BUFFER_DIR, tmpDir); @@ -755,8 +766,9 @@ public static void removeBucketOverrides(final String bucket, for (String option : options) { final String stripped = option.substring("fs.s3a.".length()); String target = bucketPrefix + stripped; - if (conf.get(target) != null) { - LOG.debug("Removing option {}", target); + String v = conf.get(target); + if (v != null) { + LOG.debug("Removing option {}; was {}", target, v); conf.unset(target); } } @@ -1270,6 +1282,16 @@ public static boolean metadataStorePersistsAuthoritativeBit(MetadataStore ms) return Boolean.valueOf(persists); } + /** + * Set the metadata store of a filesystem instance to the given + * store, via a package-private setter method. + * @param fs filesystem. + * @param ms metastore + */ + public static void setMetadataStore(S3AFileSystem fs, MetadataStore ms) { + fs.setMetadataStore(ms); +} + public static void checkListingDoesNotContainPath(S3AFileSystem fs, Path filePath) throws IOException { final RemoteIterator listIter = @@ -1346,4 +1368,41 @@ public static S3AFileStatus awaitFileStatus(S3AFileSystem fs, STABILIZATION_TIME, PROBE_INTERVAL_MILLIS, () -> fs.getFileStatus(testFilePath)); } + + /** + * This creates a set containing all current threads and some well-known + * thread names whose existence should not fail test runs. + * They are generally static cleaner threads created by various classes + * on instantiation. + * @return a set of threads to use in later assertions. + */ + public static Set listInitialThreadsForLifecycleChecks() { + Set threadSet = getCurrentThreadNames(); + // static filesystem statistics cleaner + threadSet.add( + "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner"); + // AWS progress callbacks + threadSet.add("java-sdk-progress-listener-callback-thread"); + // another AWS thread + threadSet.add("java-sdk-http-connection-reaper"); + // java.lang.UNIXProcess. maybe if chmod is called? + threadSet.add("process reaper"); + // once a quantile has been scheduled, the mutable quantile thread pool + // is initialized; it has a minimum thread size of 1. + threadSet.add("MutableQuantiles-0"); + // IDE? + threadSet.add("Attach Listener"); + return threadSet; + } + + /** + * Get a set containing the names of all active threads. + * @return the current set of threads. + */ + public static Set getCurrentThreadNames() { + return Thread.getAllStackTraces().keySet() + .stream() + .map(Thread::getName) + .collect(Collectors.toCollection(TreeSet::new)); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSignerManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSignerManager.java deleted file mode 100644 index ac759d0976a31..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSignerManager.java +++ /dev/null @@ -1,130 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a; - -import java.util.concurrent.TimeUnit; - -import com.amazonaws.SignableRequest; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.Signer; -import com.amazonaws.auth.SignerFactory; -import org.assertj.core.api.Assertions; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; - -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.test.LambdaTestUtils; - -import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS; - -/** - * Tests for the SignerManager. - */ -public class TestSignerManager { - - @Rule - public Timeout testTimeout = new Timeout( - 10_000L, TimeUnit.MILLISECONDS - ); - - @Test - public void testCustomSignerFailureIfNotRegistered() throws Exception { - LambdaTestUtils.intercept(Exception.class, - () -> SignerFactory.createSigner("testsignerUnregistered", null)); - // Expecting generic Exception.class to handle future implementation - // changes. - // For now, this is an NPE - } - - @Test - public void testCustomSignerInitialization() { - Configuration config = new Configuration(); - SignerForTest1.reset(); - SignerForTest2.reset(); - config.set(CUSTOM_SIGNERS, "testsigner1:" + SignerForTest1.class.getName()); - SignerManager signerManager = new SignerManager(); - signerManager.initCustomSigners(config); - Signer s1 = SignerFactory.createSigner("testsigner1", null); - s1.sign(null, null); - Assertions.assertThat(SignerForTest1.initialized) - .as(SignerForTest1.class.getName() + " not initialized") - .isEqualTo(true); - } - - @Test - public void testMultipleCustomSignerInitialization() { - Configuration config = new Configuration(); - SignerForTest1.reset(); - SignerForTest2.reset(); - config.set(CUSTOM_SIGNERS, - "testsigner1:" + SignerForTest1.class.getName() + "," + "testsigner2:" - + SignerForTest2.class.getName()); - SignerManager signerManager = new SignerManager(); - signerManager.initCustomSigners(config); - Signer s1 = SignerFactory.createSigner("testsigner1", null); - s1.sign(null, null); - Assertions.assertThat(SignerForTest1.initialized) - .as(SignerForTest1.class.getName() + " not initialized") - .isEqualTo(true); - - Signer s2 = SignerFactory.createSigner("testsigner2", null); - s2.sign(null, null); - Assertions.assertThat(SignerForTest2.initialized) - .as(SignerForTest2.class.getName() + " not initialized") - .isEqualTo(true); - } - - /** - * SignerForTest1. - */ - @Private - public static class SignerForTest1 implements Signer { - - private static boolean initialized = false; - - @Override - public void sign(SignableRequest request, AWSCredentials credentials) { - initialized = true; - } - - public static void reset() { - initialized = false; - } - } - - /** - * SignerForTest2. - */ - @Private - public static class SignerForTest2 implements Signer { - - private static boolean initialized = false; - - @Override - public void sign(SignableRequest request, AWSCredentials credentials) { - initialized = true; - } - - public static void reset() { - initialized = false; - } - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java new file mode 100644 index 0000000000000..651cdadfe7d93 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.auth; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import com.amazonaws.SignableRequest; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.Signer; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; +import com.amazonaws.services.s3.internal.AWSS3V4Signer; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider; +import org.apache.hadoop.fs.s3a.auth.ITestCustomSigner.CustomSignerInitializer.StoreValue; +import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; +import org.apache.hadoop.security.UserGroupInformation; + +import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS; +import static org.apache.hadoop.fs.s3a.Constants.SIGNING_ALGORITHM_S3; + +/** + * Tests for custom Signers and SignerInitializers. + */ +public class ITestCustomSigner extends AbstractS3ATestBase { + + private static final Logger LOG = LoggerFactory + .getLogger(ITestCustomSigner.class); + + private static final String TEST_ID_KEY = "TEST_ID_KEY"; + private static final String TEST_REGION_KEY = "TEST_REGION_KEY"; + + private String regionName; + + @Override + public void setup() throws Exception { + super.setup(); + regionName = determineRegion(getFileSystem().getBucket()); + LOG.info("Determined region name to be [{}] for bucket [{}]", regionName, + getFileSystem().getBucket()); + } + + @Test + public void testCustomSignerAndInitializer() + throws IOException, InterruptedException { + + UserGroupInformation ugi1 = UserGroupInformation.createRemoteUser("user1"); + FileSystem fs1 = runMkDirAndVerify(ugi1, "/customsignerpath1", "id1"); + + UserGroupInformation ugi2 = UserGroupInformation.createRemoteUser("user2"); + FileSystem fs2 = runMkDirAndVerify(ugi2, "/customsignerpath2", "id2"); + + Assertions.assertThat(CustomSignerInitializer.knownStores.size()) + .as("Num registered stores mismatch").isEqualTo(2); + fs1.close(); + Assertions.assertThat(CustomSignerInitializer.knownStores.size()) + .as("Num registered stores mismatch").isEqualTo(1); + fs2.close(); + Assertions.assertThat(CustomSignerInitializer.knownStores.size()) + .as("Num registered stores mismatch").isEqualTo(0); + } + + private FileSystem runMkDirAndVerify(UserGroupInformation ugi, + String pathString, String identifier) + throws IOException, InterruptedException { + Configuration conf = createTestConfig(identifier); + Path path = new Path(pathString); + path = path.makeQualified(getFileSystem().getUri(), + getFileSystem().getWorkingDirectory()); + + Path finalPath = path; + return ugi.doAs((PrivilegedExceptionAction) () -> { + int invocationCount = CustomSigner.invocationCount; + FileSystem fs = finalPath.getFileSystem(conf); + fs.mkdirs(finalPath); + Assertions.assertThat(CustomSigner.invocationCount) + .as("Invocation count lower than expected") + .isGreaterThan(invocationCount); + + Assertions.assertThat(CustomSigner.lastStoreValue) + .as("Store value should not be null").isNotNull(); + Assertions.assertThat(CustomSigner.lastStoreValue.conf) + .as("Configuration should not be null").isNotNull(); + Assertions.assertThat(CustomSigner.lastStoreValue.conf.get(TEST_ID_KEY)) + .as("Configuration TEST_KEY mismatch").isEqualTo(identifier); + + return fs; + }); + } + + private Configuration createTestConfig(String identifier) { + Configuration conf = createConfiguration(); + + conf.set(CUSTOM_SIGNERS, + "CustomS3Signer:" + CustomSigner.class.getName() + ":" + + CustomSignerInitializer.class.getName()); + conf.set(SIGNING_ALGORITHM_S3, "CustomS3Signer"); + + conf.set(TEST_ID_KEY, identifier); + conf.set(TEST_REGION_KEY, regionName); + + return conf; + } + + private String determineRegion(String bucketName) throws IOException { + AmazonS3 s3 = AmazonS3ClientBuilder.standard().withCredentials( + new SimpleAWSCredentialsProvider(null, createConfiguration())) + .withForceGlobalBucketAccessEnabled(true).withRegion("us-east-1") + .build(); + String region = s3.getBucketLocation(bucketName); + // See: https://forums.aws.amazon.com/thread.jspa?messageID=796829&tstart=0 + if (region.equals("US")) { + region = "us-east-1"; + } + return region; + } + + @Private + public static final class CustomSigner implements Signer { + + private static int invocationCount = 0; + private static StoreValue lastStoreValue; + + @Override + public void sign(SignableRequest request, AWSCredentials credentials) { + invocationCount++; + String host = request.getEndpoint().getHost(); + String bucketName = host.split("\\.")[0]; + try { + lastStoreValue = CustomSignerInitializer + .getStoreValue(bucketName, UserGroupInformation.getCurrentUser()); + } catch (IOException e) { + throw new RuntimeException("Failed to get current Ugi", e); + } + AWSS3V4Signer realSigner = new AWSS3V4Signer(); + realSigner.setServiceName("s3"); + realSigner.setRegionName(lastStoreValue.conf.get(TEST_REGION_KEY)); + realSigner.sign(request, credentials); + } + } + + @Private + public static final class CustomSignerInitializer + implements AwsSignerInitializer { + + private static final Map knownStores = new HashMap<>(); + + @Override + public void registerStore(String bucketName, Configuration storeConf, + DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) { + StoreKey storeKey = new StoreKey(bucketName, storeUgi); + StoreValue storeValue = new StoreValue(storeConf, dtProvider); + knownStores.put(storeKey, storeValue); + } + + @Override + public void unregisterStore(String bucketName, Configuration storeConf, + DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) { + StoreKey storeKey = new StoreKey(bucketName, storeUgi); + knownStores.remove(storeKey); + } + + public static StoreValue getStoreValue(String bucketName, + UserGroupInformation ugi) { + StoreKey storeKey = new StoreKey(bucketName, ugi); + return knownStores.get(storeKey); + } + + private static class StoreKey { + private final String bucketName; + private final UserGroupInformation ugi; + + public StoreKey(String bucketName, UserGroupInformation ugi) { + this.bucketName = bucketName; + this.ugi = ugi; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StoreKey storeKey = (StoreKey) o; + return Objects.equals(bucketName, storeKey.bucketName) && Objects + .equals(ugi, storeKey.ugi); + } + + @Override + public int hashCode() { + return Objects.hash(bucketName, ugi); + } + } + + static class StoreValue { + private final Configuration conf; + private final DelegationTokenProvider dtProvider; + + public StoreValue(Configuration conf, + DelegationTokenProvider dtProvider) { + this.conf = conf; + this.dtProvider = dtProvider; + } + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java index a741cd6f9d5fc..0b032196adfe0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java @@ -20,13 +20,15 @@ import java.io.FileNotFoundException; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.AccessDeniedException; import java.util.Arrays; import java.util.Collection; +import java.util.Optional; import java.util.concurrent.Callable; import org.assertj.core.api.Assertions; +import org.assertj.core.api.Assumptions; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -41,14 +43,11 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; -import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.S3AUtils; import org.apache.hadoop.fs.s3a.Statistic; -import org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore; -import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; -import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore; +import org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore; import org.apache.hadoop.mapred.LocatedFileStatusFetcher; import org.apache.hadoop.mapreduce.lib.input.InvalidInputException; @@ -58,9 +57,12 @@ import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeS3GuardState; import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.isS3GuardTestPropertySet; import static org.apache.hadoop.fs.s3a.S3ATestUtils.lsR; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects; import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement; @@ -81,8 +83,7 @@ * test for S3Guard + Auth to see how failures move around. *
          *
        1. Tests only run if an assumed role is provided.
        2. - *
        3. And the s3guard tests use the local metastore if - * there was not one already.
        4. + *
        5. And the S3Guard tests require DynamoDB.
        6. *
        * The tests are all bundled into one big test case. * From a purist unit test perspective, this is utterly wrong as it goes @@ -106,6 +107,7 @@ * To simplify maintenance, the operations tested are broken up into * their own methods, with fields used to share the restricted role and * created paths. + * */ @SuppressWarnings("ThrowableNotThrown") @RunWith(Parameterized.class) @@ -143,7 +145,9 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase { /** * Text used in files. */ - public static final byte[] HELLO = "hello".getBytes(Charset.forName("UTF-8")); + public static final byte[] HELLO = "hello".getBytes(StandardCharsets.UTF_8); + + private final boolean guardedInAuthMode; /** * Wildcard scan to find *.txt in the no-read directory. @@ -209,20 +213,27 @@ public ITestRestrictedReadAccess( this.name = name; this.s3guard = s3guard; this.authMode = authMode; + this.guardedInAuthMode = s3guard && authMode; + } @Override public Configuration createConfiguration() { Configuration conf = super.createConfiguration(); String bucketName = getTestBucketName(conf); - removeBucketOverrides(bucketName, conf, - S3_METADATA_STORE_IMPL, + // is s3guard enabled? + boolean guardedTestRun = isS3GuardTestPropertySet(conf); + + // in a guarded test run, except for the special case of raw, + // all DDB settings are left alone. + removeBaseAndBucketOverrides(bucketName, conf, METADATASTORE_AUTHORITATIVE); - conf.setClass(Constants.S3_METADATA_STORE_IMPL, - s3guard ? - LocalMetadataStore.class - : NullMetadataStore.class, - MetadataStore.class); + removeBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + if (!s3guard) { + removeBaseAndBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + } conf.setBoolean(METADATASTORE_AUTHORITATIVE, authMode); disableFilesystemCaching(conf); return conf; @@ -231,13 +242,21 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); + if (s3guard) { + // s3guard is required for those test runs where any of the + // guard options are set + assumeS3GuardState(true, getConfiguration()); + } assumeRoleTests(); } @Override public void teardown() throws Exception { - S3AUtils.closeAll(LOG, readonlyFS); - super.teardown(); + try { + super.teardown(); + } finally { + S3AUtils.closeAll(LOG, readonlyFS); + } } private void assumeRoleTests() { @@ -295,6 +314,7 @@ public void initNoReadAccess() throws Throwable { describe("Setting up filesystem"); S3AFileSystem realFS = getFileSystem(); + verifyS3GuardSettings(realFS, "real filesystem"); // avoiding the parameterization to steer clear of accidentally creating // patterns @@ -328,6 +348,9 @@ public void initNoReadAccess() throws Throwable { subdir2File2 = new Path(subDir2, "subdir2File2.docx"); createFile(realFS, subdir2File1, true, HELLO); createFile(realFS, subdir2File2, true, HELLO); + // execute a recursive list to make sure that S3Guard tables are always + // up to date + lsR(realFS, noReadDir, true); // create a role filesystem which does not have read access under a path // it still has write access, which can be explored in the final @@ -341,6 +364,35 @@ public void initNoReadAccess() throws Throwable { .addActions(S3_ALL_GET) .addResources(directory(noReadDir))); readonlyFS = (S3AFileSystem) basePath.getFileSystem(roleConfig); + verifyS3GuardSettings(readonlyFS, "readonly"); + } + + /** + * Verify that the FS (real or restricted) meets the + * requirement of the test. + * S3Guard tests are skipped if the (default) store is not + * a DDB store consistent across all FS instances. + * The raw tests fail if somehow the FS does still have a S3Guard metastore. + * @param fs filesystem + * @param storeType store role for error messages. + */ + protected void verifyS3GuardSettings(final S3AFileSystem fs, + final String storeType) { + if (s3guard) { + Assumptions.assumeThat(fs.getMetadataStore()) + .describedAs("Metadata store in " + + storeType + + " fs: %s", + fs.getMetadataStore()) + .isInstanceOf(DynamoDBMetadataStore.class); + } else { + Assertions.assertThat(fs.hasMetadataStore()) + .describedAs("Metadata store in " + + storeType + + " fs: %s", + fs.getMetadataStore()) + .isFalse(); + } } /** @@ -355,53 +407,50 @@ public void checkBasicFileOperations() throws Throwable { // - LIST path -> list results // Because the client has list access, this succeeds readonlyFS.listStatus(basePath); + lsR(readonlyFS, basePath, true); - // this is HEAD + "/" on S3; get on S3Guard auth - readonlyFS.listStatus(emptyDir); + + // this is HEAD + "/" on S3; get on S3Guard auth when the path exists, + accessDeniedIf(!s3guard, () -> + readonlyFS.listStatus(emptyDir)); // a recursive list of the no-read-directory works because // there is no directory marker, it becomes a LIST call. lsR(readonlyFS, noReadDir, true); - // similarly, a getFileStatus ends up being a list and generating - // a file status marker. + // similarly, a getFileStatus ends up being a list of the path + // and so working. readonlyFS.getFileStatus(noReadDir); - // empty dir checks work! - readonlyFS.getFileStatus(emptyDir); + // empty dir checks work when guarded because even in non-auth mode + // there are no checks for directories being out of date + // without S3, the HEAD path + "/" is blocked + accessDeniedIf(!s3guard, () -> + readonlyFS.getFileStatus(emptyDir)); // now look at a file; the outcome depends on the mode. - if (authMode) { - // auth mode doesn't check S3, so no failure - readonlyFS.getFileStatus(subdirFile); - } else { - accessDenied(() -> - readonlyFS.getFileStatus(subdirFile)); - } + accessDeniedIf(!guardedInAuthMode, () -> + readonlyFS.getFileStatus(subdirFile)); // irrespective of mode, the attempt to read the data will fail. // the only variable is where the failure occurs accessDenied(() -> ContractTestUtils.readUTF8(readonlyFS, subdirFile, HELLO.length)); - // the empty file is interesting - if (!authMode) { - // non-auth mode, it fails at some point in the opening process. - // due to a HEAD being called on the object - accessDenied(() -> - ContractTestUtils.readUTF8(readonlyFS, emptyFile, 0)); - } else { - // auth mode doesn't check the store. - // Furthermore, because it knows the file length is zero, - // it returns -1 without even opening the file. - // This means that permissions on the file do not get checked. - // See: HADOOP-16464. - try (FSDataInputStream is = readonlyFS.open(emptyFile)) { + // the empty file is interesting. + // auth mode doesn't check the store. + // Furthermore, because it knows the file length is zero, + // it returns -1 without even opening the file. + // This means that permissions on the file do not get checked. + // See: HADOOP-16464. + Optional optIn = accessDeniedIf( + !guardedInAuthMode, () -> readonlyFS.open(emptyFile)); + if (optIn.isPresent()) { + try (FSDataInputStream is = optIn.get()) { Assertions.assertThat(is.read()) .describedAs("read of empty file") .isEqualTo(-1); } - readonlyFS.getFileStatus(subdirFile); } } @@ -414,27 +463,31 @@ public void checkGlobOperations() throws Throwable { // baseline: the real filesystem on a subdir globFS(getFileSystem(), subdirFile, null, false, 1); // a file fails if not in auth mode - globFS(readonlyFS, subdirFile, null, !authMode, 1); + globFS(readonlyFS, subdirFile, null, !guardedInAuthMode, 1); // empty directories don't fail. - assertStatusPathEquals(emptyDir, - globFS(readonlyFS, emptyDir, null, false, 1)); + FileStatus[] st = globFS(readonlyFS, emptyDir, null, !s3guard, 1); + if (s3guard) { + assertStatusPathEquals(emptyDir, st); + } - FileStatus[] st = globFS(readonlyFS, + st = globFS(readonlyFS, noReadWildcard, - null, false, 2); - Assertions.assertThat(st) - .extracting(FileStatus::getPath) - .containsExactlyInAnyOrder(subdirFile, subdir2File1); + null, !s3guard, 2); + if (s3guard) { + Assertions.assertThat(st) + .extracting(FileStatus::getPath) + .containsExactlyInAnyOrder(subdirFile, subdir2File1); + } // there is precisely one .docx file (subdir2File2.docx) globFS(readonlyFS, new Path(noReadDir, "*/*.docx"), - null, false, 1); + null, !s3guard, 1); // there are no .doc files. globFS(readonlyFS, new Path(noReadDir, "*/*.doc"), - null, false, 0); + null, !s3guard, 0); globFS(readonlyFS, noReadDir, EVERYTHING, false, 1); // and a filter without any wildcarded pattern only finds @@ -461,15 +514,17 @@ public void checkSingleThreadedLocatedFileStatus() throws Throwable { true, HIDDEN_FILE_FILTER, true); - Assertions.assertThat(fetcher.getFileStatuses()) - .describedAs("result of located scan") - .flatExtracting(FileStatus::getPath) - .containsExactlyInAnyOrder( - emptyFile, - subdirFile, - subdir2File1, - subdir2File2); - + accessDeniedIf(!s3guard, + () -> fetcher.getFileStatuses()) + .ifPresent(stats -> { + Assertions.assertThat(stats) + .describedAs("result of located scan").flatExtracting(FileStatus::getPath) + .containsExactlyInAnyOrder( + emptyFile, + subdirFile, + subdir2File1, + subdir2File2); + }); } /** @@ -481,18 +536,22 @@ public void checkLocatedFileStatusFourThreads() throws Throwable { int threads = 4; describe("LocatedFileStatusFetcher with %d", threads); roleConfig.setInt(LIST_STATUS_NUM_THREADS, threads); - LocatedFileStatusFetcher fetcher2 = + LocatedFileStatusFetcher fetcher = new LocatedFileStatusFetcher( roleConfig, new Path[]{noReadWildcard}, true, EVERYTHING, true); - Assertions.assertThat(fetcher2.getFileStatuses()) - .describedAs("result of located scan") - .isNotNull() - .flatExtracting(FileStatus::getPath) - .containsExactlyInAnyOrder(subdirFile, subdir2File1); + accessDeniedIf(!s3guard, + () -> fetcher.getFileStatuses()) + .ifPresent(stats -> { + Assertions.assertThat(stats) + .describedAs("result of located scan") + .isNotNull() + .flatExtracting(FileStatus::getPath) + .containsExactlyInAnyOrder(subdirFile, subdir2File1); + }); } /** @@ -502,27 +561,22 @@ public void checkLocatedFileStatusScanFile() throws Throwable { // pass in a file as the base of the scan. describe("LocatedFileStatusFetcher with file %s", subdirFile); roleConfig.setInt(LIST_STATUS_NUM_THREADS, 16); - try { - Iterable fetched = new LocatedFileStatusFetcher( - roleConfig, - new Path[]{subdirFile}, - true, - TEXT_FILE, - true).getFileStatuses(); - // when not in auth mode, the HEAD request MUST have failed. - failif(!authMode, "LocatedFileStatusFetcher(" + subdirFile + ")" - + " should have failed"); - // and in auth mode, the file MUST have been found. - Assertions.assertThat(fetched) - .describedAs("result of located scan") - .isNotNull() - .flatExtracting(FileStatus::getPath) - .containsExactly(subdirFile); - } catch (AccessDeniedException e) { - // we require the HEAD request to fail with access denied in non-auth - // mode, but not in auth mode. - failif(authMode, "LocatedFileStatusFetcher(" + subdirFile + ")", e); - } + LocatedFileStatusFetcher fetcher + = new LocatedFileStatusFetcher( + roleConfig, + new Path[]{subdirFile}, + true, + TEXT_FILE, + true); + accessDeniedIf(!guardedInAuthMode, + () -> fetcher.getFileStatuses()) + .ifPresent(stats -> { + Assertions.assertThat(stats) + .describedAs("result of located scan") + .isNotNull() + .flatExtracting(FileStatus::getPath) + .containsExactly(subdirFile); + }); } /** @@ -627,6 +681,27 @@ protected AccessDeniedException accessDenied(final Callable eval) return intercept(AccessDeniedException.class, eval); } + /** + * Conditionally expect an operation to fail with an AccessDeniedException. + * @param condition the condition which must be true for access to be denied + * @param eval closure to evaluate. + * @param type of callable + * @return the return value if the call succeeded + * and did not return null. + * @throws Exception any unexpected exception + */ + protected Optional accessDeniedIf( + final boolean condition, + final Callable eval) + throws Exception { + if (condition) { + intercept(AccessDeniedException.class, eval); + return Optional.empty(); + } else { + return Optional.ofNullable(eval.call()); + } + } + /** * Assert that a status array has exactly one element and its * value is as expected. @@ -689,6 +764,8 @@ protected FileStatus[] globFS( failif(!expectAuthFailure, "Access denied in glob of " + path, e); return null; + } catch (IOException | RuntimeException e) { + throw new AssertionError("Other exception raised in glob:" + e, e); } if (expectedCount < 0) { Assertions.assertThat(st) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java new file mode 100644 index 0000000000000..ca87b5c1b34a6 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java @@ -0,0 +1,590 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.fs.s3a.auth; + +import java.io.Closeable; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.security.PrivilegedExceptionAction; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.TimeUnit; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.DefaultRequest; +import com.amazonaws.SignableRequest; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.Signer; +import com.amazonaws.auth.SignerFactory; +import org.assertj.core.api.Assertions; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.auth.TestSignerManager.SignerInitializerForTest.StoreValue; +import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.test.LambdaTestUtils; + +import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS; + +/** + * Tests for the SignerManager. + */ +public class TestSignerManager { + + private static final Text TEST_TOKEN_KIND = new Text("TestTokenKind"); + private static final Text TEST_TOKEN_SERVICE = new Text("TestTokenService"); + private static final String TEST_KEY_IDENTIFIER = "TEST_KEY_IDENTIFIER"; + private static final String BUCKET1 = "bucket1"; + private static final String BUCKET2 = "bucket2"; + private static final String TESTUSER1 = "testuser1"; + private static final String TESTUSER2 = "testuser2"; + + @Rule public Timeout testTimeout = new Timeout(10_000L, + TimeUnit.MILLISECONDS); + + @Before + public void beforeTest() { + SignerForTest1.reset(); + SignerForTest2.reset(); + SignerInitializerForTest.reset(); + SignerForInitializerTest.reset(); + SignerInitializer2ForTest.reset(); + } + + @Test + public void testPredefinedSignerInitialization() throws IOException { + // Try initializing a pre-defined Signer type. + // Should run through without an exception. + Configuration config = new Configuration(); + // Pre-defined signer types as of AWS-SDK 1.11.563 + // AWS4SignerType, QueryStringSignerType, AWSS3V4SignerType + config.set(CUSTOM_SIGNERS, "AWS4SignerType"); + SignerManager signerManager = new SignerManager("dontcare", null, config, + UserGroupInformation.getCurrentUser()); + signerManager.initCustomSigners(); + } + + @Test + public void testCustomSignerFailureIfNotRegistered() throws Exception { + Configuration config = new Configuration(); + config.set(CUSTOM_SIGNERS, "testsignerUnregistered"); + SignerManager signerManager = new SignerManager("dontcare", null, config, + UserGroupInformation.getCurrentUser()); + // Make sure the config is respected. + signerManager.initCustomSigners(); + // Simulate a call from the AWS SDK to create the signer. + LambdaTestUtils.intercept(Exception.class, + () -> SignerFactory.createSigner("testsignerUnregistered", null)); + // Expecting generic Exception.class to handle future implementation + // changes. + // For now, this is an NPE + } + + @Test + public void testCustomSignerInitialization() throws IOException { + Configuration config = new Configuration(); + config.set(CUSTOM_SIGNERS, "testsigner1:" + SignerForTest1.class.getName()); + SignerManager signerManager = new SignerManager("dontcare", null, config, + UserGroupInformation.getCurrentUser()); + signerManager.initCustomSigners(); + Signer s1 = SignerFactory.createSigner("testsigner1", null); + s1.sign(null, null); + Assertions.assertThat(SignerForTest1.initialized) + .as(SignerForTest1.class.getName() + " not initialized") + .isEqualTo(true); + } + + @Test + public void testMultipleCustomSignerInitialization() throws IOException { + Configuration config = new Configuration(); + config.set(CUSTOM_SIGNERS, + "testsigner1:" + SignerForTest1.class.getName() + "," + "testsigner2:" + + SignerForTest2.class.getName()); + SignerManager signerManager = new SignerManager("dontcare", null, config, + UserGroupInformation.getCurrentUser()); + signerManager.initCustomSigners(); + Signer s1 = SignerFactory.createSigner("testsigner1", null); + s1.sign(null, null); + Assertions.assertThat(SignerForTest1.initialized) + .as(SignerForTest1.class.getName() + " not initialized") + .isEqualTo(true); + + Signer s2 = SignerFactory.createSigner("testsigner2", null); + s2.sign(null, null); + Assertions.assertThat(SignerForTest2.initialized) + .as(SignerForTest2.class.getName() + " not initialized") + .isEqualTo(true); + } + + @Test + public void testSimpleSignerInitializer() throws IOException { + Configuration config = new Configuration(); + config.set(CUSTOM_SIGNERS, + "testsigner1:" + SignerForTest1.class.getName() + ":" + + SignerInitializerForTest.class.getName()); + + Token token = createTokenForTest("identifier"); + DelegationTokenProvider dtProvider = new DelegationTokenProviderForTest( + token); + + UserGroupInformation ugi = UserGroupInformation + .createRemoteUser("testuser"); + + SignerManager signerManager = new SignerManager("bucket1", dtProvider, + config, ugi); + signerManager.initCustomSigners(); + Assertions.assertThat(SignerInitializerForTest.instanceCount) + .as(SignerInitializerForTest.class.getName() + + " creation count mismatch").isEqualTo(1); + Assertions.assertThat(SignerInitializerForTest.registerCount) + .as(SignerInitializerForTest.class.getName() + + " registration count mismatch").isEqualTo(1); + Assertions.assertThat(SignerInitializerForTest.unregisterCount) + .as(SignerInitializerForTest.class.getName() + + " registration count mismatch").isEqualTo(0); + + signerManager.close(); + Assertions.assertThat(SignerInitializerForTest.unregisterCount) + .as(SignerInitializerForTest.class.getName() + + " registration count mismatch").isEqualTo(1); + } + + @Test + public void testMultipleSignerInitializers() throws IOException { + Configuration config = new Configuration(); + config.set(CUSTOM_SIGNERS, + "testsigner1:" + SignerForTest1.class.getName() + ":" + + SignerInitializerForTest.class.getName() + "," // 2nd signer + + "testsigner2:" + SignerForTest2.class.getName() + "," + // 3rd signer + + "testsigner3:" + SignerForTest2.class.getName() + ":" + + SignerInitializer2ForTest.class.getName()); + + Token token = createTokenForTest("identifier"); + DelegationTokenProvider dtProvider = new DelegationTokenProviderForTest( + token); + + UserGroupInformation ugi = UserGroupInformation + .createRemoteUser("testuser"); + + SignerManager signerManager = new SignerManager("bucket1", dtProvider, + config, ugi); + signerManager.initCustomSigners(); + + Assertions.assertThat(SignerInitializerForTest.instanceCount) + .as(SignerInitializerForTest.class.getName() + + " creation count mismatch").isEqualTo(1); + Assertions.assertThat(SignerInitializerForTest.registerCount) + .as(SignerInitializerForTest.class.getName() + + " registration count mismatch").isEqualTo(1); + Assertions.assertThat(SignerInitializerForTest.unregisterCount) + .as(SignerInitializerForTest.class.getName() + + " registration count mismatch").isEqualTo(0); + + Assertions.assertThat(SignerInitializer2ForTest.instanceCount) + .as(SignerInitializer2ForTest.class.getName() + + " creation count mismatch").isEqualTo(1); + Assertions.assertThat(SignerInitializer2ForTest.registerCount) + .as(SignerInitializer2ForTest.class.getName() + + " registration count mismatch").isEqualTo(1); + Assertions.assertThat(SignerInitializer2ForTest.unregisterCount) + .as(SignerInitializer2ForTest.class.getName() + + " registration count mismatch").isEqualTo(0); + + signerManager.close(); + Assertions.assertThat(SignerInitializerForTest.unregisterCount) + .as(SignerInitializerForTest.class.getName() + + " registration count mismatch").isEqualTo(1); + Assertions.assertThat(SignerInitializer2ForTest.unregisterCount) + .as(SignerInitializer2ForTest.class.getName() + + " registration count mismatch").isEqualTo(1); + } + + @Test + public void testSignerInitializerMultipleInstances() + throws IOException, InterruptedException { + + String id1 = "id1"; + String id2 = "id2"; + String id3 = "id3"; + UserGroupInformation ugiU1 = UserGroupInformation + .createRemoteUser(TESTUSER1); + UserGroupInformation ugiU2 = UserGroupInformation + .createRemoteUser(TESTUSER2); + + SignerManager signerManagerU1B1 = fakeS3AInstanceCreation(id1, + SignerForInitializerTest.class, SignerInitializerForTest.class, BUCKET1, + ugiU1); + SignerManager signerManagerU2B1 = fakeS3AInstanceCreation(id2, + SignerForInitializerTest.class, SignerInitializerForTest.class, BUCKET1, + ugiU2); + SignerManager signerManagerU2B2 = fakeS3AInstanceCreation(id3, + SignerForInitializerTest.class, SignerInitializerForTest.class, BUCKET2, + ugiU2); + + Assertions.assertThat(SignerInitializerForTest.instanceCount) + .as(SignerInitializerForTest.class.getName() + + " creation count mismatch").isEqualTo(3); + Assertions.assertThat(SignerInitializerForTest.registerCount) + .as(SignerInitializerForTest.class.getName() + + " registration count mismatch").isEqualTo(3); + Assertions.assertThat(SignerInitializerForTest.unregisterCount) + .as(SignerInitializerForTest.class.getName() + + " registration count mismatch").isEqualTo(0); + + // Simulate U1B1 making a request + attemptSignAndVerify(id1, BUCKET1, ugiU1, false); + + // Simulate U2B1 making a request + attemptSignAndVerify(id2, BUCKET1, ugiU2, false); + + // Simulate U2B2 making a request + attemptSignAndVerify(id3, BUCKET2, ugiU2, false); + + // Simulate U1B2 (not defined - so Signer should get a null) + attemptSignAndVerify("dontcare", BUCKET2, ugiU1, true); + + closeAndVerifyNull(signerManagerU1B1, BUCKET1, ugiU1, 2); + closeAndVerifyNull(signerManagerU2B2, BUCKET2, ugiU2, 1); + closeAndVerifyNull(signerManagerU2B1, BUCKET1, ugiU2, 0); + + Assertions.assertThat(SignerInitializerForTest.unregisterCount) + .as(SignerInitializerForTest.class.getName() + + " registration count mismatch").isEqualTo(3); + } + + private void attemptSignAndVerify(String identifier, String bucket, + UserGroupInformation ugi, boolean expectNullStoreInfo) + throws IOException, InterruptedException { + ugi.doAs((PrivilegedExceptionAction) () -> { + Signer signer = new SignerForInitializerTest(); + SignableRequest signableRequest = constructSignableRequest(bucket); + signer.sign(signableRequest, null); + verifyStoreValueInSigner(expectNullStoreInfo, bucket, identifier); + return null; + }); + } + + private void verifyStoreValueInSigner(boolean expectNull, String bucketName, + String identifier) throws IOException { + if (expectNull) { + Assertions.assertThat(SignerForInitializerTest.retrievedStoreValue) + .as("Retrieved store value expected to be null").isNull(); + } else { + StoreValue storeValue = SignerForInitializerTest.retrievedStoreValue; + Assertions.assertThat(storeValue).as("StoreValue should not be null") + .isNotNull(); + Assertions.assertThat(storeValue.getBucketName()) + .as("Bucket Name mismatch").isEqualTo(bucketName); + Configuration conf = storeValue.getStoreConf(); + Assertions.assertThat(conf).as("Configuration should not be null") + .isNotNull(); + Assertions.assertThat(conf.get(TEST_KEY_IDENTIFIER)) + .as("Identifier mistmatch").isEqualTo(identifier); + Token token = storeValue.getDtProvider() + .getFsDelegationToken(); + String tokenId = new String(token.getIdentifier(), + StandardCharsets.UTF_8); + Assertions.assertThat(tokenId) + .as("Mismatch in delegation token identifier").isEqualTo( + createTokenIdentifierString(identifier, bucketName, + UserGroupInformation.getCurrentUser().getShortUserName())); + } + } + + private void closeAndVerifyNull(Closeable closeable, String bucketName, + UserGroupInformation ugi, int expectedCount) + throws IOException, InterruptedException { + closeable.close(); + attemptSignAndVerify("dontcare", bucketName, ugi, true); + Assertions.assertThat(SignerInitializerForTest.storeCache.size()) + .as("StoreCache size mismatch").isEqualTo(expectedCount); + } + + /** + * SignerForTest1. + */ + @Private + public static class SignerForTest1 implements Signer { + + private static boolean initialized = false; + + @Override + public void sign(SignableRequest request, AWSCredentials credentials) { + initialized = true; + } + + public static void reset() { + initialized = false; + } + } + + /** + * SignerForTest2. + */ + @Private + public static class SignerForTest2 implements Signer { + + private static boolean initialized = false; + + @Override + public void sign(SignableRequest request, AWSCredentials credentials) { + initialized = true; + } + + public static void reset() { + initialized = false; + } + } + + /** + * SignerInitializerForTest. + */ + @Private + public static class SignerInitializerForTest implements AwsSignerInitializer { + + private static int registerCount = 0; + private static int unregisterCount = 0; + private static int instanceCount = 0; + + private static final Map storeCache = new HashMap<>(); + + public SignerInitializerForTest() { + instanceCount++; + } + + @Override + public void registerStore(String bucketName, Configuration storeConf, + DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) { + registerCount++; + StoreKey storeKey = new StoreKey(bucketName, storeUgi); + StoreValue storeValue = new StoreValue(bucketName, storeConf, dtProvider); + storeCache.put(storeKey, storeValue); + } + + @Override + public void unregisterStore(String bucketName, Configuration storeConf, + DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) { + unregisterCount++; + StoreKey storeKey = new StoreKey(bucketName, storeUgi); + storeCache.remove(storeKey); + } + + public static void reset() { + registerCount = 0; + unregisterCount = 0; + instanceCount = 0; + storeCache.clear(); + } + + public static StoreValue getStoreInfo(String bucketName, + UserGroupInformation storeUgi) { + StoreKey storeKey = new StoreKey(bucketName, storeUgi); + return storeCache.get(storeKey); + } + + private static class StoreKey { + private final String bucketName; + private final UserGroupInformation ugi; + + public StoreKey(String bucketName, UserGroupInformation ugi) { + this.bucketName = bucketName; + this.ugi = ugi; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StoreKey storeKey = (StoreKey) o; + return Objects.equals(bucketName, storeKey.bucketName) && Objects + .equals(ugi, storeKey.ugi); + } + + @Override + public int hashCode() { + return Objects.hash(bucketName, ugi); + } + } + + static class StoreValue { + private final String bucketName; + private final Configuration storeConf; + private final DelegationTokenProvider dtProvider; + + public StoreValue(String bucketName, Configuration storeConf, + DelegationTokenProvider dtProvider) { + this.bucketName = bucketName; + this.storeConf = storeConf; + this.dtProvider = dtProvider; + } + + String getBucketName() { + return bucketName; + } + + Configuration getStoreConf() { + return storeConf; + } + + DelegationTokenProvider getDtProvider() { + return dtProvider; + } + } + } + + /** + * To be used in conjunction with {@link SignerInitializerForTest}. + */ + @Private + public static class SignerForInitializerTest implements Signer { + + private static StoreValue retrievedStoreValue; + + @Override + public void sign(SignableRequest request, AWSCredentials credentials) { + String bucketName = request.getEndpoint().getHost(); + try { + retrievedStoreValue = SignerInitializerForTest + .getStoreInfo(bucketName, UserGroupInformation.getCurrentUser()); + } catch (IOException e) { + throw new RuntimeException("Failed to get current ugi", e); + } + } + + public static void reset() { + retrievedStoreValue = null; + } + } + + /** + * DelegationTokenProviderForTest. + */ + @Private + private static class DelegationTokenProviderForTest + implements DelegationTokenProvider { + + private final Token token; + + private DelegationTokenProviderForTest( + Token token) { + this.token = token; + } + + @Override + public Token getFsDelegationToken() + throws IOException { + return this.token; + } + } + + /** + * SignerInitializer2ForTest. + */ + @Private + public static class SignerInitializer2ForTest + implements AwsSignerInitializer { + + private static int registerCount = 0; + private static int unregisterCount = 0; + private static int instanceCount = 0; + + public SignerInitializer2ForTest() { + instanceCount++; + } + + @Override + public void registerStore(String bucketName, Configuration storeConf, + DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) { + registerCount++; + } + + @Override + public void unregisterStore(String bucketName, Configuration storeConf, + DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) { + unregisterCount++; + } + + public static void reset() { + registerCount = 0; + unregisterCount = 0; + instanceCount = 0; + } + } + + private Token createTokenForTest(String idString) { + byte[] identifier = idString.getBytes(StandardCharsets.UTF_8); + byte[] password = "notapassword".getBytes(StandardCharsets.UTF_8); + Token token = new Token<>(identifier, password, + TEST_TOKEN_KIND, TEST_TOKEN_SERVICE); + return token; + } + + private SignerManager fakeS3AInstanceCreation(String identifier, + Class signerClazz, + Class signerInitializerClazz, + String bucketName, UserGroupInformation ugi) { + // Simulate new S3A instance interactions. + Objects.requireNonNull(signerClazz, "SignerClazz missing"); + Objects.requireNonNull(signerInitializerClazz, + "SignerInitializerClazzMissing"); + Configuration config = new Configuration(); + config.set(TEST_KEY_IDENTIFIER, identifier); + config.set(CUSTOM_SIGNERS, + signerClazz.getCanonicalName() + ":" + signerClazz.getName() + ":" + + signerInitializerClazz.getName()); + Token token1 = createTokenForTest( + createTokenIdentifierString(identifier, bucketName, + ugi.getShortUserName())); + DelegationTokenProvider dtProvider1 = new DelegationTokenProviderForTest( + token1); + SignerManager signerManager = new SignerManager(bucketName, dtProvider1, + config, ugi); + signerManager.initCustomSigners(); + return signerManager; + } + + private String createTokenIdentifierString(String identifier, + String bucketName, String user) { + return identifier + "_" + bucketName + "_" + user; + } + + private SignableRequest constructSignableRequest(String bucketName) + throws URISyntaxException { + DefaultRequest signableRequest = new DefaultRequest( + AmazonWebServiceRequest.NOOP, "fakeservice"); + URI uri = new URI("s3://" + bucketName + "/"); + signableRequest.setEndpoint(uri); + return signableRequest; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java index 6023daa13875b..1cf3fb4a3f65f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java @@ -24,6 +24,7 @@ import java.util.stream.Collectors; import com.amazonaws.services.s3.AmazonS3; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -111,7 +112,9 @@ protected Configuration createConfiguration() { MAGIC_COMMITTER_ENABLED, S3A_COMMITTER_FACTORY_KEY, FS_S3A_COMMITTER_NAME, - FS_S3A_COMMITTER_STAGING_CONFLICT_MODE); + FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, + FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES, + FAST_UPLOAD_BUFFER); conf.setBoolean(MAGIC_COMMITTER_ENABLED, true); conf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE); @@ -209,6 +212,7 @@ public static String randomJobId() throws Exception { */ @Override public void teardown() throws Exception { + Thread.currentThread().setName("teardown"); LOG.info("AbstractCommitITest::teardown"); waitForConsistency(); // make sure there are no failures any more @@ -359,7 +363,7 @@ private String pathToPrefix(Path path) { * @throws IOException IO Failure */ protected SuccessData verifySuccessMarker(Path dir) throws IOException { - return validateSuccessFile(dir, "", getFileSystem(), "query"); + return validateSuccessFile(dir, "", getFileSystem(), "query", 0); } /** @@ -437,13 +441,15 @@ public static TaskAttemptContext taskAttemptForJob(JobId jobId, * @param committerName name of committer to match, or "" * @param fs filesystem * @param origin origin (e.g. "teragen" for messages) + * @param minimumFileCount minimum number of files to have been created * @return the success data * @throws IOException IO failure */ public static SuccessData validateSuccessFile(final Path outputPath, final String committerName, final S3AFileSystem fs, - final String origin) throws IOException { + final String origin, + final int minimumFileCount) throws IOException { SuccessData successData = loadSuccessFile(fs, outputPath, origin); String commitDetails = successData.toString(); LOG.info("Committer name " + committerName + "\n{}", @@ -456,6 +462,9 @@ public static SuccessData validateSuccessFile(final Path outputPath, assertEquals("Wrong committer in " + commitDetails, committerName, successData.getCommitter()); } + Assertions.assertThat(successData.getFilenames()) + .describedAs("Files committed") + .hasSizeGreaterThanOrEqualTo(minimumFileCount); return successData; } @@ -485,8 +494,9 @@ public static SuccessData loadSuccessFile(final S3AFileSystem fs, status.isFile()); assertTrue("0 byte success file " + success + " from " + origin - + "; a s3guard committer was not used", + + "; an S3A committer was not used", status.getLen() > 0); + LOG.info("Loading committer success file {}", success); return SuccessData.load(fs, success); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java deleted file mode 100644 index 1a518474bcb03..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java +++ /dev/null @@ -1,223 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.commit; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.UUID; - -import com.google.common.collect.Sets; -import org.assertj.core.api.Assertions; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3AUtils; -import org.apache.hadoop.fs.s3a.commit.files.SuccessData; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.Mapper; -import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.hadoop.util.DurationInfo; - -import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; -import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_STAGING_UUID; -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_TMP_PATH; - -/** - * Test for an MR Job with all the different committers. - */ -public abstract class AbstractITCommitMRJob extends AbstractYarnClusterITest { - - private static final Logger LOG = - LoggerFactory.getLogger(AbstractITCommitMRJob.class); - - @Override - protected Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - disableFilesystemCaching(conf); - return conf; - } - - @Rule - public final TemporaryFolder temp = new TemporaryFolder(); - - @Test - public void testMRJob() throws Exception { - describe("Run a simple MR Job"); - - S3AFileSystem fs = getFileSystem(); - // final dest is in S3A - Path outputPath = path(getMethodName()); - // create and delete to force in a tombstone marker -see HADOOP-16207 - fs.mkdirs(outputPath); - fs.delete(outputPath, true); - - String commitUUID = UUID.randomUUID().toString(); - String suffix = isUniqueFilenames() ? ("-" + commitUUID) : ""; - int numFiles = getTestFileCount(); - List expectedFiles = new ArrayList<>(numFiles); - Set expectedKeys = Sets.newHashSet(); - for (int i = 0; i < numFiles; i += 1) { - File file = temp.newFile(i + ".text"); - try (FileOutputStream out = new FileOutputStream(file)) { - out.write(("file " + i).getBytes(StandardCharsets.UTF_8)); - } - String filename = String.format("part-m-%05d%s", i, suffix); - Path path = new Path(outputPath, filename); - expectedFiles.add(path.toString()); - expectedKeys.add("/" + fs.pathToKey(path)); - } - Collections.sort(expectedFiles); - - Job mrJob = createJob(); - JobConf jobConf = (JobConf) mrJob.getConfiguration(); - - mrJob.setOutputFormatClass(LoggingTextOutputFormat.class); - FileOutputFormat.setOutputPath(mrJob, outputPath); - - File mockResultsFile = temp.newFile("committer.bin"); - mockResultsFile.delete(); - String committerPath = "file:" + mockResultsFile; - jobConf.set("mock-results-file", committerPath); - jobConf.set(FS_S3A_COMMITTER_STAGING_UUID, commitUUID); - jobConf.set(FS_S3A_COMMITTER_STAGING_TMP_PATH, "/staging"); - - mrJob.setInputFormatClass(TextInputFormat.class); - FileInputFormat.addInputPath(mrJob, new Path(temp.getRoot().toURI())); - - mrJob.setMapperClass(MapClass.class); - mrJob.setNumReduceTasks(0); - - // an attempt to set up log4j properly, which clearly doesn't work - URL log4j = getClass().getClassLoader().getResource("log4j.properties"); - if (log4j != null && log4j.getProtocol().equals("file")) { - Path log4jPath = new Path(log4j.toURI()); - LOG.debug("Using log4j path {}", log4jPath); - mrJob.addFileToClassPath(log4jPath); - String sysprops = String.format("-Xmx256m -Dlog4j.configuration=%s", - log4j); - jobConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, sysprops); - jobConf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, sysprops); - jobConf.set("yarn.app.mapreduce.am.command-opts", sysprops); - } - - applyCustomConfigOptions(jobConf); - // fail fast if anything goes wrong - mrJob.setMaxMapAttempts(1); - - mrJob.submit(); - try (DurationInfo ignore = new DurationInfo(LOG, "Job Execution")) { - boolean succeeded = mrJob.waitForCompletion(true); - assertTrue("MR job failed", succeeded); - } - - waitForConsistency(); - verifyPathExists(fs, - "MR job Output directory not found," - + " even though the job did not report a failure", - outputPath); - assertIsDirectory(outputPath); - FileStatus[] results = fs.listStatus(outputPath, - S3AUtils.HIDDEN_FILE_FILTER); - int fileCount = results.length; - List actualFiles = new ArrayList<>(fileCount); - assertTrue("No files in output directory", fileCount != 0); - LOG.info("Found {} files", fileCount); - for (FileStatus result : results) { - LOG.debug("result: {}", result); - actualFiles.add(result.getPath().toString()); - } - Collections.sort(actualFiles); - - SuccessData successData = validateSuccessFile(outputPath, committerName(), - fs, "MR job"); - List successFiles = successData.getFilenames(); - String commitData = successData.toString(); - assertFalse("No filenames in " + commitData, - successFiles.isEmpty()); - - Assertions.assertThat(actualFiles) - .describedAs("Committed files in the job output directory") - .containsExactlyInAnyOrderElementsOf(expectedFiles); - - Assertions.assertThat(successFiles) - .describedAs("List of committed files in %s", commitData) - .containsExactlyInAnyOrderElementsOf(expectedKeys); - - assertPathDoesNotExist("temporary dir", - new Path(outputPath, CommitConstants.TEMPORARY)); - customPostExecutionValidation(outputPath, successData); - } - - /** - * Test Mapper. - * This is executed in separate process, and must not make any assumptions - * about external state. - */ - public static class MapClass - extends Mapper { - - private int operations; - private String id = ""; - private LongWritable l = new LongWritable(); - private Text t = new Text(); - - @Override - protected void setup(Context context) - throws IOException, InterruptedException { - super.setup(context); - // force in Log4J logging - org.apache.log4j.BasicConfigurator.configure(); - boolean scaleMap = context.getConfiguration() - .getBoolean(KEY_SCALE_TESTS_ENABLED, false); - operations = scaleMap ? SCALE_TEST_KEYS : BASE_TEST_KEYS; - id = context.getTaskAttemptID().toString(); - } - - @Override - protected void map(LongWritable key, Text value, Context context) - throws IOException, InterruptedException { - for (int i = 0; i < operations; i++) { - l.set(i); - t.set(String.format("%s:%05d", id, i)); - context.write(l, t); - } - } - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java index 822e3617f1b0c..cacd54d12e90c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java @@ -25,7 +25,10 @@ import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -178,6 +181,20 @@ public void teardown() throws Exception { super.teardown(); } + /** + * This only looks for leakage of committer thread pools, + * and not any other leaked threads, such as those from S3A FS instances. + */ + @AfterClass + public static void checkForThreadLeakage() { + List committerThreads = getCurrentThreadNames().stream() + .filter(n -> n.startsWith(AbstractS3ACommitter.THREAD_PREFIX)) + .collect(Collectors.toList()); + Assertions.assertThat(committerThreads) + .describedAs("Outstanding committer threads") + .isEmpty(); + } + /** * Add the specified job to the current list of jobs to abort in teardown. * @param jobData job data. @@ -518,6 +535,7 @@ protected void commit(AbstractS3ACommitter committer, describe("\ncommitting job"); committer.commitJob(jContext); describe("commit complete\n"); + verifyCommitterHasNoThreads(committer); } } @@ -574,7 +592,7 @@ public void testRecoveryAndCleanup() throws Exception { // Commit the task. This will promote data and metadata to where // job commits will pick it up on commit or abort. - committer.commitTask(tContext); + commitTask(committer, tContext); assertTaskAttemptPathDoesNotExist(committer, tContext); Configuration conf2 = jobData.job.getConfiguration(); @@ -600,6 +618,7 @@ public void testRecoveryAndCleanup() throws Exception { committer2.abortJob(jContext2, JobStatus.State.KILLED); // now, state of system may still have pending data assertNoMultipartUploadsPending(outDir); + verifyCommitterHasNoThreads(committer2); } protected void assertTaskAttemptPathDoesNotExist( @@ -742,7 +761,7 @@ public void testCommitLifecycle() throws Exception { describe("2. Committing task"); assertTrue("No files to commit were found by " + committer, committer.needsTaskCommit(tContext)); - committer.commitTask(tContext); + commitTask(committer, tContext); // this is only task commit; there MUST be no part- files in the dest dir waitForConsistency(); @@ -758,7 +777,7 @@ public void testCommitLifecycle() throws Exception { describe("3. Committing job"); assertMultipartUploadsPending(outDir); - committer.commitJob(jContext); + commitJob(committer, jContext); // validate output describe("4. Validating content"); @@ -809,7 +828,7 @@ public void testCommitterWithFailure() throws Exception { // now fail job expectSimulatedFailureOnJobCommit(jContext, committer); - committer.commitJob(jContext); + commitJob(committer, jContext); // but the data got there, due to the order of operations. validateContent(outDir, shouldExpectSuccessMarker()); @@ -1011,6 +1030,7 @@ public void testAbortTaskThenJob() throws Exception { committer.abortJob(jobData.jContext, JobStatus.State.FAILED); assertJobAbortCleanedUp(jobData); + verifyCommitterHasNoThreads(committer); } /** @@ -1064,6 +1084,7 @@ public void testFailAbort() throws Exception { // try again; expect abort to be idempotent. committer.abortJob(jContext, JobStatus.State.FAILED); assertNoMultipartUploadsPending(outDir); + verifyCommitterHasNoThreads(committer); } public void assertPart0000DoesNotExist(Path dir) throws Exception { @@ -1223,8 +1244,8 @@ public void testOutputFormatIntegration() throws Throwable { validateTaskAttemptPathAfterWrite(dest); assertTrue("Committer does not have data to commit " + committer, committer.needsTaskCommit(tContext)); - committer.commitTask(tContext); - committer.commitJob(jContext); + commitTask(committer, tContext); + commitJob(committer, jContext); // validate output verifySuccessMarker(outDir); } @@ -1257,6 +1278,7 @@ public void testAMWorkflow() throws Throwable { AbstractS3ACommitter committer2 = (AbstractS3ACommitter) outputFormat.getOutputCommitter(newAttempt); committer2.abortTask(tContext); + verifyCommitterHasNoThreads(committer2); assertNoMultipartUploadsPending(getOutDir()); } @@ -1306,19 +1328,19 @@ public void testParallelJobsToAdjacentPaths() throws Throwable { // at this point, job1 and job2 both have uncommitted tasks // commit tasks in order task 2, task 1. - committer2.commitTask(tContext2); - committer1.commitTask(tContext1); + commitTask(committer2, tContext2); + commitTask(committer1, tContext1); assertMultipartUploadsPending(job1Dest); assertMultipartUploadsPending(job2Dest); // commit jobs in order job 1, job 2 - committer1.commitJob(jContext1); + commitJob(committer1, jContext1); assertNoMultipartUploadsPending(job1Dest); getPart0000(job1Dest); assertMultipartUploadsPending(job2Dest); - committer2.commitJob(jContext2); + commitJob(committer2, jContext2); getPart0000(job2Dest); assertNoMultipartUploadsPending(job2Dest); } finally { @@ -1379,4 +1401,36 @@ protected void validateTaskAttemptWorkingDirectory( TaskAttemptContext context) throws IOException { } + /** + * Commit a task then validate the state of the committer afterwards. + * @param committer committer + * @param tContext task context + * @throws IOException IO failure + */ + protected void commitTask(final AbstractS3ACommitter committer, + final TaskAttemptContext tContext) throws IOException { + committer.commitTask(tContext); + verifyCommitterHasNoThreads(committer); + } + + /** + * Commit a job then validate the state of the committer afterwards. + * @param committer committer + * @param jContext job context + * @throws IOException IO failure + */ + protected void commitJob(final AbstractS3ACommitter committer, + final JobContext jContext) throws IOException { + committer.commitJob(jContext); + verifyCommitterHasNoThreads(committer); + } + + /** + * Verify that the committer does not have a thread pool. + * @param committer committer to validate. + */ + protected void verifyCommitterHasNoThreads(AbstractS3ACommitter committer) { + assertFalse("Committer has an active thread pool", + committer.hasThreadPool()); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractYarnClusterITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractYarnClusterITest.java index 2e8f1f090403c..783c62686bad7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractYarnClusterITest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractYarnClusterITest.java @@ -19,8 +19,12 @@ package org.apache.hadoop.fs.s3a.commit; import java.io.IOException; -import java.util.UUID; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import org.junit.AfterClass; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,19 +37,22 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster; import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; import static org.apache.hadoop.fs.s3a.S3ATestUtils.deployService; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.prepareTestConfiguration; import static org.apache.hadoop.fs.s3a.S3ATestUtils.terminateService; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_TMP_PATH; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES; /** * Full integration test MR jobs. * - * This is all done on shared static mini YARN and HDFS clusters, set up before - * any of the tests methods run. + * This is all done on shared static mini YARN and (optionally) HDFS clusters, + * set up before any of the tests methods run. * * To isolate tests properly for parallel test runs, that static state * needs to be stored in the final classes implementing the tests, and @@ -61,38 +68,54 @@ * If two subclasses of this class are instantiated in the same JVM, in order, * they are guaranteed to be isolated. * - * History: this is a superclass extracted from - * {@link AbstractITCommitMRJob} while adding support for testing terasorting. - * */ public abstract class AbstractYarnClusterITest extends AbstractCommitITest { private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnClusterITest.class); - private static final int TEST_FILE_COUNT = 2; - private static final int SCALE_TEST_FILE_COUNT = 50; + private static final int TEST_FILE_COUNT = 1; + private static final int SCALE_TEST_FILE_COUNT = 10; - public static final int SCALE_TEST_KEYS = 1000; + public static final int SCALE_TEST_KEYS = 100; public static final int BASE_TEST_KEYS = 10; + public static final int NO_OF_NODEMANAGERS = 2; + private boolean scaleTest; - private boolean uniqueFilenames = false; + /** + * The static cluster binding with the lifecycle of this test; served + * through instance-level methods for sharing across methods in the + * suite. + */ + @SuppressWarnings("StaticNonFinalField") + private static ClusterBinding clusterBinding; + + + @AfterClass + public static void teardownClusters() throws IOException { + terminateCluster(clusterBinding); + clusterBinding = null; + } /** * This is the cluster binding which every subclass must create. */ protected static final class ClusterBinding { + private String clusterName; + private final MiniDFSClusterService hdfs; private final MiniMRYarnCluster yarn; public ClusterBinding( + final String clusterName, final MiniDFSClusterService hdfs, final MiniMRYarnCluster yarn) { - this.hdfs = checkNotNull(hdfs); + this.clusterName = clusterName; + this.hdfs = hdfs; this.yarn = checkNotNull(yarn); } @@ -100,6 +123,18 @@ public MiniDFSClusterService getHdfs() { return hdfs; } + /** + * Get the cluster FS, which will either be HDFS or the local FS. + * @return a filesystem. + * @throws IOException failure + */ + public FileSystem getClusterFS() throws IOException { + MiniDFSClusterService miniCluster = getHdfs(); + return miniCluster != null + ? miniCluster.getClusterFS() + : FileSystem.getLocal(yarn.getConfig()); + } + public MiniMRYarnCluster getYarn() { return yarn; } @@ -108,6 +143,10 @@ public Configuration getConf() { return getYarn().getConfig(); } + public String getClusterName() { + return clusterName; + } + public void terminate() { terminateService(getYarn()); terminateService(getHdfs()); @@ -115,74 +154,111 @@ public void terminate() { } /** - * Create the cluster binding. This must be done in - * class setup of the (final) subclass. - * The HDFS and YARN clusters share the same configuration, so + * Create the cluster binding. + * The configuration will be patched by propagating down options + * from the maven build (S3Guard binding etc) and turning off unwanted + * YARN features. + * + * If an HDFS cluster is requested, + * the HDFS and YARN clusters will share the same configuration, so * the HDFS cluster binding is implicitly propagated to YARN. + * If one is not requested, the local filesystem is used as the cluster FS. * @param conf configuration to start with. + * @param useHDFS should an HDFS cluster be instantiated. * @return the cluster binding. * @throws IOException failure. */ - protected static ClusterBinding createCluster(JobConf conf) - throws IOException { - + protected static ClusterBinding createCluster( + final JobConf conf, + final boolean useHDFS) throws IOException { + prepareTestConfiguration(conf); conf.setBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, false); conf.setLong(CommonConfigurationKeys.FS_DU_INTERVAL_KEY, Long.MAX_VALUE); - - // create a unique cluster name. - String clusterName = "yarn-" + UUID.randomUUID(); - MiniDFSClusterService miniDFSClusterService = deployService(conf, - new MiniDFSClusterService()); + // minicluster tests overreact to not enough disk space. + conf.setBoolean(YarnConfiguration.NM_DISK_HEALTH_CHECK_ENABLE, false); + conf.setInt(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 100); + // create a unique cluster name based on the current time in millis. + String timestamp = LocalDateTime.now().format( + DateTimeFormatter.ofPattern("yyyy-MM-dd-HH.mm.ss.SS")); + String clusterName = "yarn-" + timestamp; + MiniDFSClusterService miniDFSClusterService = + useHDFS + ? deployService(conf, new MiniDFSClusterService()) + : null; MiniMRYarnCluster yarnCluster = deployService(conf, - new MiniMRYarnCluster(clusterName, 2)); - return new ClusterBinding(miniDFSClusterService, yarnCluster); + new MiniMRYarnCluster(clusterName, NO_OF_NODEMANAGERS)); + return new ClusterBinding(clusterName, miniDFSClusterService, yarnCluster); } - protected static void terminateCluster(ClusterBinding clusterBinding) { - if (clusterBinding != null) { - clusterBinding.terminate(); + /** + * Terminate the cluster if it is not null. + * @param cluster the cluster + */ + protected static void terminateCluster(ClusterBinding cluster) { + if (cluster != null) { + cluster.terminate(); } } /** - * Get the cluster binding for this subclass - * @return + * Get the cluster binding for this subclass. + * @return the cluster binding */ - protected abstract ClusterBinding getClusterBinding(); - - protected MiniDFSClusterService getHdfs() { - return getClusterBinding().getHdfs(); + protected ClusterBinding getClusterBinding() { + return clusterBinding; } - protected MiniMRYarnCluster getYarn() { return getClusterBinding().getYarn(); } - public FileSystem getLocalFS() { - return getHdfs().getLocalFS(); + /** + * Get the cluster filesystem -hdfs or local. + * @return the filesystem shared across the yarn nodes. + */ + protected FileSystem getClusterFS() throws IOException { + return getClusterBinding().getClusterFS(); } - protected FileSystem getDFS() { - return getHdfs().getClusterFS(); - } + + /** + * We stage work into a temporary directory rather than directly under + * the user's home directory, as that is often rejected by CI test + * runners. + */ + @Rule + public final TemporaryFolder stagingFilesDir = new TemporaryFolder(); /** * The name of the committer as returned by - * {@link AbstractS3ACommitter#getName()} and used for committer construction. + * {@link AbstractS3ACommitter#getName()} + * and used for committer construction. */ protected abstract String committerName(); + /** + * binding on demand rather than in a BeforeClass static method. + * Subclasses can override this to change the binding options. + * @return the cluster binding + */ + protected ClusterBinding demandCreateClusterBinding() throws Exception { + return createCluster(new JobConf(), false); + } + @Override public void setup() throws Exception { super.setup(); - assertNotNull("cluster is not bound", - getClusterBinding()); scaleTest = getTestPropertyBool( getConfiguration(), KEY_SCALE_TESTS_ENABLED, DEFAULT_SCALE_TESTS_ENABLED); + if (getClusterBinding() == null) { + clusterBinding = demandCreateClusterBinding(); + } + assertNotNull("cluster is not bound", + getClusterBinding()); + } @Override @@ -190,28 +266,46 @@ protected int getTestTimeoutMillis() { return SCALE_TEST_TIMEOUT_SECONDS * 1000; } - protected JobConf newJobConf() { - return new JobConf(getYarn().getConfig()); + /** + * Create a job configuration. + * This creates a new job conf from the yarn + * cluster configuration then calls + * {@link #applyCustomConfigOptions(JobConf)} to allow it to be customized. + * @return the new job configuration. + * @throws IOException failure + */ + protected JobConf newJobConf() throws IOException { + JobConf jobConf = new JobConf(getYarn().getConfig()); + jobConf.addResource(getConfiguration()); + applyCustomConfigOptions(jobConf); + return jobConf; } - protected Job createJob() throws IOException { - Configuration jobConf = getClusterBinding().getConf(); - jobConf.addResource(getConfiguration()); + protected Job createJob(Configuration jobConf) throws IOException { Job mrJob = Job.getInstance(jobConf, getMethodName()); patchConfigurationForCommitter(mrJob.getConfiguration()); return mrJob; } + /** + * Patch the (job) configuration for this committer. + * @param jobConf configuration to patch + * @return a configuration which will run this configuration. + */ protected Configuration patchConfigurationForCommitter( final Configuration jobConf) { jobConf.setBoolean(FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES, - uniqueFilenames); + isUniqueFilenames()); bindCommitter(jobConf, CommitConstants.S3A_COMMITTER_FACTORY, committerName()); // pass down the scale test flag - jobConf.setBoolean(KEY_SCALE_TESTS_ENABLED, scaleTest); + jobConf.setBoolean(KEY_SCALE_TESTS_ENABLED, isScaleTest()); + // and fix the commit dir to the local FS across all workers. + String staging = stagingFilesDir.getRoot().getAbsolutePath(); + LOG.info("Staging temp dir is {}", staging); + jobConf.set(FS_S3A_COMMITTER_STAGING_TMP_PATH, staging); return jobConf; } @@ -220,7 +314,7 @@ protected Configuration patchConfigurationForCommitter( * @return the number of mappers to create. */ public int getTestFileCount() { - return scaleTest ? SCALE_TEST_FILE_COUNT : TEST_FILE_COUNT; + return isScaleTest() ? SCALE_TEST_FILE_COUNT : TEST_FILE_COUNT; } /** @@ -258,6 +352,6 @@ public boolean isScaleTest() { } public boolean isUniqueFilenames() { - return uniqueFilenames; + return false; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java index 5e6fb82362cdc..4ee39f1bfa08e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java @@ -73,7 +73,7 @@ public class TestTasks extends HadoopTestBase { * more checks on single thread than parallel ops. * @return a list of parameter tuples. */ - @Parameterized.Parameters + @Parameterized.Parameters(name = "threads={0}") public static Collection params() { return Arrays.asList(new Object[][]{ {0}, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java new file mode 100644 index 0000000000000..1045a2929c097 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java @@ -0,0 +1,644 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.commit.integration; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; + +import com.google.common.collect.Sets; +import org.assertj.core.api.Assertions; +import org.junit.FixMethodOrder; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.MethodSorters; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.s3a.commit.AbstractYarnClusterITest; +import org.apache.hadoop.fs.s3a.commit.CommitConstants; +import org.apache.hadoop.fs.s3a.commit.LoggingTextOutputFormat; +import org.apache.hadoop.fs.s3a.commit.files.SuccessData; +import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter; +import org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitter; +import org.apache.hadoop.fs.s3a.commit.staging.PartitionedStagingCommitter; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.DurationInfo; + +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.lsR; +import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_TMP_PATH; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants._SUCCESS; +import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_STAGING_UUID; +import static org.apache.hadoop.fs.s3a.commit.staging.Paths.getMultipartUploadCommitsDirectory; +import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.STAGING_UPLOADS; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Test an MR Job with all the different committers. + *

        + * This is a fairly complex parameterization: it is designed to + * avoid the overhead of starting a Yarn cluster for + * individual committer types, so speed up operations. + *

        + * It also implicitly guarantees that there is never more than one of these + * MR jobs active at a time, so avoids overloading the test machine with too + * many processes. + * How the binding works: + *

          + *
        1. + * Each parameterized suite is configured through its own + * {@link CommitterTestBinding} subclass. + *
        2. + *
        3. + * JUnit runs these test suites one parameterized binding at a time. + *
        4. + *
        5. + * The test suites are declared to be executed in ascending order, so + * that for a specific binding, the order is {@link #test_000()}, + * {@link #test_100()} {@link #test_200_execute()} and finally + * {@link #test_500()}. + *
        6. + *
        7. + * {@link #test_000()} calls {@link CommitterTestBinding#validate()} to + * as to validate the state of the committer. This is primarily to + * verify that the binding setup mechanism is working. + *
        8. + *
        9. + * {@link #test_100()} is relayed to + * {@link CommitterTestBinding#test_100()}, + * for any preflight tests. + *
        10. + *
        11. + * The {@link #test_200_execute()} test runs the MR job for that + * particular binding with standard reporting and verification of the + * outcome. + *
        12. + *
        13. + * {@link #test_500()} test is relayed to + * {@link CommitterTestBinding#test_500()}, for any post-MR-job tests. + *
        + * + * A new S3A FileSystem instance is created for each test_ method, so the + * pre-execute and post-execute validators cannot inspect the state of the + * FS as part of their tests. + * However, as the MR workers and AM all run in their own processes, there's + * generally no useful information about the job in the local S3AFileSystem + * instance. + */ +@RunWith(Parameterized.class) +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class ITestS3ACommitterMRJob extends AbstractYarnClusterITest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestS3ACommitterMRJob.class); + + /** + * Test array for parameterized test runs. + * + * @return the committer binding for this run. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {new DirectoryCommitterTestBinding()}, + {new PartitionCommitterTestBinding()}, + {new MagicCommitterTestBinding()}, + }); + } + + /** + * The committer binding for this instance. + */ + private final CommitterTestBinding committerTestBinding; + + /** + * Parameterized constructor. + * @param committerTestBinding binding for the test. + */ + public ITestS3ACommitterMRJob( + final CommitterTestBinding committerTestBinding) { + this.committerTestBinding = committerTestBinding; + } + + @Override + public void setup() throws Exception { + super.setup(); + // configure the test binding for this specific test case. + committerTestBinding.setup(getClusterBinding(), getFileSystem()); + } + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + disableFilesystemCaching(conf); + return conf; + } + + @Rule + public final TemporaryFolder localFilesDir = new TemporaryFolder(); + + @Override + protected String committerName() { + return committerTestBinding.getCommitterName(); + } + + @Override + public boolean useInconsistentClient() { + return committerTestBinding.useInconsistentClient(); + } + + /** + * Verify that the committer binding is happy. + */ + @Test + public void test_000() throws Throwable { + committerTestBinding.validate(); + + } + @Test + public void test_100() throws Throwable { + committerTestBinding.test_100(); + } + + @Test + public void test_200_execute() throws Exception { + describe("Run an MR with committer %s", committerName()); + + S3AFileSystem fs = getFileSystem(); + // final dest is in S3A + // we can't use the method name as the template places square braces into + // that and URI creation fails. + + Path outputPath = path("ITestS3ACommitterMRJob-execute-"+ committerName()); + // create and delete to force in a tombstone marker -see HADOOP-16207 + fs.mkdirs(outputPath); + fs.delete(outputPath, true); + + String commitUUID = UUID.randomUUID().toString(); + String suffix = isUniqueFilenames() ? ("-" + commitUUID) : ""; + int numFiles = getTestFileCount(); + + // create all the input files on the local FS. + List expectedFiles = new ArrayList<>(numFiles); + Set expectedKeys = Sets.newHashSet(); + for (int i = 0; i < numFiles; i += 1) { + File file = localFilesDir.newFile(i + ".text"); + try (FileOutputStream out = new FileOutputStream(file)) { + out.write(("file " + i).getBytes(StandardCharsets.UTF_8)); + } + String filename = String.format("part-m-%05d%s", i, suffix); + Path path = new Path(outputPath, filename); + expectedFiles.add(path.toString()); + expectedKeys.add("/" + fs.pathToKey(path)); + } + Collections.sort(expectedFiles); + + Job mrJob = createJob(newJobConf()); + JobConf jobConf = (JobConf) mrJob.getConfiguration(); + + mrJob.setOutputFormatClass(LoggingTextOutputFormat.class); + FileOutputFormat.setOutputPath(mrJob, outputPath); + + File mockResultsFile = localFilesDir.newFile("committer.bin"); + mockResultsFile.delete(); + String committerPath = "file:" + mockResultsFile; + jobConf.set("mock-results-file", committerPath); + + // setting up staging options is harmless for other committers + jobConf.set(FS_S3A_COMMITTER_STAGING_UUID, commitUUID); + + mrJob.setInputFormatClass(TextInputFormat.class); + FileInputFormat.addInputPath(mrJob, + new Path(localFilesDir.getRoot().toURI())); + + mrJob.setMapperClass(MapClass.class); + mrJob.setNumReduceTasks(0); + + // an attempt to set up log4j properly, which clearly doesn't work + URL log4j = getClass().getClassLoader().getResource("log4j.properties"); + if (log4j != null && "file".equals(log4j.getProtocol())) { + Path log4jPath = new Path(log4j.toURI()); + LOG.debug("Using log4j path {}", log4jPath); + mrJob.addFileToClassPath(log4jPath); + String sysprops = String.format("-Xmx128m -Dlog4j.configuration=%s", + log4j); + jobConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, sysprops); + jobConf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, sysprops); + jobConf.set("yarn.app.mapreduce.am.command-opts", sysprops); + } + + applyCustomConfigOptions(jobConf); + // fail fast if anything goes wrong + mrJob.setMaxMapAttempts(1); + + try (DurationInfo ignore = new DurationInfo(LOG, "Job Submit")) { + mrJob.submit(); + } + String jobID = mrJob.getJobID().toString(); + String logLocation = "logs under " + + getYarn().getTestWorkDir().getAbsolutePath(); + try (DurationInfo ignore = new DurationInfo(LOG, "Job Execution")) { + mrJob.waitForCompletion(true); + } + JobStatus status = mrJob.getStatus(); + if (!mrJob.isSuccessful()) { + // failure of job. + // be as meaningful as possible. + String message = + String.format("Job %s failed in state %s with cause %s.\n" + + "Consult %s", + jobID, + status.getState(), + status.getFailureInfo(), + logLocation); + LOG.error(message); + fail(message); + } + + waitForConsistency(); + Path successPath = new Path(outputPath, _SUCCESS); + SuccessData successData = validateSuccessFile(outputPath, + committerName(), + fs, + "MR job " + jobID, + 1); + String commitData = successData.toString(); + + FileStatus[] results = fs.listStatus(outputPath, + S3AUtils.HIDDEN_FILE_FILTER); + int fileCount = results.length; + Assertions.assertThat(fileCount) + .describedAs("No files from job %s in output directory %s; see %s", + jobID, + outputPath, + logLocation) + .isNotEqualTo(0); + + List actualFiles = Arrays.stream(results) + .map(s -> s.getPath().toString()) + .sorted() + .collect(Collectors.toList()); + + Assertions.assertThat(actualFiles) + .describedAs("Files found in %s", outputPath) + .isEqualTo(expectedFiles); + + Assertions.assertThat(successData.getFilenames()) + .describedAs("Success files listed in %s:%s", + successPath, commitData) + .isNotEmpty() + .containsExactlyInAnyOrderElementsOf(expectedKeys); + + assertPathDoesNotExist("temporary dir should only be from" + + " classic file committers", + new Path(outputPath, CommitConstants.TEMPORARY)); + customPostExecutionValidation(outputPath, successData); + } + + @Override + protected void applyCustomConfigOptions(final JobConf jobConf) + throws IOException { + committerTestBinding.applyCustomConfigOptions(jobConf); + } + + @Override + protected void customPostExecutionValidation(final Path destPath, + final SuccessData successData) throws Exception { + committerTestBinding.validateResult(destPath, successData); + } + + /** + * This is the extra test which committer test bindings can add. + */ + @Test + public void test_500() throws Throwable { + committerTestBinding.test_500(); + } + + /** + * Test Mapper. + * This is executed in separate process, and must not make any assumptions + * about external state. + */ + public static class MapClass + extends Mapper { + + private int operations; + + private String id = ""; + + private LongWritable l = new LongWritable(); + + private Text t = new Text(); + + @Override + protected void setup(Context context) + throws IOException, InterruptedException { + super.setup(context); + // force in Log4J logging + org.apache.log4j.BasicConfigurator.configure(); + // and pick up scale test flag as passed down + boolean scaleMap = context.getConfiguration() + .getBoolean(KEY_SCALE_TESTS_ENABLED, false); + operations = scaleMap ? SCALE_TEST_KEYS : BASE_TEST_KEYS; + id = context.getTaskAttemptID().toString(); + } + + @Override + protected void map(LongWritable key, Text value, Context context) + throws IOException, InterruptedException { + for (int i = 0; i < operations; i++) { + l.set(i); + t.set(String.format("%s:%05d", id, i)); + context.write(l, t); + } + } + } + + /** + * A binding class for committer tests. + * Subclasses of this will be instantiated and drive the parameterized + * test suite. + * + * These classes will be instantiated in a static array of the suite, and + * not bound to a cluster binding or filesystem. + * + * The per-method test {@link #setup()} method will call + * {@link #setup(ClusterBinding, S3AFileSystem)}, to link the instance + * to the specific test cluster and test filesystem in use + * in that test. + */ + private abstract static class CommitterTestBinding { + + /** + * Name. + */ + private final String committerName; + + /** + * Cluster binding. + */ + private ClusterBinding binding; + + /** + * The S3A filesystem. + */ + private S3AFileSystem remoteFS; + + /** + * Constructor. + * @param committerName name of the committer for messages. + */ + protected CommitterTestBinding(final String committerName) { + this.committerName = committerName; + } + + /** + * Set up the test binding: this is called during test setup. + * @param cluster the active test cluster. + * @param fs the S3A Filesystem used as a destination. + */ + private void setup( + ClusterBinding cluster, + S3AFileSystem fs) { + this.binding = cluster; + this.remoteFS = fs; + } + + protected String getCommitterName() { + return committerName; + } + + protected ClusterBinding getBinding() { + return binding; + } + + protected S3AFileSystem getRemoteFS() { + return remoteFS; + } + + protected FileSystem getClusterFS() throws IOException { + return getBinding().getClusterFS(); + } + + @Override + public String toString() { + return committerName; + } + + /** + * Override point to let implementations tune the MR Job conf. + * @param jobConf configuration + */ + protected void applyCustomConfigOptions(JobConf jobConf) + throws IOException { + } + + /** + * Should the inconsistent S3A client be used? + * @return true for inconsistent listing + */ + public abstract boolean useInconsistentClient(); + + /** + * Override point for any committer specific validation operations; + * called after the base assertions have all passed. + * @param destPath destination of work + * @param successData loaded success data + * @throws Exception failure + */ + protected void validateResult(Path destPath, + SuccessData successData) + throws Exception { + + } + + /** + * A test to run before the main {@link #test_200_execute()} test is + * invoked. + * @throws Throwable failure. + */ + void test_100() throws Throwable { + + } + + /** + * A test to run after the main {@link #test_200_execute()} test is + * invoked. + * @throws Throwable failure. + */ + void test_500() throws Throwable { + + } + + /** + * Validate the state of the binding. + * This is called in {@link #test_000()} so will + * fail independently of the other tests. + * @throws Throwable failure. + */ + public void validate() throws Throwable { + assertNotNull("Not bound to a cluster", binding); + assertNotNull("No cluster filesystem", getClusterFS()); + assertNotNull("No yarn cluster", binding.getYarn()); + } + } + + /** + * The directory staging committer. + */ + private static final class DirectoryCommitterTestBinding + extends CommitterTestBinding { + + private DirectoryCommitterTestBinding() { + super(DirectoryStagingCommitter.NAME); + } + + /** + * @return true for inconsistent listing + */ + public boolean useInconsistentClient() { + return true; + } + + /** + * Verify that staging commit dirs are made absolute under the user's + * home directory, so, in a secure cluster, private. + */ + @Override + void test_100() throws Throwable { + FileSystem fs = getClusterFS(); + Configuration conf = fs.getConf(); + String pri = "private"; + conf.set(FS_S3A_COMMITTER_STAGING_TMP_PATH, pri); + Path dir = getMultipartUploadCommitsDirectory(conf, "uuid"); + Assertions.assertThat(dir.isAbsolute()) + .describedAs("non-absolute path") + .isTrue(); + String stagingTempDir = dir.toString().toLowerCase(Locale.ENGLISH); + String self = UserGroupInformation.getCurrentUser() + .getShortUserName().toLowerCase(Locale.ENGLISH); + Assertions.assertThat(stagingTempDir) + .describedAs("Staging committer temp path in cluster") + .contains(pri + "/" + self) + .endsWith("uuid/" + STAGING_UPLOADS); + } + } + + /** + * The partition committer test binding. + */ + private static final class PartitionCommitterTestBinding + extends CommitterTestBinding { + + private PartitionCommitterTestBinding() { + super(PartitionedStagingCommitter.NAME); + } + + /** + * @return true for inconsistent listing + */ + public boolean useInconsistentClient() { + return true; + } + } + + /** + * The magic committer test binding. + * This includes extra result validation. + */ + private static final class MagicCommitterTestBinding + extends CommitterTestBinding { + + private MagicCommitterTestBinding() { + super(MagicS3GuardCommitter.NAME); + } + + /** + * @return we need a consistent store. + */ + public boolean useInconsistentClient() { + return false; + } + + /** + * The result validation here is that there isn't a __magic directory + * any more. + * @param destPath destination of work + * @param successData loaded success data + * @throws Exception failure + */ + @Override + protected void validateResult(final Path destPath, + final SuccessData successData) + throws Exception { + Path magicDir = new Path(destPath, MAGIC); + + // if an FNFE isn't raised on getFileStatus, list out the directory + // tree + S3AFileSystem fs = getRemoteFS(); + // log the contents + lsR(fs, destPath, true); + intercept(FileNotFoundException.class, () -> { + final FileStatus st = fs.getFileStatus(magicDir); + StringBuilder result = new StringBuilder("Found magic dir which should" + + " have been deleted at ").append(st).append('\n'); + result.append(" ["); + applyLocatedFiles(fs.listFiles(magicDir, true), + (status) -> result.append(" ").append(status.getPath()).append('\n')); + result.append("]"); + return result.toString(); + }); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java deleted file mode 100644 index e403ab49b168e..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.commit.magic; - -import java.io.FileNotFoundException; -import java.io.IOException; - -import org.junit.AfterClass; -import org.junit.BeforeClass; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob; -import org.apache.hadoop.fs.s3a.commit.files.SuccessData; -import org.apache.hadoop.mapred.JobConf; - -import static org.apache.hadoop.fs.s3a.S3ATestUtils.lsR; -import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; - -/** - * Full integration test for the Magic Committer. - * - * There's no need to disable the committer setting for the filesystem here, - * because the committers are being instantiated in their own processes; - * the settings in {@link AbstractITCommitMRJob#applyCustomConfigOptions(JobConf)} are - * passed down to these processes. - */ -public final class ITestMagicCommitMRJob extends AbstractITCommitMRJob { - - /** - * The static cluster binding with the lifecycle of this test; served - * through instance-level methods for sharing across methods in the - * suite. - */ - @SuppressWarnings("StaticNonFinalField") - private static ClusterBinding clusterBinding; - - @BeforeClass - public static void setupClusters() throws IOException { - clusterBinding = createCluster(new JobConf()); - } - - @AfterClass - public static void teardownClusters() throws IOException { - terminateCluster(clusterBinding); - } - - @Override - public ClusterBinding getClusterBinding() { - return clusterBinding; - } - - /** - * Need consistency here. - * @return false - */ - @Override - public boolean useInconsistentClient() { - return false; - } - - @Override - protected String committerName() { - return MagicS3GuardCommitter.NAME; - } - - /** - * Turn on the magic commit support for the FS, else nothing will work. - * @param conf configuration - */ - @Override - protected void applyCustomConfigOptions(JobConf conf) { - conf.setBoolean(MAGIC_COMMITTER_ENABLED, true); - } - - /** - * Check that the magic dir was cleaned up. - * {@inheritDoc} - */ - @Override - protected void customPostExecutionValidation(Path destPath, - SuccessData successData) throws Exception { - Path magicDir = new Path(destPath, MAGIC); - - // if an FNFE isn't raised on getFileStatus, list out the directory - // tree - S3AFileSystem fs = getFileSystem(); - // log the contents - lsR(fs, destPath, true); - intercept(FileNotFoundException.class, () -> { - final FileStatus st = fs.getFileStatus(magicDir); - StringBuilder result = new StringBuilder("Found magic dir which should" - + " have been deleted at ").append(st).append('\n'); - result.append("["); - applyLocatedFiles(fs.listFiles(magicDir, true), - (status) -> result.append(status.getPath()).append('\n')); - result.append("["); - return result.toString(); - }); - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java index fd585d05b2e72..f368bf25c77c7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java @@ -251,6 +251,12 @@ public static void verifyExistenceChecked(FileSystem mockS3, Path path) verify(mockS3).getFileStatus(path); } + /** + * Verify that mkdirs was invoked once. + * @param mockS3 mock + * @param path path to check + * @throws IOException from the mkdirs signature. + */ public static void verifyMkdirsInvoked(FileSystem mockS3, Path path) throws IOException { verify(mockS3).mkdirs(path); @@ -320,12 +326,7 @@ public abstract static class JobCommitterTest @Before public void setupJob() throws Exception { - this.jobConf = new JobConf(); - jobConf.set(InternalCommitterConstants.FS_S3A_COMMITTER_STAGING_UUID, - UUID.randomUUID().toString()); - jobConf.setBoolean( - CommitConstants.CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, - false); + this.jobConf = createJobConf(); this.job = new JobContextImpl(jobConf, JOB_ID); this.results = new StagingTestBase.ClientResults(); @@ -338,6 +339,16 @@ public void setupJob() throws Exception { wrapperFS.setAmazonS3Client(mockClient); } + protected JobConf createJobConf() { + JobConf conf = new JobConf(); + conf.set(InternalCommitterConstants.FS_S3A_COMMITTER_STAGING_UUID, + UUID.randomUUID().toString()); + conf.setBoolean( + CommitConstants.CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, + false); + return conf; + } + public S3AFileSystem getMockS3A() { return mockFS; } @@ -461,6 +472,11 @@ public List getDeletes() { return deletes; } + public List getDeletePaths() { + return deletes.stream().map(DeleteObjectRequest::getKey).collect( + Collectors.toList()); + } + public void resetDeletes() { deletes.clear(); } @@ -478,6 +494,14 @@ public void resetRequests() { requests.clear(); } + public void addUpload(String id, String key) { + activeUploads.put(id, key); + } + + public void addUploads(Map uploadMap) { + activeUploads.putAll(uploadMap); + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( @@ -648,8 +672,9 @@ public static AmazonS3 newMockS3Client(final ClientResults results, } CompleteMultipartUploadRequest req = getArgumentAt(invocation, 0, CompleteMultipartUploadRequest.class); + String uploadId = req.getUploadId(); + removeUpload(results, uploadId); results.commits.add(req); - results.activeUploads.remove(req.getUploadId()); return newResult(req); } @@ -669,14 +694,7 @@ public static AmazonS3 newMockS3Client(final ClientResults results, AbortMultipartUploadRequest req = getArgumentAt(invocation, 0, AbortMultipartUploadRequest.class); String id = req.getUploadId(); - String p = results.activeUploads.remove(id); - if (p == null) { - // upload doesn't exist - AmazonS3Exception ex = new AmazonS3Exception( - "not found " + id); - ex.setStatusCode(404); - throw ex; - } + removeUpload(results, id); results.aborts.add(req); return null; } @@ -729,6 +747,24 @@ public static AmazonS3 newMockS3Client(final ClientResults results, return mockClient; } + /** + * Remove an upload from the upload map. + * @param results result set + * @param uploadId The upload ID to remove + * @throws AmazonS3Exception with error code 404 if the id is unknown. + */ + protected static void removeUpload(final ClientResults results, + final String uploadId) { + String removed = results.activeUploads.remove(uploadId); + if (removed == null) { + // upload doesn't exist + AmazonS3Exception ex = new AmazonS3Exception( + "not found " + uploadId); + ex.setStatusCode(404); + throw ex; + } + } + private static CompleteMultipartUploadResult newResult( CompleteMultipartUploadRequest req) { return new CompleteMultipartUploadResult(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java new file mode 100644 index 0000000000000..6d93e5fa788ff --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.commit.staging; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import com.amazonaws.services.s3.model.PartETag; +import com.google.common.collect.Maps; +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; +import org.apache.hadoop.fs.s3a.commit.CommitConstants; +import org.apache.hadoop.fs.s3a.commit.files.PendingSet; +import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.DurationInfo; + +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.CONFLICT_MODE_APPEND; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_CONFLICT_MODE; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.PENDINGSET_SUFFIX; +import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.BUCKET; +import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.outputPath; +import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.outputPathUri; +import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.pathIsDirectory; + +/** + * Scale test of the directory committer: if there are many, many files + * does job commit overload. + * This is a mock test as to avoid the overhead of going near S3; + * it does use a lot of local filesystem files though so as to + * simulate real large scale deployment better. + */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class TestDirectoryCommitterScale + extends StagingTestBase.JobCommitterTest { + + private static final Logger LOG = + LoggerFactory.getLogger(TestDirectoryCommitterScale.class); + + public static final int TASKS = 500; + + public static final int FILES_PER_TASK = 10; + + public static final int TOTAL_COMMIT_COUNT = FILES_PER_TASK * TASKS; + + public static final int BLOCKS_PER_TASK = 1000; + + private static File stagingDir; + + private static LocalFileSystem localFS; + + private static Path stagingPath; + + private static Map activeUploads = + Maps.newHashMap(); + + @Override + DirectoryCommitterForTesting newJobCommitter() throws Exception { + return new DirectoryCommitterForTesting(outputPath, + createTaskAttemptForJob()); + } + + @BeforeClass + public static void setupStaging() throws Exception { + stagingDir = File.createTempFile("staging", ""); + stagingDir.delete(); + stagingDir.mkdir(); + stagingPath = new Path(stagingDir.toURI()); + localFS = FileSystem.getLocal(new Configuration()); + } + + + @AfterClass + public static void teardownStaging() throws IOException { + try { + if (stagingDir != null) { + FileUtils.deleteDirectory(stagingDir); + } + } catch (IOException ignored) { + + } + } + + @Override + protected JobConf createJobConf() { + JobConf conf = super.createJobConf(); + conf.setInt( + CommitConstants.FS_S3A_COMMITTER_THREADS, + 100); + return conf; + } + + protected Configuration getJobConf() { + return getJob().getConfiguration(); + } + + @Test + public void test_010_createTaskFiles() throws Exception { + try (DurationInfo ignored = + new DurationInfo(LOG, "Creating %d test files in %s", + TOTAL_COMMIT_COUNT, stagingDir)) { + createTasks(); + } + } + + /** + * Create the mock uploads of the tasks and save + * to .pendingset files. + * @throws IOException failure. + */ + private void createTasks() throws IOException { + // create a stub multipart commit containing multiple files. + + // step1: a list of tags. + // this is the md5sum of hadoop 3.2.1.tar + String tag = "9062dcf18ffaee254821303bbd11c72b"; + List etags = IntStream.rangeClosed(1, BLOCKS_PER_TASK + 1) + .mapToObj(i -> new PartETag(i, tag)) + .collect(Collectors.toList()); + SinglePendingCommit base = new SinglePendingCommit(); + base.setBucket(BUCKET); + base.setJobId("0000"); + base.setLength(914688000); + base.bindCommitData(etags); + // these get overwritten + base.setDestinationKey("/base"); + base.setUploadId("uploadId"); + base.setUri(outputPathUri.toString()); + + SinglePendingCommit[] singles = new SinglePendingCommit[FILES_PER_TASK]; + byte[] bytes = base.toBytes(); + for (int i = 0; i < FILES_PER_TASK; i++) { + singles[i] = SinglePendingCommit.serializer().fromBytes(bytes); + } + // now create the files, using this as the template + + int uploadCount = 0; + for (int task = 0; task < TASKS; task++) { + PendingSet pending = new PendingSet(); + String taskId = String.format("task-%04d", task); + + for (int i = 0; i < FILES_PER_TASK; i++) { + String uploadId = String.format("%05d-task-%04d-file-%02d", + uploadCount, task, i); + // longer paths to take up more space. + Path p = new Path(outputPath, + "datasets/examples/testdirectoryscale/" + + "year=2019/month=09/day=26/hour=20/second=53" + + uploadId); + URI dest = p.toUri(); + SinglePendingCommit commit = singles[i]; + String key = dest.getPath(); + commit.setDestinationKey(key); + commit.setUri(dest.toString()); + commit.touch(Instant.now().toEpochMilli()); + commit.setTaskId(taskId); + commit.setUploadId(uploadId); + pending.add(commit); + activeUploads.put(uploadId, key); + } + Path path = new Path(stagingPath, + String.format("task-%04d." + PENDINGSET_SUFFIX, task)); + pending.save(localFS, path, true); + } + } + + + @Test + public void test_020_loadFilesToAttempt() throws Exception { + DirectoryStagingCommitter committer = newJobCommitter(); + + Configuration jobConf = getJobConf(); + jobConf.set( + FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); + FileSystem mockS3 = getMockS3A(); + pathIsDirectory(mockS3, outputPath); + try (DurationInfo ignored = + new DurationInfo(LOG, "listing pending uploads")) { + AbstractS3ACommitter.ActiveCommit activeCommit + = committer.listPendingUploadsToCommit(getJob()); + Assertions.assertThat(activeCommit.getSourceFiles()) + .describedAs("Source files of %s", activeCommit) + .hasSize(TASKS); + } + } + + @Test + public void test_030_commitFiles() throws Exception { + DirectoryCommitterForTesting committer = newJobCommitter(); + StagingTestBase.ClientResults results = getMockResults(); + results.addUploads(activeUploads); + Configuration jobConf = getJobConf(); + jobConf.set( + FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); + S3AFileSystem mockS3 = getMockS3A(); + pathIsDirectory(mockS3, outputPath); + + try (DurationInfo ignored = + new DurationInfo(LOG, "Committing Job")) { + committer.commitJob(getJob()); + } + + Assertions.assertThat(results.getCommits()) + .describedAs("commit count") + .hasSize(TOTAL_COMMIT_COUNT); + AbstractS3ACommitter.ActiveCommit activeCommit = committer.activeCommit; + Assertions.assertThat(activeCommit.getCommittedObjects()) + .describedAs("committed objects in active commit") + .hasSize(Math.min(TOTAL_COMMIT_COUNT, + CommitConstants.SUCCESS_MARKER_FILE_LIMIT)); + Assertions.assertThat(activeCommit.getCommittedFileCount()) + .describedAs("committed objects in active commit") + .isEqualTo(TOTAL_COMMIT_COUNT); + + } + + @Test + public void test_040_abortFiles() throws Exception { + DirectoryStagingCommitter committer = newJobCommitter(); + getMockResults().addUploads(activeUploads); + Configuration jobConf = getJobConf(); + jobConf.set( + FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); + FileSystem mockS3 = getMockS3A(); + pathIsDirectory(mockS3, outputPath); + + committer.abortJob(getJob(), JobStatus.State.FAILED); + } + + + /** + * Committer overridden for better testing. + */ + private static final class DirectoryCommitterForTesting extends + DirectoryStagingCommitter { + private ActiveCommit activeCommit; + + private DirectoryCommitterForTesting(Path outputPath, + TaskAttemptContext context) throws IOException { + super(outputPath, context); + } + + @Override + protected void initOutput(Path out) throws IOException { + super.initOutput(out); + setOutputPath(out); + } + + /** + * Returns the mock FS without checking FS type. + * @param out output path + * @param config job/task config + * @return a filesystem. + * @throws IOException failure to get the FS + */ + @Override + protected FileSystem getDestinationFS(Path out, Configuration config) + throws IOException { + return out.getFileSystem(config); + } + + @Override + public Path getJobAttemptPath(JobContext context) { + return stagingPath; + } + + @Override + protected void commitJobInternal(final JobContext context, + final ActiveCommit pending) + throws IOException { + activeCommit = pending; + super.commitJobInternal(context, pending); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java index 8939296719ab6..15ea75476a93d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java @@ -35,6 +35,7 @@ import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; import com.google.common.collect.Sets; +import org.assertj.core.api.Assertions; import org.hamcrest.core.StringStartsWith; import org.junit.After; import org.junit.Before; @@ -535,33 +536,31 @@ public void testJobCommitFailure() throws Exception { return jobCommitter.toString(); }); - assertEquals("Should have succeeded to commit some uploads", - 5, results.getCommits().size()); - - assertEquals("Should have deleted the files that succeeded", - 5, results.getDeletes().size()); Set commits = results.getCommits() .stream() - .map((commit) -> commit.getBucketName() + commit.getKey()) + .map(commit -> + "s3a://" + commit.getBucketName() + "/" + commit.getKey()) .collect(Collectors.toSet()); Set deletes = results.getDeletes() .stream() - .map((delete) -> delete.getBucketName() + delete.getKey()) + .map(delete -> + "s3a://" + delete.getBucketName() + "/" + delete.getKey()) .collect(Collectors.toSet()); - assertEquals("Committed and deleted objects should match", - commits, deletes); - - assertEquals("Mismatch in aborted upload count", - 7, results.getAborts().size()); + Assertions.assertThat(commits) + .describedAs("Committed objects compared to deleted paths %s", results) + .containsExactlyInAnyOrderElementsOf(deletes); + Assertions.assertThat(results.getAborts()) + .describedAs("aborted count in %s", results) + .hasSize(7); Set uploadIds = getCommittedIds(results.getCommits()); uploadIds.addAll(getAbortedIds(results.getAborts())); - - assertEquals("Should have committed/deleted or aborted all uploads", - uploads, uploadIds); + Assertions.assertThat(uploadIds) + .describedAs("Combined commit/delete and aborted upload IDs") + .containsExactlyInAnyOrderElementsOf(uploads); assertPathDoesNotExist(fs, "jobAttemptPath not deleted", jobAttemptPath); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java index 994ecef3a83ec..98075b827a7c2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java @@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.PathExistsException; +import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; @@ -84,17 +85,18 @@ protected void verifyFailureConflictOutcome() throws Exception { () -> committer.setupJob(getJob())); // but there are no checks in job commit (HADOOP-15469) - committer.commitJob(getJob()); + // this is done by calling the preCommit method directly, + committer.preCommitJob(getJob(), AbstractS3ACommitter.ActiveCommit.empty()); - reset((FileSystem) getMockS3A()); + reset(getMockS3A()); pathDoesNotExist(getMockS3A(), outputPath); committer.setupJob(getJob()); verifyExistenceChecked(getMockS3A(), outputPath); verifyMkdirsInvoked(getMockS3A(), outputPath); - verifyNoMoreInteractions((FileSystem) getMockS3A()); + verifyNoMoreInteractions(getMockS3A()); - reset((FileSystem) getMockS3A()); + reset(getMockS3A()); pathDoesNotExist(getMockS3A(), outputPath); committer.commitJob(getJob()); verifyCompletion(getMockS3A()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java index e7410e33fba94..872097ff6f032 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java @@ -18,20 +18,21 @@ package org.apache.hadoop.fs.s3a.commit.staging; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.List; import java.util.UUID; -import com.google.common.collect.Lists; import org.junit.Test; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.MockS3AFileSystem; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.PathCommitException; +import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -59,37 +60,59 @@ PartitionedStagingCommitter newJobCommitter() throws IOException { /** * Subclass of the Partitioned Staging committer used in the test cases. */ - private static final class PartitionedStagingCommitterForTesting + private final class PartitionedStagingCommitterForTesting extends PartitionedCommitterForTesting { - private boolean aborted = false; + private boolean aborted; private PartitionedStagingCommitterForTesting(TaskAttemptContext context) throws IOException { super(StagingTestBase.outputPath, context); } + /** + * Generate pending uploads to commit. + * This is quite complex as the mock pending uploads need to be saved + * to a filesystem for the next stage of the commit process. + * To simulate multiple commit, more than one .pendingset file is created, + * @param context job context + * @return an active commit containing a list of paths to valid pending set + * file. + * @throws IOException IO failure + */ @Override - protected List listPendingUploadsToCommit( + protected ActiveCommit listPendingUploadsToCommit( JobContext context) throws IOException { - List pending = Lists.newArrayList(); + LocalFileSystem localFS = FileSystem.getLocal(getConf()); + ActiveCommit activeCommit = new ActiveCommit(localFS, + new ArrayList<>(0)); + // need to create some pending entries. for (String dateint : Arrays.asList("20161115", "20161116")) { + PendingSet pendingSet = new PendingSet(); for (String hour : Arrays.asList("13", "14")) { + String uploadId = UUID.randomUUID().toString(); String key = OUTPUT_PREFIX + "/dateint=" + dateint + "/hour=" + hour + - "/" + UUID.randomUUID().toString() + ".parquet"; + "/" + uploadId + ".parquet"; SinglePendingCommit commit = new SinglePendingCommit(); commit.setBucket(BUCKET); commit.setDestinationKey(key); commit.setUri("s3a://" + BUCKET + "/" + key); - commit.setUploadId(UUID.randomUUID().toString()); + commit.setUploadId(uploadId); ArrayList etags = new ArrayList<>(); etags.add("tag1"); commit.setEtags(etags); - pending.add(commit); + pendingSet.add(commit); + // register the upload so commit operations are not rejected + getMockResults().addUpload(uploadId, key); } + File file = File.createTempFile("staging", ".pendingset"); + file.deleteOnExit(); + Path path = new Path(file.toURI()); + pendingSet.save(localFS, path, true); + activeCommit.add(path); } - return pending; + return activeCommit; } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java index 8116b79eb7fe7..4b568263ba71b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java @@ -27,6 +27,7 @@ import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import org.assertj.core.api.Assertions; import org.junit.BeforeClass; import org.junit.Test; @@ -114,18 +115,7 @@ public void testFail() throws Exception { reset(mockS3); committer.commitTask(getTAC()); - Set files = Sets.newHashSet(); - for (InitiateMultipartUploadRequest request : - getMockResults().getRequests().values()) { - assertEquals(BUCKET, request.getBucketName()); - files.add(request.getKey()); - } - assertEquals("Should have the right number of uploads", - relativeFiles.size(), files.size()); - - Set expected = buildExpectedList(committer); - - assertEquals("Should have correct paths", expected, files); + verifyFilesCreated(committer); } @Test @@ -146,18 +136,29 @@ public void testAppend() throws Exception { pathExists(mockS3, new Path(outputPath, relativeFiles.get(2)).getParent()); committer.commitTask(getTAC()); + verifyFilesCreated(committer); + } + + /** + * Verify that the files created matches that expected. + * @param committer committer + */ + protected void verifyFilesCreated( + final PartitionedStagingCommitter committer) { Set files = Sets.newHashSet(); for (InitiateMultipartUploadRequest request : getMockResults().getRequests().values()) { assertEquals(BUCKET, request.getBucketName()); files.add(request.getKey()); } - assertEquals("Should have the right number of uploads", - relativeFiles.size(), files.size()); + Assertions.assertThat(files) + .describedAs("Should have the right number of uploads") + .hasSize(relativeFiles.size()); Set expected = buildExpectedList(committer); - - assertEquals("Should have correct paths", expected, files); + Assertions.assertThat(files) + .describedAs("Should have correct paths") + .containsExactlyInAnyOrderElementsOf(expected); } @Test @@ -180,18 +181,7 @@ public void testReplace() throws Exception { pathExists(mockS3, new Path(outputPath, relativeFiles.get(3)).getParent()); committer.commitTask(getTAC()); - Set files = Sets.newHashSet(); - for (InitiateMultipartUploadRequest request : - getMockResults().getRequests().values()) { - assertEquals(BUCKET, request.getBucketName()); - files.add(request.getKey()); - } - assertEquals("Should have the right number of uploads", - relativeFiles.size(), files.size()); - - Set expected = buildExpectedList(committer); - - assertEquals("Should have correct paths", expected, files); + verifyFilesCreated(committer); } public Set buildExpectedList(StagingCommitter committer) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitMRJob.java deleted file mode 100644 index 1e44086b1e125..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitMRJob.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.commit.staging.integration; - -import java.io.IOException; - -import org.junit.AfterClass; -import org.junit.BeforeClass; - -import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob; -import org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitter; -import org.apache.hadoop.mapred.JobConf; - -/** - * Full integration test for the directory committer. - */ -public final class ITestDirectoryCommitMRJob extends AbstractITCommitMRJob { - - /** - * The static cluster binding with the lifecycle of this test; served - * through instance-level methods for sharing across methods in the - * suite. - */ - @SuppressWarnings("StaticNonFinalField") - private static ClusterBinding clusterBinding; - - @BeforeClass - public static void setupClusters() throws IOException { - clusterBinding = createCluster(new JobConf()); } - - @AfterClass - public static void teardownClusters() throws IOException { - terminateCluster(clusterBinding); - } - - @Override - public ClusterBinding getClusterBinding() { - return clusterBinding; - } - - @Override - protected String committerName() { - return DirectoryStagingCommitter.NAME; - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionCommitMRJob.java deleted file mode 100644 index 6106974ce74ed..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionCommitMRJob.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.commit.staging.integration; - -import java.io.IOException; - -import org.junit.AfterClass; -import org.junit.BeforeClass; - -import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob; -import org.apache.hadoop.fs.s3a.commit.staging.PartitionedStagingCommitter; -import org.apache.hadoop.mapred.JobConf; - -/** - * Full integration test for the partition committer. - */ -public final class ITestPartitionCommitMRJob extends AbstractITCommitMRJob { - - /** - * The static cluster binding with the lifecycle of this test; served - * through instance-level methods for sharing across methods in the - * suite. - */ - @SuppressWarnings("StaticNonFinalField") - private static ClusterBinding clusterBinding; - - @BeforeClass - public static void setupClusters() throws IOException { - clusterBinding = createCluster(new JobConf()); - } - - @AfterClass - public static void teardownClusters() throws IOException { - terminateCluster(clusterBinding); - } - - @Override - public ClusterBinding getClusterBinding() { - return clusterBinding; - } - - @Override - protected String committerName() { - return PartitionedStagingCommitter.NAME; - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJob.java deleted file mode 100644 index 218c72ac50ea0..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJob.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.commit.staging.integration; - -import java.io.IOException; - -import org.hamcrest.core.StringContains; -import org.hamcrest.core.StringEndsWith; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob; -import org.apache.hadoop.fs.s3a.commit.staging.StagingCommitter; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.security.UserGroupInformation; - -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_TMP_PATH; -import static org.apache.hadoop.fs.s3a.commit.staging.Paths.getMultipartUploadCommitsDirectory; -import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.STAGING_UPLOADS; - -/** - * Full integration test for the staging committer. - */ -public final class ITestStagingCommitMRJob extends AbstractITCommitMRJob { - - /** - * The static cluster binding with the lifecycle of this test; served - * through instance-level methods for sharing across methods in the - * suite. - */ - @SuppressWarnings("StaticNonFinalField") - private static ClusterBinding clusterBinding; - - @BeforeClass - public static void setupClusters() throws IOException { - clusterBinding = createCluster(new JobConf()); - } - - @AfterClass - public static void teardownClusters() throws IOException { - terminateCluster(clusterBinding); - } - - @Override - public ClusterBinding getClusterBinding() { - return clusterBinding; - } - - @Override - protected String committerName() { - return StagingCommitter.NAME; - } - - /** - * Verify that staging commit dirs are made absolute under the user's - * home directory, so, in a secure cluster, private. - */ - @Test - public void testStagingDirectory() throws Throwable { - FileSystem hdfs = getDFS(); - Configuration conf = hdfs.getConf(); - conf.set(FS_S3A_COMMITTER_STAGING_TMP_PATH, "private"); - Path dir = getMultipartUploadCommitsDirectory(conf, "UUID"); - assertThat("Directory " + dir + " path is wrong", - dir.toString(), - StringEndsWith.endsWith("UUID/" - + STAGING_UPLOADS)); - assertTrue("path unqualified", dir.isAbsolute()); - String self = UserGroupInformation.getCurrentUser().getShortUserName(); - assertThat(dir.toString(), - StringContains.containsString("/user/" + self + "/private")); - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJobBadDest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJobBadDest.java deleted file mode 100644 index 72488132faf76..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitMRJobBadDest.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.commit.staging.integration; - -import java.io.IOException; - -import org.junit.AfterClass; -import org.junit.BeforeClass; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob; -import org.apache.hadoop.fs.s3a.commit.staging.StagingCommitter; -import org.apache.hadoop.mapred.FileAlreadyExistsException; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.hadoop.test.LambdaTestUtils; - -/** - * This is a test to verify that the committer will fail if the destination - * directory exists, and that this happens in job setup. - */ -public final class ITestStagingCommitMRJobBadDest extends AbstractITCommitMRJob { - - /** - * The static cluster binding with the lifecycle of this test; served - * through instance-level methods for sharing across methods in the - * suite. - */ - @SuppressWarnings("StaticNonFinalField") - private static ClusterBinding clusterBinding; - - @BeforeClass - public static void setupClusters() throws IOException { - clusterBinding = createCluster(new JobConf()); - } - - @AfterClass - public static void teardownClusters() throws IOException { - terminateCluster(clusterBinding); - } - - @Override - public ClusterBinding getClusterBinding() { - return clusterBinding; - } - - @Override - protected String committerName() { - return StagingCommitter.NAME; - } - - /** - * create the destination directory and expect a failure. - * @param conf configuration - */ - @Override - protected void applyCustomConfigOptions(JobConf conf) throws IOException { - // This is the destination in the S3 FS - String outdir = conf.get(FileOutputFormat.OUTDIR); - S3AFileSystem fs = getFileSystem(); - Path outputPath = new Path(outdir); - fs.mkdirs(outputPath); - } - - @Override - public void testMRJob() throws Exception { - LambdaTestUtils.intercept(FileAlreadyExistsException.class, - "Output directory", - super::testMRJob); - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortDirectoryCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortDirectoryCommitter.java deleted file mode 100644 index cb9cdd0f33455..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortDirectoryCommitter.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.commit.terasort; - -import java.io.IOException; - -import org.junit.AfterClass; -import org.junit.BeforeClass; - -import org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitter; -import org.apache.hadoop.mapred.JobConf; - -/** - * Terasort with the directory committer. - */ -public final class ITestTerasortDirectoryCommitter extends AbstractCommitTerasortIT { - - /** - * The static cluster binding with the lifecycle of this test; served - * through instance-level methods for sharing across methods in the - * suite. - */ - @SuppressWarnings("StaticNonFinalField") - private static ClusterBinding clusterBinding; - - @BeforeClass - public static void setupClusters() throws IOException { - clusterBinding = createCluster(new JobConf()); - } - - @AfterClass - public static void teardownClusters() throws IOException { - clusterBinding.terminate(); - } - - @Override - public ClusterBinding getClusterBinding() { - return clusterBinding; - } - - @Override - protected String committerName() { - return DirectoryStagingCommitter.NAME; - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortMagicCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortMagicCommitter.java deleted file mode 100644 index e1b4eac627a59..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortMagicCommitter.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.commit.terasort; - -import java.io.IOException; - -import org.junit.AfterClass; -import org.junit.BeforeClass; - -import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter; -import org.apache.hadoop.mapred.JobConf; - -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_COMMITTER_ENABLED; - -/** - * Terasort with the magic committer. - */ -public final class ITestTerasortMagicCommitter - extends AbstractCommitTerasortIT { - - /** - * The static cluster binding with the lifecycle of this test; served - * through instance-level methods for sharing across methods in the - * suite. - */ - @SuppressWarnings("StaticNonFinalField") - private static ClusterBinding clusterBinding; - - @BeforeClass - public static void setupClusters() throws IOException { - clusterBinding = createCluster(new JobConf()); - } - - @AfterClass - public static void teardownClusters() throws IOException { - clusterBinding.terminate(); - } - - @Override - public ClusterBinding getClusterBinding() { - return clusterBinding; - } - @Override - protected String committerName() { - return MagicS3GuardCommitter.NAME; - } - - /** - * Turn on the magic commit support for the FS, else nothing will work. - * @param conf configuration - */ - @Override - protected void applyCustomConfigOptions(JobConf conf) { - conf.setBoolean(MAGIC_COMMITTER_ENABLED, true); - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/AbstractCommitTerasortIT.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java similarity index 50% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/AbstractCommitTerasortIT.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java index 479b3c80f7c55..dc6c6d19db9ab 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/AbstractCommitTerasortIT.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java @@ -19,24 +19,33 @@ package org.apache.hadoop.fs.s3a.commit.terasort; import java.io.File; -import java.nio.charset.Charset; +import java.io.FileNotFoundException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; -import java.util.function.BiConsumer; +import java.util.function.Consumer; +import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; +import org.junit.runner.RunWith; import org.junit.runners.MethodSorters; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.io.FileUtils; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.examples.terasort.TeraGen; import org.apache.hadoop.examples.terasort.TeraSort; import org.apache.hadoop.examples.terasort.TeraSortConfigKeys; import org.apache.hadoop.examples.terasort.TeraValidate; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.AbstractYarnClusterITest; +import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter; +import org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitter; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.StringUtils; @@ -44,45 +53,79 @@ import org.apache.hadoop.util.ToolRunner; import static java.util.Optional.empty; -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.CONFLICT_MODE_APPEND; -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_CONFLICT_MODE; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.lsR; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_COMMITTER_ENABLED; /** * Runs Terasort against S3A. * - * This is all done on a shared mini YARN and HDFS clusters, set up before - * any of the tests methods run. - * + * Parameterized by committer name, using a YARN cluster + * shared across all test runs. * The tests run in sequence, so each operation is isolated. - * This also means that the test paths deleted in test + * This also means that the test paths are deleted in test * teardown; shared variables must all be static. + * + * The test is a scale test; for each parameter it takes a few minutes to + * run the full suite. + * Before anyone calls that out as slow: try running the test with the file + * committer. */ @FixMethodOrder(MethodSorters.NAME_ASCENDING) +@RunWith(Parameterized.class) @SuppressWarnings("StaticNonFinalField") -public abstract class AbstractCommitTerasortIT extends - AbstractYarnClusterITest { +public class ITestTerasortOnS3A extends AbstractYarnClusterITest { private static final Logger LOG = - LoggerFactory.getLogger(AbstractCommitTerasortIT.class); + LoggerFactory.getLogger(ITestTerasortOnS3A.class); - // all the durations are optional as they only get filled in when - // a test run successfully completes. Failed tests don't have numbers. - private static Optional terasortDuration = empty(); + public static final int EXPECTED_PARTITION_COUNT = 10; - private static Optional teragenStageDuration = empty(); + public static final int PARTITION_SAMPLE_SIZE = 1000; - private static Optional terasortStageDuration = empty(); + public static final int ROW_COUNT = 1000; - private static Optional teravalidateStageDuration = empty(); + /** + * Duration tracker created in the first of the test cases and closed + * in {@link #test_140_teracomplete()}. + */ + private static Optional terasortDuration = empty(); + /** + * Tracker of which stages are completed and how long they took. + */ + private static Map completedStages = new HashMap<>(); + + /** Name of the committer for this run. */ + private final String committerName; + + /** Base path for all the terasort input and output paths. */ private Path terasortPath; + /** Input (teragen) path. */ private Path sortInput; + /** Path where sorted data goes. */ private Path sortOutput; + /** Path for validated job's output. */ private Path sortValidate; + /** + * Test array for parameterized test runs. + * + * @return the committer binding for this run. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {DirectoryStagingCommitter.NAME}, + {MagicS3GuardCommitter.NAME}}); + } + + public ITestTerasortOnS3A(final String committerName) { + this.committerName = committerName; + } + /** * Not using special paths here. * @return false @@ -92,6 +135,11 @@ public boolean useInconsistentClient() { return false; } + @Override + protected String committerName() { + return committerName; + } + @Override public void setup() throws Exception { super.setup(); @@ -100,44 +148,88 @@ public void setup() throws Exception { } /** - * Set up for terasorting by initializing paths. - * The paths used must be unique across parallel runs. + * Set up the job conf with the options for terasort chosen by the scale + * options. + * @param conf configuration */ - private void prepareToTerasort() { + @Override + protected void applyCustomConfigOptions(JobConf conf) { // small sample size for faster runs - Configuration yarnConfig = getYarn().getConfig(); - yarnConfig.setInt(TeraSortConfigKeys.SAMPLE_SIZE.key(), 1000); - yarnConfig.setBoolean( - TeraSortConfigKeys.USE_SIMPLE_PARTITIONER.key(), - true); - yarnConfig.setBoolean( + conf.setBoolean(MAGIC_COMMITTER_ENABLED, true); + conf.setInt(TeraSortConfigKeys.SAMPLE_SIZE.key(), + getSampleSizeForEachPartition()); + conf.setInt(TeraSortConfigKeys.NUM_PARTITIONS.key(), + getExpectedPartitionCount()); + conf.setBoolean( TeraSortConfigKeys.USE_SIMPLE_PARTITIONER.key(), false); - terasortPath = new Path("/terasort-" + getClass().getSimpleName()) + } + + private int getExpectedPartitionCount() { + return EXPECTED_PARTITION_COUNT; + } + + private int getSampleSizeForEachPartition() { + return PARTITION_SAMPLE_SIZE; + } + + protected int getRowCount() { + return ROW_COUNT; + } + + /** + * Set up the terasort by initializing paths variables + * The paths used must be unique across parameterized runs but + * common across all test cases in a single parameterized run. + */ + private void prepareToTerasort() { + // small sample size for faster runs + terasortPath = new Path("/terasort-" + committerName) .makeQualified(getFileSystem()); sortInput = new Path(terasortPath, "sortin"); sortOutput = new Path(terasortPath, "sortout"); sortValidate = new Path(terasortPath, "validate"); - if (!terasortDuration.isPresent()) { - terasortDuration = Optional.of(new DurationInfo(LOG, "Terasort")); - } + + } + + /** + * Declare that a stage has completed. + * @param stage stage name/key in the map + * @param d duration. + */ + private static void completedStage(final String stage, + final DurationInfo d) { + completedStages.put(stage, d); } /** - * Execute a single stage in the terasort, - * @param stage Stage name for messages/assertions. + * Declare a stage which is required for this test case. + * @param stage stage name + */ + private static void requireStage(final String stage) { + Assume.assumeTrue( + "Required stage was not completed: " + stage, + completedStages.get(stage) != null); + } + + /** + * Execute a single stage in the terasort. + * Updates the completed stages map with the stage duration -if successful. + * @param stage Stage name for the stages map. * @param jobConf job conf - * @param dest destination directory -the _SUCCESS File will be expected here. + * @param dest destination directory -the _SUCCESS file will be expected here. * @param tool tool to run. * @param args args for the tool. + * @param minimumFileCount minimum number of files to have been created * @throws Exception any failure */ - private Optional executeStage( + private void executeStage( final String stage, final JobConf jobConf, final Path dest, final Tool tool, - final String[] args) throws Exception { + final String[] args, + final int minimumFileCount) throws Exception { int result; DurationInfo d = new DurationInfo(LOG, stage); try { @@ -145,22 +237,30 @@ private Optional executeStage( } finally { d.close(); } + dumpOutputTree(dest); assertEquals(stage + "(" + StringUtils.join(", ", args) + ")" + " failed", 0, result); - validateSuccessFile(dest, committerName(), getFileSystem(), stage); - return Optional.of(d); + validateSuccessFile(dest, committerName(), getFileSystem(), stage, + minimumFileCount); + completedStage(stage, d); } /** * Set up terasort by cleaning out the destination, and note the initial * time before any of the jobs are executed. + * + * This is executed first for each parameterized run. + * It is where all variables which need to be reset for each run need + * to be reset. */ @Test public void test_100_terasort_setup() throws Throwable { describe("Setting up for a terasort"); getFileSystem().delete(terasortPath, true); + completedStages = new HashMap<>(); + terasortDuration = Optional.of(new DurationInfo(LOG, false, "Terasort")); } @Test @@ -170,42 +270,46 @@ public void test_110_teragen() throws Throwable { JobConf jobConf = newJobConf(); patchConfigurationForCommitter(jobConf); - teragenStageDuration = executeStage("Teragen", + executeStage("teragen", jobConf, sortInput, new TeraGen(), - new String[]{Integer.toString(SCALE_TEST_KEYS), sortInput.toString()}); + new String[]{Integer.toString(getRowCount()), sortInput.toString()}, + 1); } + @Test public void test_120_terasort() throws Throwable { describe("Terasort from %s to %s", sortInput, sortOutput); + requireStage("teragen"); getFileSystem().delete(sortOutput, true); loadSuccessFile(getFileSystem(), sortInput, "previous teragen stage"); JobConf jobConf = newJobConf(); patchConfigurationForCommitter(jobConf); - // this job adds some data, so skip it. - jobConf.set(FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); - terasortStageDuration = executeStage("TeraSort", + executeStage("terasort", jobConf, sortOutput, new TeraSort(), - new String[]{sortInput.toString(), sortOutput.toString()}); + new String[]{sortInput.toString(), sortOutput.toString()}, + 1); } @Test public void test_130_teravalidate() throws Throwable { describe("TeraValidate from %s to %s", sortOutput, sortValidate); + requireStage("terasort"); getFileSystem().delete(sortValidate, true); loadSuccessFile(getFileSystem(), sortOutput, "previous terasort stage"); JobConf jobConf = newJobConf(); patchConfigurationForCommitter(jobConf); - teravalidateStageDuration = executeStage("TeraValidate", + executeStage("teravalidate", jobConf, sortValidate, new TeraValidate(), - new String[]{sortOutput.toString(), sortValidate.toString()}); + new String[]{sortOutput.toString(), sortValidate.toString()}, + 1); } /** @@ -214,7 +318,10 @@ public void test_130_teravalidate() throws Throwable { */ @Test public void test_140_teracomplete() throws Throwable { - terasortDuration.get().close(); + terasortDuration.ifPresent(d -> { + d.close(); + completedStage("overall", d); + }); final StringBuilder results = new StringBuilder(); results.append("\"Operation\"\t\"Duration\"\n"); @@ -222,19 +329,20 @@ public void test_140_teracomplete() throws Throwable { // this is how you dynamically create a function in a method // for use afterwards. // Works because there's no IOEs being raised in this sequence. - BiConsumer> stage = - (s, od) -> - results.append(String.format("\"%s\"\t\"%s\"\n", - s, - od.map(DurationInfo::getDurationString).orElse(""))); - - stage.accept("Generate", teragenStageDuration); - stage.accept("Terasort", terasortStageDuration); - stage.accept("Validate", teravalidateStageDuration); - stage.accept("Completed", terasortDuration); + Consumer stage = (s) -> { + DurationInfo duration = completedStages.get(s); + results.append(String.format("\"%s\"\t\"%s\"\n", + s, + duration == null ? "" : duration)); + }; + + stage.accept("teragen"); + stage.accept("terasort"); + stage.accept("teravalidate"); + stage.accept("overall"); String text = results.toString(); File resultsFile = File.createTempFile("results", ".csv"); - FileUtils.write(resultsFile, text, Charset.forName("UTF-8")); + FileUtils.write(resultsFile, text, StandardCharsets.UTF_8); LOG.info("Results are in {}\n{}", resultsFile, text); } @@ -252,4 +360,18 @@ public void test_150_teracleanup() throws Throwable { public void test_200_directory_deletion() throws Throwable { getFileSystem().delete(terasortPath, true); } + + /** + * Dump the files under a path -but not fail if the path is not present., + * @param path path to dump + * @throws Exception any failure. + */ + protected void dumpOutputTree(Path path) throws Exception { + LOG.info("Files under output directory {}", path); + try { + lsR(getFileSystem(), path, true); + } catch (FileNotFoundException e) { + LOG.info("Output directory {} not found", path); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index f6162644e2535..6e20fbcda7efd 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -58,7 +58,7 @@ log4j.logger.org.apache.hadoop.ipc.Server=WARN # Log S3Guard classes #log4j.logger.org.apache.hadoop.fs.s3a.s3guard=DEBUG # if set to debug, this will log the PUT/DELETE operations on a store -#log4j.logger.org.apache.hadoop.fs.s3a.s3guard.Operations=DEBUG +log4j.logger.org.apache.hadoop.fs.s3a.s3guard.Operations=DEBUG # Log Committer classes #log4j.logger.org.apache.hadoop.fs.s3a.commit=DEBUG diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSdkConfiguration.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSdkConfiguration.java index 980b683f66e41..27004db578788 100644 --- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSdkConfiguration.java +++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestAdlSdkConfiguration.java @@ -98,6 +98,7 @@ public void testSSLChannelMode(SSLChannelMode expectedMode, conf = AdlStorageConfiguration.getConfiguration(); conf.set(ADL_SSL_CHANNEL_MODE, sslChannelModeConfigValue); fs = (AdlFileSystem) (AdlStorageConfiguration.createStorageConnector(conf)); + Assume.assumeNotNull(fs); SSLChannelMode sslChannelMode = fs.getAdlClient().getSSLChannelMode(); Assert.assertEquals( diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 723a25cadb5af..bec774977a85e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -25,9 +25,12 @@ import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.AuthConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation; @@ -69,6 +72,7 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class AbfsConfiguration{ + private final Configuration rawConfig; private final String accountName; private final boolean isSecure; @@ -174,6 +178,10 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_USE_UPN) private boolean useUpn; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ABFS_LATENCY_TRACK, + DefaultValue = DEFAULT_ABFS_LATENCY_TRACK) + private boolean trackLatency; + private Map storageAccountKeys; public AbfsConfiguration(final Configuration rawConfig, String accountName) @@ -467,6 +475,10 @@ public boolean isUpnUsed() { return this.useUpn; } + public boolean shouldTrackLatency() { + return this.trackLatency; + } + public AccessTokenProvider getTokenProvider() throws TokenAccessProviderException { AuthType authType = getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey); if (authType == AuthType.OAuth) { @@ -486,13 +498,25 @@ public AccessTokenProvider getTokenProvider() throws TokenAccessProviderExceptio String password = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD); tokenProvider = new UserPasswordTokenProvider(authEndpoint, username, password); } else if (tokenProviderClass == MsiTokenProvider.class) { + String authEndpoint = getTrimmedPasswordString( + FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT, + AuthConfigurations.DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT); String tenantGuid = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT); String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID); - tokenProvider = new MsiTokenProvider(tenantGuid, clientId); + String authority = getTrimmedPasswordString( + FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY, + AuthConfigurations.DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY); + authority = appendSlashIfNeeded(authority); + tokenProvider = new MsiTokenProvider(authEndpoint, tenantGuid, + clientId, authority); } else if (tokenProviderClass == RefreshTokenBasedTokenProvider.class) { + String authEndpoint = getTrimmedPasswordString( + FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN_ENDPOINT, + AuthConfigurations.DEFAULT_FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN_ENDPOINT); String refreshToken = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN); String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID); - tokenProvider = new RefreshTokenBasedTokenProvider(clientId, refreshToken); + tokenProvider = new RefreshTokenBasedTokenProvider(authEndpoint, + clientId, refreshToken); } else { throw new IllegalArgumentException("Failed to initialize " + tokenProviderClass); } @@ -649,4 +673,18 @@ void setDisableOutputStreamFlush(boolean disableOutputStreamFlush) { this.disableOutputStreamFlush = disableOutputStreamFlush; } + private String getTrimmedPasswordString(String key, String defaultValue) throws IOException { + String value = getPasswordString(key); + if (StringUtils.isBlank(value)) { + value = defaultValue; + } + return value.trim(); + } + + private String appendSlashIfNeeded(String authority) { + if (!authority.endsWith(AbfsHttpConstants.FORWARD_SLASH)) { + authority = authority + AbfsHttpConstants.FORWARD_SLASH; + } + return authority; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 2a4e76b28bb62..c6640c4acbbda 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -56,6 +56,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; @@ -111,7 +112,7 @@ public void initialize(URI uri, Configuration configuration) this.setWorkingDirectory(this.getHomeDirectory()); if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) { - if (!this.fileSystemExists()) { + if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH)) == null) { try { this.createFileSystem(); } catch (AzureBlobFileSystemException ex) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 7f1bf103c2d52..e1143419cce53 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -35,6 +35,7 @@ import java.nio.charset.StandardCharsets; import java.text.ParseException; import java.text.SimpleDateFormat; +import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Date; @@ -81,6 +82,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; +import org.apache.hadoop.fs.azurebfs.services.LatencyTracker; import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials; import org.apache.hadoop.fs.azurebfs.utils.Base64; import org.apache.hadoop.fs.azurebfs.utils.CRC64; @@ -130,6 +132,7 @@ public class AzureBlobFileSystemStore implements Closeable { private final AuthType authType; private final UserGroupInformation userGroupInformation; private final IdentityTransformer identityTransformer; + private final LatencyTracker latencyTracker; public AzureBlobFileSystemStore(URI uri, boolean isSecureScheme, Configuration configuration) throws IOException { @@ -162,6 +165,7 @@ public AzureBlobFileSystemStore(URI uri, boolean isSecureScheme, Configuration c this.authType = abfsConfiguration.getAuthType(accountName); boolean usingOauth = (authType == AuthType.OAuth); boolean useHttps = (usingOauth || abfsConfiguration.isHttpsAlwaysUsed()) ? true : isSecureScheme; + this.latencyTracker = new LatencyTracker(fileSystemName, accountName, this.abfsConfiguration); initializeClient(uri, fileSystemName, accountName, useHttps); this.identityTransformer = new IdentityTransformer(abfsConfiguration.getRawConfiguration()); } @@ -211,10 +215,16 @@ private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, In public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { if (!isNamespaceEnabledSet) { + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; + LOG.debug("Get root ACL status"); try { - client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + AbfsHttpConstants.ROOT_PATH); + AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + AbfsHttpConstants.ROOT_PATH); + res = op.getResult(); isNamespaceEnabled = true; + success = true; } catch (AbfsRestOperationException ex) { // Get ACL status is a HEAD request, its response doesn't contain errorCode // So can only rely on its status code to determine its account type. @@ -222,6 +232,8 @@ public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { throw ex; } isNamespaceEnabled = false; + } finally { + latencyTracker.recordClientLatency(start, "getIsNamespaceEnabled", "getAclStatus", success, res); } isNamespaceEnabledSet = true; } @@ -265,182 +277,295 @@ public AbfsConfiguration getAbfsConfiguration() { } public Hashtable getFilesystemProperties() throws AzureBlobFileSystemException { - LOG.debug("getFilesystemProperties for filesystem: {}", - client.getFileSystem()); + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; + + try { + LOG.debug("getFilesystemProperties for filesystem: {}", + client.getFileSystem()); + + final Hashtable parsedXmsProperties; - final Hashtable parsedXmsProperties; + final AbfsRestOperation op = client.getFilesystemProperties(); + res = op.getResult(); - final AbfsRestOperation op = client.getFilesystemProperties(); - final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); + final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); - parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties); + parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties); + success = true; - return parsedXmsProperties; + return parsedXmsProperties; + } finally { + latencyTracker.recordClientLatency(start, "getFilesystemProperties", "getFilesystemProperties", success, res); + } } public void setFilesystemProperties(final Hashtable properties) throws AzureBlobFileSystemException { - if (properties == null || properties.isEmpty()) { - return; - } - - LOG.debug("setFilesystemProperties for filesystem: {} with properties: {}", - client.getFileSystem(), - properties); + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - final String commaSeparatedProperties; try { - commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties); - } catch (CharacterCodingException ex) { - throw new InvalidAbfsRestOperationException(ex); - } + if (properties == null || properties.isEmpty()) { + return; + } + + LOG.debug("setFilesystemProperties for filesystem: {} with properties: {}", + client.getFileSystem(), + properties); - client.setFilesystemProperties(commaSeparatedProperties); + final String commaSeparatedProperties; + try { + commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties); + } catch (CharacterCodingException ex) { + throw new InvalidAbfsRestOperationException(ex); + } + + final AbfsRestOperation op = client.setFilesystemProperties(commaSeparatedProperties); + res = op.getResult(); + + success = true; + } finally { + latencyTracker.recordClientLatency(start, "setFilesystemProperties", "setFilesystemProperties", success, res); + } } public Hashtable getPathStatus(final Path path) throws AzureBlobFileSystemException { - LOG.debug("getPathStatus for filesystem: {} path: {}", - client.getFileSystem(), - path); + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - final Hashtable parsedXmsProperties; - final AbfsRestOperation op = client.getPathStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + try { + LOG.debug("getPathStatus for filesystem: {} path: {}", + client.getFileSystem(), + path); + + final Hashtable parsedXmsProperties; + final AbfsRestOperation op = client.getPathStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + res = op.getResult(); + + final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); - final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); + parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties); - parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties); + success = true; - return parsedXmsProperties; + return parsedXmsProperties; + } finally { + latencyTracker.recordClientLatency(start, "getPathStatus", "getPathStatus", success, res); + } } public void setPathProperties(final Path path, final Hashtable properties) throws AzureBlobFileSystemException { - LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}", - client.getFileSystem(), - path, - properties); + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - final String commaSeparatedProperties; try { - commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties); - } catch (CharacterCodingException ex) { - throw new InvalidAbfsRestOperationException(ex); + LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}", + client.getFileSystem(), + path, + properties); + + final String commaSeparatedProperties; + try { + commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties); + } catch (CharacterCodingException ex) { + throw new InvalidAbfsRestOperationException(ex); + } + final AbfsRestOperation op = client.setPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), commaSeparatedProperties); + res = op.getResult(); + + success = true; + } finally { + latencyTracker.recordClientLatency(start, "setPathProperties", "setPathProperties", success, res); } - client.setPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), commaSeparatedProperties); } public void createFilesystem() throws AzureBlobFileSystemException { - LOG.debug("createFilesystem for filesystem: {}", - client.getFileSystem()); + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - client.createFilesystem(); + try { + LOG.debug("createFilesystem for filesystem: {}", + client.getFileSystem()); + + final AbfsRestOperation op = client.createFilesystem(); + res = op.getResult(); + + success = true; + } finally { + latencyTracker.recordClientLatency(start, "createFilesystem", "createFilesystem", success, res); + } } public void deleteFilesystem() throws AzureBlobFileSystemException { - LOG.debug("deleteFilesystem for filesystem: {}", - client.getFileSystem()); + Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; + + try { + LOG.debug("deleteFilesystem for filesystem: {}", + client.getFileSystem()); + + final AbfsRestOperation op = client.deleteFilesystem(); + res = op.getResult(); - client.deleteFilesystem(); + success = true; + } finally { + latencyTracker.recordClientLatency(start, "deleteFilesystem", "deleteFilesystem", success, res); + } } public OutputStream createFile(final Path path, final boolean overwrite, final FsPermission permission, final FsPermission umask) throws AzureBlobFileSystemException { - boolean isNamespaceEnabled = getIsNamespaceEnabled(); - LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", - client.getFileSystem(), - path, - overwrite, - permission.toString(), - umask.toString(), - isNamespaceEnabled); - - client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null); - - return new AbfsOutputStream( - client, - AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), - 0, - abfsConfiguration.getWriteBufferSize(), - abfsConfiguration.isFlushEnabled(), - abfsConfiguration.isOutputStreamFlushDisabled()); + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; + + try { + boolean isNamespaceEnabled = getIsNamespaceEnabled(); + LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", + client.getFileSystem(), + path, + overwrite, + permission.toString(), + umask.toString(), + isNamespaceEnabled); + + final AbfsRestOperation op = client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite, + isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null); + res = op.getResult(); + + success = true; + + return new AbfsOutputStream( + client, + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), + 0, + abfsConfiguration.getWriteBufferSize(), + abfsConfiguration.isFlushEnabled(), + abfsConfiguration.isOutputStreamFlushDisabled()); + } finally { + latencyTracker.recordClientLatency(start, "createFile", "createPath", success, res); + } } public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask) throws AzureBlobFileSystemException { - boolean isNamespaceEnabled = getIsNamespaceEnabled(); - LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}", - client.getFileSystem(), - path, - permission, - umask, - isNamespaceEnabled); + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), false, true, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null); + try { + boolean isNamespaceEnabled = getIsNamespaceEnabled(); + LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}", + client.getFileSystem(), + path, + permission, + umask, + isNamespaceEnabled); + + final AbfsRestOperation op = client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), false, true, + isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null); + res = op.getResult(); + + success = true; + } finally { + latencyTracker.recordClientLatency(start, "createDirectory", "createPath", success, res); + } } public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) throws AzureBlobFileSystemException { - LOG.debug("openFileForRead filesystem: {} path: {}", - client.getFileSystem(), - path); + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - final AbfsRestOperation op = client.getPathStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + try { + LOG.debug("openFileForRead filesystem: {} path: {}", + client.getFileSystem(), + path); - final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); - final long contentLength = Long.parseLong(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); - final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + final AbfsRestOperation op = client.getPathStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + res = op.getResult(); - if (parseIsDirectory(resourceType)) { - throw new AbfsRestOperationException( - AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), - AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), - "openFileForRead must be used with files and not directories", - null); - } + final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); + final long contentLength = Long.parseLong(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + + if (parseIsDirectory(resourceType)) { + throw new AbfsRestOperationException( + AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + "openFileForRead must be used with files and not directories", + null); + } + + success = true; - // Add statistics for InputStream - return new AbfsInputStream(client, statistics, - AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength, - abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), - abfsConfiguration.getTolerateOobAppends(), eTag); + // Add statistics for InputStream + return new AbfsInputStream(client, statistics, + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength, + abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), + abfsConfiguration.getTolerateOobAppends(), eTag); + } finally { + latencyTracker.recordClientLatency(start, "openFileForRead", "getPathStatus", success, res); + } } public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws AzureBlobFileSystemException { - LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", - client.getFileSystem(), - path, - overwrite); + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - final AbfsRestOperation op = client.getPathStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + try { + LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", + client.getFileSystem(), + path, + overwrite); - final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); - final Long contentLength = Long.valueOf(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); + final AbfsRestOperation op = client.getPathStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + res = op.getResult(); - if (parseIsDirectory(resourceType)) { - throw new AbfsRestOperationException( - AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), - AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), - "openFileForRead must be used with files and not directories", - null); - } + final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); + final Long contentLength = Long.valueOf(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); + + if (parseIsDirectory(resourceType)) { + throw new AbfsRestOperationException( + AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + "openFileForRead must be used with files and not directories", + null); + } + + final long offset = overwrite ? 0 : contentLength; - final long offset = overwrite ? 0 : contentLength; + success = true; - return new AbfsOutputStream( - client, - AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), - offset, - abfsConfiguration.getWriteBufferSize(), - abfsConfiguration.isFlushEnabled(), - abfsConfiguration.isOutputStreamFlushDisabled()); + return new AbfsOutputStream( + client, + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), + offset, + abfsConfiguration.getWriteBufferSize(), + abfsConfiguration.isFlushEnabled(), + abfsConfiguration.isOutputStreamFlushDisabled()); + } finally { + latencyTracker.recordClientLatency(start, "openFileForWrite", "getPathStatus", success, res); + } } public void rename(final Path source, final Path destination) throws AzureBlobFileSystemException { + final Instant startAggregate = latencyTracker.getLatencyInstant(); + long countAggregate = 0; + boolean shouldContinue = true; if (isAtomicRenameKey(source.getName())) { LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename," @@ -455,15 +580,34 @@ public void rename(final Path source, final Path destination) throws String continuation = null; do { - AbfsRestOperation op = client.renamePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(source), - AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation); - continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - } while (continuation != null && !continuation.isEmpty()); + try { + AbfsRestOperation op = client.renamePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(source), + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation); + res = op.getResult(); + continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + success = true; + countAggregate++; + shouldContinue = continuation != null && !continuation.isEmpty(); + } finally { + if(shouldContinue) { + latencyTracker.recordClientLatency(start, "rename", "renamePath", success, res); + } else { + latencyTracker.recordClientLatency(start, "rename", "renamePath", success, startAggregate, countAggregate, res); + } + } + } while (shouldContinue); } public void delete(final Path path, final boolean recursive) throws AzureBlobFileSystemException { + final Instant startAggregate = latencyTracker.getLatencyInstant(); + long countAggregate = 0; + boolean shouldContinue = true; + LOG.debug("delete filesystem: {} path: {} recursive: {}", client.getFileSystem(), path, @@ -472,70 +616,102 @@ public void delete(final Path path, final boolean recursive) String continuation = null; do { - AbfsRestOperation op = client.deletePath( - AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation); - continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - } while (continuation != null && !continuation.isEmpty()); + try { + AbfsRestOperation op = client.deletePath( + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation); + res = op.getResult(); + continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + success = true; + countAggregate++; + shouldContinue = continuation != null && !continuation.isEmpty(); + } finally { + if (shouldContinue) { + latencyTracker.recordClientLatency(start, "delete", "deletePath", success, res); + } else { + latencyTracker.recordClientLatency(start, "delete", "deletePath", success, startAggregate, countAggregate, res); + } + } + } while (shouldContinue); } public FileStatus getFileStatus(final Path path) throws IOException { - boolean isNamespaceEnabled = getIsNamespaceEnabled(); - LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}", - client.getFileSystem(), - path, - isNamespaceEnabled); + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; + String calleeName = null; - final AbfsRestOperation op; - if (path.isRoot()) { - op = isNamespaceEnabled - ? client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + AbfsHttpConstants.ROOT_PATH) - : client.getFilesystemProperties(); - } else { - op = client.getPathStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); - } - - final long blockSize = abfsConfiguration.getAzureBlockSize(); - final AbfsHttpOperation result = op.getResult(); - - final String eTag = result.getResponseHeader(HttpHeaderConfigurations.ETAG); - final String lastModified = result.getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); - final String permissions = result.getResponseHeader((HttpHeaderConfigurations.X_MS_PERMISSIONS)); - final boolean hasAcl = AbfsPermission.isExtendedAcl(permissions); - final long contentLength; - final boolean resourceIsDir; + try { + boolean isNamespaceEnabled = getIsNamespaceEnabled(); + LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}", + client.getFileSystem(), + path, + isNamespaceEnabled); + + final AbfsRestOperation op; + if (path.isRoot()) { + if (isNamespaceEnabled) { + calleeName = "getAclStatus"; + op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + AbfsHttpConstants.ROOT_PATH); + } else { + calleeName = "getFilesystemProperties"; + op = client.getFilesystemProperties(); + } + } else { + calleeName = "getPathStatus"; + op = client.getPathStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + } - if (path.isRoot()) { - contentLength = 0; - resourceIsDir = true; - } else { - contentLength = parseContentLength(result.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); - resourceIsDir = parseIsDirectory(result.getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE)); - } + res = op.getResult(); + final long blockSize = abfsConfiguration.getAzureBlockSize(); + final AbfsHttpOperation result = op.getResult(); + + final String eTag = result.getResponseHeader(HttpHeaderConfigurations.ETAG); + final String lastModified = result.getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); + final String permissions = result.getResponseHeader((HttpHeaderConfigurations.X_MS_PERMISSIONS)); + final boolean hasAcl = AbfsPermission.isExtendedAcl(permissions); + final long contentLength; + final boolean resourceIsDir; + + if (path.isRoot()) { + contentLength = 0; + resourceIsDir = true; + } else { + contentLength = parseContentLength(result.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); + resourceIsDir = parseIsDirectory(result.getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE)); + } - final String transformedOwner = identityTransformer.transformIdentityForGetRequest( + final String transformedOwner = identityTransformer.transformIdentityForGetRequest( result.getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER), true, userName); - final String transformedGroup = identityTransformer.transformIdentityForGetRequest( + final String transformedGroup = identityTransformer.transformIdentityForGetRequest( result.getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP), false, primaryUserGroup); - return new VersionedFileStatus( - transformedOwner, - transformedGroup, - permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) - : AbfsPermission.valueOf(permissions), - hasAcl, - contentLength, - resourceIsDir, - 1, - blockSize, - parseLastModifiedTime(lastModified), - path, - eTag); + success = true; + + return new VersionedFileStatus( + transformedOwner, + transformedGroup, + permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) + : AbfsPermission.valueOf(permissions), + hasAcl, + contentLength, + resourceIsDir, + 1, + blockSize, + parseLastModifiedTime(lastModified), + path, + eTag); + } finally { + latencyTracker.recordClientLatency(start, "getFileStatus", calleeName, success, res); + } } /** @@ -559,6 +735,10 @@ public FileStatus[] listStatus(final Path path) throws IOException { * */ @InterfaceStability.Unstable public FileStatus[] listStatus(final Path path, final String startFrom) throws IOException { + final Instant startAggregate = latencyTracker.getLatencyInstant(); + long countAggregate = 0; + boolean shouldContinue = true; + LOG.debug("listStatus filesystem: {} path: {}, startFrom: {}", client.getFileSystem(), path, @@ -576,53 +756,69 @@ public FileStatus[] listStatus(final Path path, final String startFrom) throws I ArrayList fileStatuses = new ArrayList<>(); do { - AbfsRestOperation op = client.listPath(relativePath, false, LIST_MAX_RESULTS, continuation); - continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); - ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); - if (retrievedSchema == null) { - throw new AbfsRestOperationException( - AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), - AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), - "listStatusAsync path not found", - null, op.getResult()); - } + Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - long blockSize = abfsConfiguration.getAzureBlockSize(); - - for (ListResultEntrySchema entry : retrievedSchema.paths()) { - final String owner = identityTransformer.transformIdentityForGetRequest(entry.owner(), true, userName); - final String group = identityTransformer.transformIdentityForGetRequest(entry.group(), false, primaryUserGroup); - final FsPermission fsPermission = entry.permissions() == null - ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) - : AbfsPermission.valueOf(entry.permissions()); - final boolean hasAcl = AbfsPermission.isExtendedAcl(entry.permissions()); - - long lastModifiedMillis = 0; - long contentLength = entry.contentLength() == null ? 0 : entry.contentLength(); - boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory(); - if (entry.lastModified() != null && !entry.lastModified().isEmpty()) { - lastModifiedMillis = parseLastModifiedTime(entry.lastModified()); + try { + AbfsRestOperation op = client.listPath(relativePath, false, LIST_MAX_RESULTS, continuation); + res = op.getResult(); + continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); + if (retrievedSchema == null) { + throw new AbfsRestOperationException( + AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + "listStatusAsync path not found", + null, op.getResult()); } - Path entryPath = new Path(File.separator + entry.name()); - entryPath = entryPath.makeQualified(this.uri, entryPath); - - fileStatuses.add( - new VersionedFileStatus( - owner, - group, - fsPermission, - hasAcl, - contentLength, - isDirectory, - 1, - blockSize, - lastModifiedMillis, - entryPath, - entry.eTag())); - } + long blockSize = abfsConfiguration.getAzureBlockSize(); + + for (ListResultEntrySchema entry : retrievedSchema.paths()) { + final String owner = identityTransformer.transformIdentityForGetRequest(entry.owner(), true, userName); + final String group = identityTransformer.transformIdentityForGetRequest(entry.group(), false, primaryUserGroup); + final FsPermission fsPermission = entry.permissions() == null + ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) + : AbfsPermission.valueOf(entry.permissions()); + final boolean hasAcl = AbfsPermission.isExtendedAcl(entry.permissions()); + + long lastModifiedMillis = 0; + long contentLength = entry.contentLength() == null ? 0 : entry.contentLength(); + boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory(); + if (entry.lastModified() != null && !entry.lastModified().isEmpty()) { + lastModifiedMillis = parseLastModifiedTime(entry.lastModified()); + } - } while (continuation != null && !continuation.isEmpty()); + Path entryPath = new Path(File.separator + entry.name()); + entryPath = entryPath.makeQualified(this.uri, entryPath); + + fileStatuses.add( + new VersionedFileStatus( + owner, + group, + fsPermission, + hasAcl, + contentLength, + isDirectory, + 1, + blockSize, + lastModifiedMillis, + entryPath, + entry.eTag())); + } + + success = true; + countAggregate++; + shouldContinue = continuation != null && !continuation.isEmpty(); + } finally { + if (shouldContinue) { + latencyTracker.recordClientLatency(start, "listStatus", "listPath", success, res); + } else { + latencyTracker.recordClientLatency(start, "listStatus", "listPath", success, startAggregate, countAggregate, res); + } + } + } while (shouldContinue); return fileStatuses.toArray(new FileStatus[fileStatuses.size()]); } @@ -684,212 +880,361 @@ private String generateContinuationTokenForNonXns(final String path, final Strin public void setOwner(final Path path, final String owner, final String group) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { - throw new UnsupportedOperationException( - "This operation is only valid for storage accounts with the hierarchical namespace enabled."); - } + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - LOG.debug( - "setOwner filesystem: {} path: {} owner: {} group: {}", - client.getFileSystem(), - path.toString(), - owner, - group); + try { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "setOwner filesystem: {} path: {} owner: {} group: {}", + client.getFileSystem(), + path.toString(), + owner, + group); - final String transformedOwner = identityTransformer.transformUserOrGroupForSetRequest(owner); - final String transformedGroup = identityTransformer.transformUserOrGroupForSetRequest(group); + final String transformedOwner = identityTransformer.transformUserOrGroupForSetRequest(owner); + final String transformedGroup = identityTransformer.transformUserOrGroupForSetRequest(group); - client.setOwner(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), transformedOwner, transformedGroup); + final AbfsRestOperation op = client.setOwner(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), transformedOwner, transformedGroup); + + res = op.getResult(); + + success = true; + } finally { + latencyTracker.recordClientLatency(start, "setOwner", "setOwner", success, res); + } } public void setPermission(final Path path, final FsPermission permission) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { - throw new UnsupportedOperationException( - "This operation is only valid for storage accounts with the hierarchical namespace enabled."); - } + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; - LOG.debug( - "setPermission filesystem: {} path: {} permission: {}", - client.getFileSystem(), - path.toString(), - permission.toString()); - client.setPermission(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), - String.format(AbfsHttpConstants.PERMISSION_FORMAT, permission.toOctal())); + try { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "setPermission filesystem: {} path: {} permission: {}", + client.getFileSystem(), + path.toString(), + permission.toString()); + final AbfsRestOperation op = client.setPermission(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + String.format(AbfsHttpConstants.PERMISSION_FORMAT, permission.toOctal())); + + res = op.getResult(); + + success = true; + } finally { + latencyTracker.recordClientLatency(start, "setPermission", "setPermission", success, res); + } } public void modifyAclEntries(final Path path, final List aclSpec) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { - throw new UnsupportedOperationException( - "This operation is only valid for storage accounts with the hierarchical namespace enabled."); - } + final Instant startAggregate = latencyTracker.getLatencyInstant(); + long countAggregate = 0; + Instant startSet = null; - LOG.debug( - "modifyAclEntries filesystem: {} path: {} aclSpec: {}", - client.getFileSystem(), - path.toString(), - AclEntry.aclSpecToString(aclSpec)); + boolean successGet = false; + boolean successSet = false; + AbfsHttpOperation resultGet = null; + AbfsHttpOperation resultSet = null; + + try { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "modifyAclEntries filesystem: {} path: {} aclSpec: {}", + client.getFileSystem(), + path.toString(), + AclEntry.aclSpecToString(aclSpec)); - identityTransformer.transformAclEntriesForSetRequest(aclSpec); - final Map modifyAclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); - boolean useUpn = AbfsAclHelper.isUpnFormatAclEntries(modifyAclEntries); + identityTransformer.transformAclEntriesForSetRequest(aclSpec); + final Map modifyAclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); + boolean useUpn = AbfsAclHelper.isUpnFormatAclEntries(modifyAclEntries); - final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), useUpn); - final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), useUpn); + resultGet = op.getResult(); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); - final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); + final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); - AbfsAclHelper.modifyAclEntriesInternal(aclEntries, modifyAclEntries); + AbfsAclHelper.modifyAclEntriesInternal(aclEntries, modifyAclEntries); - client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), - AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + successGet = true; + countAggregate++; + startSet = latencyTracker.getLatencyInstant(); + + final AbfsRestOperation setAclOp = client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + resultSet = setAclOp.getResult(); + + successSet = true; + countAggregate++; + } finally { + latencyTracker.recordClientLatency(startAggregate, startSet, "modifyAclEntries", "getAclStatus", successGet, resultGet); + latencyTracker.recordClientLatency(startSet, "modifyAclEntries", "setAcl", successSet, startAggregate, countAggregate, resultSet); + } } public void removeAclEntries(final Path path, final List aclSpec) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { - throw new UnsupportedOperationException( - "This operation is only valid for storage accounts with the hierarchical namespace enabled."); - } + final Instant startAggregate = latencyTracker.getLatencyInstant(); + long countAggregate = 0; + Instant startSet = null; - LOG.debug( - "removeAclEntries filesystem: {} path: {} aclSpec: {}", - client.getFileSystem(), - path.toString(), - AclEntry.aclSpecToString(aclSpec)); + boolean successGet = false; + boolean successSet = false; + AbfsHttpOperation resultGet = null; + AbfsHttpOperation resultSet = null; - identityTransformer.transformAclEntriesForSetRequest(aclSpec); - final Map removeAclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); - boolean isUpnFormat = AbfsAclHelper.isUpnFormatAclEntries(removeAclEntries); + try { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "removeAclEntries filesystem: {} path: {} aclSpec: {}", + client.getFileSystem(), + path.toString(), + AclEntry.aclSpecToString(aclSpec)); + + identityTransformer.transformAclEntriesForSetRequest(aclSpec); + final Map removeAclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); + boolean isUpnFormat = AbfsAclHelper.isUpnFormatAclEntries(removeAclEntries); - final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), isUpnFormat); - final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), isUpnFormat); + resultGet = op.getResult(); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); - final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); + final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); - AbfsAclHelper.removeAclEntriesInternal(aclEntries, removeAclEntries); + AbfsAclHelper.removeAclEntriesInternal(aclEntries, removeAclEntries); - client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), - AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + successGet = true; + countAggregate++; + startSet = latencyTracker.getLatencyInstant(); + + final AbfsRestOperation setAclOp = client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + resultSet = setAclOp.getResult(); + + successSet = true; + countAggregate++; + } finally { + latencyTracker.recordClientLatency(startAggregate, startSet, "removeAclEntries", "getAclStatus", successGet, resultGet); + latencyTracker.recordClientLatency(startSet, "removeAclEntries", "setAcl", successSet, startAggregate, countAggregate, resultSet); + } } public void removeDefaultAcl(final Path path) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { - throw new UnsupportedOperationException( - "This operation is only valid for storage accounts with the hierarchical namespace enabled."); - } + final Instant startAggregate = latencyTracker.getLatencyInstant(); + long countAggregate = 0; + Instant startSet = null; - LOG.debug( - "removeDefaultAcl filesystem: {} path: {}", - client.getFileSystem(), - path.toString()); + boolean successGet = false; + boolean successSet = false; + AbfsHttpOperation resultGet = null; + AbfsHttpOperation resultSet = null; + + try { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "removeDefaultAcl filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); - final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); - final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); - final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); - final Map defaultAclEntries = new HashMap<>(); + final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); + resultGet = op.getResult(); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); + final Map defaultAclEntries = new HashMap<>(); - for (Map.Entry aclEntry : aclEntries.entrySet()) { - if (aclEntry.getKey().startsWith("default:")) { - defaultAclEntries.put(aclEntry.getKey(), aclEntry.getValue()); + for (Map.Entry aclEntry : aclEntries.entrySet()) { + if (aclEntry.getKey().startsWith("default:")) { + defaultAclEntries.put(aclEntry.getKey(), aclEntry.getValue()); + } } - } - aclEntries.keySet().removeAll(defaultAclEntries.keySet()); + aclEntries.keySet().removeAll(defaultAclEntries.keySet()); + + successGet = true; + countAggregate++; + startSet = latencyTracker.getLatencyInstant(); - client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), - AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + final AbfsRestOperation setAclOp = client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + resultSet = setAclOp.getResult(); + + successSet = true; + countAggregate++; + } finally { + latencyTracker.recordClientLatency(startAggregate, startSet, "removeDefaultAcl", "getAclStatus", successGet, resultGet); + latencyTracker.recordClientLatency(startSet, "removeDefaultAcl", "setAcl", successSet, startAggregate, countAggregate, resultSet); + } } public void removeAcl(final Path path) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { - throw new UnsupportedOperationException( - "This operation is only valid for storage accounts with the hierarchical namespace enabled."); - } + final Instant startAggregate = latencyTracker.getLatencyInstant(); + long countAggregate = 0; + Instant startSet = null; - LOG.debug( - "removeAcl filesystem: {} path: {}", - client.getFileSystem(), - path.toString()); - final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); - final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + boolean successGet = false; + boolean successSet = false; + AbfsHttpOperation resultGet = null; + AbfsHttpOperation resultSet = null; + + try { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "removeAcl filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); + final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); + resultGet = op.getResult(); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + + final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); + final Map newAclEntries = new HashMap<>(); - final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); - final Map newAclEntries = new HashMap<>(); + newAclEntries.put(AbfsHttpConstants.ACCESS_USER, aclEntries.get(AbfsHttpConstants.ACCESS_USER)); + newAclEntries.put(AbfsHttpConstants.ACCESS_GROUP, aclEntries.get(AbfsHttpConstants.ACCESS_GROUP)); + newAclEntries.put(AbfsHttpConstants.ACCESS_OTHER, aclEntries.get(AbfsHttpConstants.ACCESS_OTHER)); - newAclEntries.put(AbfsHttpConstants.ACCESS_USER, aclEntries.get(AbfsHttpConstants.ACCESS_USER)); - newAclEntries.put(AbfsHttpConstants.ACCESS_GROUP, aclEntries.get(AbfsHttpConstants.ACCESS_GROUP)); - newAclEntries.put(AbfsHttpConstants.ACCESS_OTHER, aclEntries.get(AbfsHttpConstants.ACCESS_OTHER)); + successGet = true; + countAggregate++; + startSet = latencyTracker.getLatencyInstant(); - client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), - AbfsAclHelper.serializeAclSpec(newAclEntries), eTag); + final AbfsRestOperation setAclOp = client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + AbfsAclHelper.serializeAclSpec(newAclEntries), eTag); + + resultSet = setAclOp.getResult(); + + successSet = true; + countAggregate++; + } finally { + latencyTracker.recordClientLatency(startAggregate, startSet, "removeAcl", "getAclStatus", successGet, resultGet); + latencyTracker.recordClientLatency(startSet, "removeAcl", "setAcl", successSet, startAggregate, countAggregate, resultSet); + } } public void setAcl(final Path path, final List aclSpec) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { - throw new UnsupportedOperationException( - "This operation is only valid for storage accounts with the hierarchical namespace enabled."); - } + final Instant startAggregate = latencyTracker.getLatencyInstant(); + long countAggregate = 0; + Instant startSet = null; - LOG.debug( - "setAcl filesystem: {} path: {} aclspec: {}", - client.getFileSystem(), - path.toString(), - AclEntry.aclSpecToString(aclSpec)); + boolean successGet = false; + boolean successSet = false; + AbfsHttpOperation resultGet = null; + AbfsHttpOperation resultSet = null; - identityTransformer.transformAclEntriesForSetRequest(aclSpec); - final Map aclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); - final boolean isUpnFormat = AbfsAclHelper.isUpnFormatAclEntries(aclEntries); + try { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "setAcl filesystem: {} path: {} aclspec: {}", + client.getFileSystem(), + path.toString(), + AclEntry.aclSpecToString(aclSpec)); + + identityTransformer.transformAclEntriesForSetRequest(aclSpec); + final Map aclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); + final boolean isUpnFormat = AbfsAclHelper.isUpnFormatAclEntries(aclEntries); - final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), isUpnFormat); - final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), isUpnFormat); + resultGet = op.getResult(); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); - final Map getAclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); + final Map getAclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); - AbfsAclHelper.setAclEntriesInternal(aclEntries, getAclEntries); + AbfsAclHelper.setAclEntriesInternal(aclEntries, getAclEntries); - client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), - AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + startSet = latencyTracker.getLatencyInstant(); + successGet = true; + countAggregate++; + + final AbfsRestOperation setAclOp = client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + resultSet = setAclOp.getResult(); + + successSet = true; + countAggregate++; + } finally { + latencyTracker.recordClientLatency(startAggregate, startSet, "setAcl", "getAclStatus", successGet, resultGet); + latencyTracker.recordClientLatency(startSet, "setAcl", "setAcl", successSet, startAggregate, countAggregate, resultSet); + } } public AclStatus getAclStatus(final Path path) throws IOException { - if (!getIsNamespaceEnabled()) { - throw new UnsupportedOperationException( - "This operation is only valid for storage accounts with the hierarchical namespace enabled."); - } + final Instant start = latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation result = null; - LOG.debug( - "getAclStatus filesystem: {} path: {}", - client.getFileSystem(), - path.toString()); - AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); - AbfsHttpOperation result = op.getResult(); - - final String transformedOwner = identityTransformer.transformIdentityForGetRequest( - result.getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER), - true, - userName); - final String transformedGroup = identityTransformer.transformIdentityForGetRequest( - result.getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP), - false, - primaryUserGroup); - - final String permissions = result.getResponseHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS); - final String aclSpecString = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL); - - final List aclEntries = AclEntry.parseAclSpec(AbfsAclHelper.processAclString(aclSpecString), true); - identityTransformer.transformAclEntriesForGetRequest(aclEntries, userName, primaryUserGroup); - final FsPermission fsPermission = permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) - : AbfsPermission.valueOf(permissions); - - final AclStatus.Builder aclStatusBuilder = new AclStatus.Builder(); - aclStatusBuilder.owner(transformedOwner); - aclStatusBuilder.group(transformedGroup); - - aclStatusBuilder.setPermission(fsPermission); - aclStatusBuilder.stickyBit(fsPermission.getStickyBit()); - aclStatusBuilder.addEntries(aclEntries); - return aclStatusBuilder.build(); + try { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "getAclStatus filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); + AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); + result = op.getResult(); + + final String transformedOwner = identityTransformer.transformIdentityForGetRequest( + result.getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER), + true, + userName); + final String transformedGroup = identityTransformer.transformIdentityForGetRequest( + result.getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP), + false, + primaryUserGroup); + + final String permissions = result.getResponseHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS); + final String aclSpecString = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL); + + final List aclEntries = AclEntry.parseAclSpec(AbfsAclHelper.processAclString(aclSpecString), true); + identityTransformer.transformAclEntriesForGetRequest(aclEntries, userName, primaryUserGroup); + final FsPermission fsPermission = permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) + : AbfsPermission.valueOf(permissions); + + final AclStatus.Builder aclStatusBuilder = new AclStatus.Builder(); + aclStatusBuilder.owner(transformedOwner); + aclStatusBuilder.group(transformedGroup); + + aclStatusBuilder.setPermission(fsPermission); + aclStatusBuilder.stickyBit(fsPermission.getStickyBit()); + aclStatusBuilder.addEntries(aclEntries); + success = true; + return aclStatusBuilder.build(); + } finally { + latencyTracker.recordClientLatency(start, "getAclStatus", "getAclStatus", success, result); + } } public boolean isAtomicRenameKey(String key) { @@ -930,7 +1275,7 @@ private void initializeClient(URI uri, String fileSystemName, String accountName abfsConfiguration.getRawConfiguration()); } - this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy(), tokenProvider); + this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy(), tokenProvider, latencyTracker); } private String getOctalNotation(FsPermission fsPermission) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AuthConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AuthConfigurations.java new file mode 100644 index 0000000000000..4fd8ddf0b4fe3 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AuthConfigurations.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

        + * http://www.apache.org/licenses/LICENSE-2.0 + *

        + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.constants; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Responsible to keep all the Azure Blob File System auth related + * configurations. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class AuthConfigurations { + + /** Default OAuth token end point for the MSI flow. */ + public static final String DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT = + "http://169.254.169.254/metadata/identity/oauth2/token"; + /** Default value for authority for the MSI flow. */ + public static final String DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY = + "https://login.microsoftonline.com/"; + /** Default OAuth token end point for the refresh token flow. */ + public static final String + DEFAULT_FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN_ENDPOINT = + "https://login.microsoftonline.com/Common/oauth2/token"; + + private AuthConfigurations() { + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index cd86f184fcf10..409ffc3c1240c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -102,12 +102,20 @@ public final class ConfigurationKeys { public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT = "fs.azure.account.oauth2.client.endpoint"; /** Key for oauth msi tenant id: {@value}. */ public static final String FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT = "fs.azure.account.oauth2.msi.tenant"; + /** Key for oauth msi endpoint: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT = "fs.azure.account.oauth2.msi.endpoint"; + /** Key for oauth msi Authority: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY = "fs.azure.account.oauth2.msi.authority"; /** Key for oauth user name: {@value}. */ public static final String FS_AZURE_ACCOUNT_OAUTH_USER_NAME = "fs.azure.account.oauth2.user.name"; /** Key for oauth user password: {@value}. */ public static final String FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD = "fs.azure.account.oauth2.user.password"; /** Key for oauth refresh token: {@value}. */ public static final String FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN = "fs.azure.account.oauth2.refresh.token"; + /** Key for oauth AAD refresh token endpoint: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN_ENDPOINT = "fs.azure.account.oauth2.refresh.token.endpoint"; + /** Key for enabling the tracking of ABFS API latency and sending the latency numbers to the ABFS API service */ + public static final String FS_AZURE_ABFS_LATENCY_TRACK = "fs.azure.abfs.latency.track"; public static String accountProperty(String property, String account) { return property + "." + account; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index e0c355a07b2d3..3a4385ea3c4fe 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -67,6 +67,7 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_ENABLE_HTTPS = true; public static final boolean DEFAULT_USE_UPN = false; + public static final boolean DEFAULT_ABFS_LATENCY_TRACK = false; private FileSystemConfigurations() {} } \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java index 6204a957e395e..3de312865525a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java @@ -101,6 +101,9 @@ public static AzureADToken getTokenUsingClientCreds(String authEndpoint, * an Azure VM with MSI extension * enabled. * + * @param authEndpoint the OAuth 2.0 token endpoint associated + * with the user's directory (obtain from + * Active Directory configuration) * @param tenantGuid (optional) The guid of the AAD tenant. Can be {@code null}. * @param clientId (optional) The clientId guid of the MSI service * principal to use. Can be {@code null}. @@ -109,17 +112,16 @@ public static AzureADToken getTokenUsingClientCreds(String authEndpoint, * @return {@link AzureADToken} obtained using the creds * @throws IOException throws IOException if there is a failure in obtaining the token */ - public static AzureADToken getTokenFromMsi(String tenantGuid, String clientId, - boolean bypassCache) throws IOException { - String authEndpoint = "http://169.254.169.254/metadata/identity/oauth2/token"; - + public static AzureADToken getTokenFromMsi(final String authEndpoint, + final String tenantGuid, final String clientId, String authority, + boolean bypassCache) throws IOException { QueryParams qp = new QueryParams(); qp.add("api-version", "2018-02-01"); qp.add("resource", RESOURCE_NAME); - if (tenantGuid != null && tenantGuid.length() > 0) { - String authority = "https://login.microsoftonline.com/" + tenantGuid; + authority = authority + tenantGuid; + LOG.debug("MSI authority : {}", authority); qp.add("authority", authority); } @@ -141,14 +143,17 @@ public static AzureADToken getTokenFromMsi(String tenantGuid, String clientId, /** * Gets Azure Active Directory token using refresh token. * + * @param authEndpoint the OAuth 2.0 token endpoint associated + * with the user's directory (obtain from + * Active Directory configuration) * @param clientId the client ID (GUID) of the client web app obtained from Azure Active Directory configuration * @param refreshToken the refresh token * @return {@link AzureADToken} obtained using the refresh token * @throws IOException throws IOException if there is a failure in connecting to Azure AD */ - public static AzureADToken getTokenUsingRefreshToken(String clientId, - String refreshToken) throws IOException { - String authEndpoint = "https://login.microsoftonline.com/Common/oauth2/token"; + public static AzureADToken getTokenUsingRefreshToken( + final String authEndpoint, final String clientId, + final String refreshToken) throws IOException { QueryParams qp = new QueryParams(); qp.add("grant_type", "refresh_token"); qp.add("refresh_token", refreshToken); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/MsiTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/MsiTokenProvider.java index 2deb9d246d1bd..38f3045580824 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/MsiTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/MsiTokenProvider.java @@ -28,21 +28,29 @@ */ public class MsiTokenProvider extends AccessTokenProvider { + private final String authEndpoint; + + private final String authority; + private final String tenantGuid; private final String clientId; private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class); - public MsiTokenProvider(final String tenantGuid, final String clientId) { + public MsiTokenProvider(final String authEndpoint, final String tenantGuid, + final String clientId, final String authority) { + this.authEndpoint = authEndpoint; this.tenantGuid = tenantGuid; this.clientId = clientId; + this.authority = authority; } @Override protected AzureADToken refreshToken() throws IOException { LOG.debug("AADToken: refreshing token from MSI"); - AzureADToken token = AzureADAuthenticator.getTokenFromMsi(tenantGuid, clientId, false); + AzureADToken token = AzureADAuthenticator + .getTokenFromMsi(authEndpoint, tenantGuid, clientId, authority, false); return token; } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java index 949d5bf1d80b2..1c1bd2b3b5511 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java @@ -31,6 +31,8 @@ public class RefreshTokenBasedTokenProvider extends AccessTokenProvider { private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class); + private final String authEndpoint; + private final String clientId; private final String refreshToken; @@ -41,9 +43,12 @@ public class RefreshTokenBasedTokenProvider extends AccessTokenProvider { * @param clientId the client ID (GUID) of the client web app obtained from Azure Active Directory configuration * @param refreshToken the refresh token */ - public RefreshTokenBasedTokenProvider(String clientId, String refreshToken) { + public RefreshTokenBasedTokenProvider(final String authEndpoint, + String clientId, String refreshToken) { + Preconditions.checkNotNull(authEndpoint, "authEndpoint"); Preconditions.checkNotNull(clientId, "clientId"); Preconditions.checkNotNull(refreshToken, "refreshToken"); + this.authEndpoint = authEndpoint; this.clientId = clientId; this.refreshToken = refreshToken; } @@ -52,6 +57,7 @@ public RefreshTokenBasedTokenProvider(String clientId, String refreshToken) { @Override protected AzureADToken refreshToken() throws IOException { LOG.debug("AADToken: refreshing refresh-token based token"); - return AzureADAuthenticator.getTokenUsingRefreshToken(clientId, refreshToken); + return AzureADAuthenticator + .getTokenUsingRefreshToken(authEndpoint, clientId, refreshToken); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index edcd1972c23b7..ce107cb4fb307 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -60,6 +60,7 @@ public class AbfsClient implements Closeable { private final String filesystem; private final AbfsConfiguration abfsConfiguration; private final String userAgent; + protected final LatencyTracker latencyTracker; private final AccessTokenProvider tokenProvider; @@ -67,7 +68,8 @@ public class AbfsClient implements Closeable { public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, final ExponentialRetryPolicy exponentialRetryPolicy, - final AccessTokenProvider tokenProvider) { + final AccessTokenProvider tokenProvider, + final LatencyTracker latencyTracker) { this.baseUrl = baseUrl; this.sharedKeyCredentials = sharedKeyCredentials; String baseUrlString = baseUrl.toString(); @@ -88,6 +90,7 @@ public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredent this.userAgent = initializeUserAgent(abfsConfiguration, sslProviderName); this.tokenProvider = tokenProvider; + this.latencyTracker = latencyTracker; } @Override diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 5579877b5f623..94cebf0dc541b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -21,8 +21,10 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.io.UnsupportedEncodingException; import java.net.HttpURLConnection; import java.net.URL; +import java.net.URLEncoder; import java.util.List; import java.util.UUID; @@ -161,6 +163,44 @@ public String toString() { return sb.toString(); } + // Returns a trace message for the ABFS API logging service to consume + public String toKvpString() { + String urlStr = null; + + try{ + urlStr = URLEncoder.encode(url.toString(), "UTF-8"); + } catch(UnsupportedEncodingException e) { + urlStr = "https%3A%2F%2Ffailed%2Fto%2Fencode%2Furl"; + } + + final StringBuilder sb = new StringBuilder(); + sb.append("s="); + sb.append(statusCode); + sb.append(" e="); + sb.append(storageErrorCode); + sb.append(" ci="); + sb.append(clientRequestId); + sb.append(" ri="); + sb.append(requestId); + if (isTraceEnabled) { + sb.append(" ct="); + sb.append(connectionTimeMs); + sb.append(" st="); + sb.append(sendRequestTimeMs); + sb.append(" rt="); + sb.append(recvResponseTimeMs); + } + sb.append(" bs="); + sb.append(bytesSent); + sb.append(" br="); + sb.append(bytesReceived); + sb.append(" m="); + sb.append(method); + sb.append(" u="); + sb.append(urlStr); + return sb.toString(); + } + /** * Initializes a new HTTP request and opens the connection. * diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index fe48cb9323712..ee003a2be7cf9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -22,6 +22,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.net.HttpURLConnection; +import java.time.Instant; import com.google.common.base.Preconditions; @@ -226,8 +227,13 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti throw new IllegalArgumentException("requested read length is more than will fit after requested offset in buffer"); } final AbfsRestOperation op; + final Instant start = client.latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; try { op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag); + res = op.getResult(); + success = true; } catch (AzureBlobFileSystemException ex) { if (ex instanceof AbfsRestOperationException) { AbfsRestOperationException ere = (AbfsRestOperationException) ex; @@ -236,6 +242,8 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti } } throw new IOException(ex); + } finally { + client.latencyTracker.recordClientLatency(start, "readRemote", "read", success, res); } long bytesRead = op.getResult().getBytesReceived(); if (bytesRead > Integer.MAX_VALUE) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index fd56eb0a015b9..c2703d78ab274 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -23,6 +23,7 @@ import java.io.InterruptedIOException; import java.io.OutputStream; import java.net.HttpURLConnection; +import java.time.Instant; import java.nio.ByteBuffer; import java.util.Locale; import java.util.concurrent.ConcurrentLinkedDeque; @@ -289,10 +290,18 @@ private synchronized void writeCurrentBufferToService() throws IOException { final Future job = completionService.submit(new Callable() { @Override public Void call() throws Exception { - client.append(path, offset, bytes, 0, - bytesLength); - byteBufferPool.putBuffer(ByteBuffer.wrap(bytes)); - return null; + final Instant start = client.latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; + try { + res = client.append(path, offset, bytes, 0, + bytesLength).getResult(); + byteBufferPool.putBuffer(ByteBuffer.wrap(bytes)); + success = true; + return null; + } finally { + client.latencyTracker.recordClientLatency(start, "writeCurrentBufferToService", "append", success, res); + } } }); @@ -334,8 +343,13 @@ private synchronized void flushWrittenBytesToServiceAsync() throws IOException { private synchronized void flushWrittenBytesToServiceInternal(final long offset, final boolean retainUncommitedData, final boolean isClose) throws IOException { + final Instant start = client.latencyTracker.getLatencyInstant(); + boolean success = false; + AbfsHttpOperation res = null; + try { - client.flush(path, offset, retainUncommitedData, isClose); + res = client.flush(path, offset, retainUncommitedData, isClose).getResult(); + success = true; } catch (AzureBlobFileSystemException ex) { if (ex instanceof AbfsRestOperationException) { if (((AbfsRestOperationException) ex).getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { @@ -343,6 +357,8 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, } } throw new IOException(ex); + } finally { + client.latencyTracker.recordClientLatency(start, "flushWrittenBytesToServiceInternal", "flush", success, res); } this.lastFlushOffset = offset; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java index 4196c10ed4728..cfecaac410e2d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java @@ -121,7 +121,15 @@ public AbfsHttpOperation getResult() { * HTTP operations. */ void execute() throws AzureBlobFileSystemException { + // see if we have latency reports from the previous requests + String latencyHeader = this.client.latencyTracker.getClientLatency(); + + if(latencyHeader != null && !latencyHeader.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader("x-ms-abfs-client-latency", latencyHeader)); + } + int retryCount = 0; + while (!executeHttpOperation(retryCount++)) { try { Thread.sleep(client.getRetryPolicy().getRetryInterval(retryCount)); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LatencyTracker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LatencyTracker.java new file mode 100644 index 0000000000000..0d2978abfe865 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LatencyTracker.java @@ -0,0 +1,203 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.services; + +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.time.Duration; +import java.time.Instant; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.ConcurrentLinkedQueue; + + +/** + * {@code LatencyTracker} keeps track of service latencies observed by {@code AbfsClient}. Every request adds + * its information (success/failure, latency etc) to the {@code LatencyTracker}'s queue. + * When a request is made, we check {@code LatencyTracker} to see if there are any latency numbers to be reported. + * If there are any, the stats are added to an HTTP header ({@code x-ms-abfs-client-latency}) on the next request. * + */ +public class LatencyTracker { + + // the logger + private static final Logger LOG = LoggerFactory.getLogger(LatencyTracker.class); + + // the queue to hold latency information + private final ConcurrentLinkedQueue Q = new ConcurrentLinkedQueue(); + + // whether the latency tracker has been enabled + private boolean enabled = false; + + // the host name + private String hostName; + + // the file system name + private String filesystemName; + + // the account name + private String accountName; + + // singleton latency reporting format + private String singletonLatencyReportingFormat; + + // aggregate latency reporting format + private String aggregateLatencyReportingFormat; + + public LatencyTracker(String filesystemName, String accountName, AbfsConfiguration configuration) { + this(filesystemName, accountName, configuration.shouldTrackLatency()); + } + + protected LatencyTracker(String filesystemName, String accountName, boolean enabled) { + this.enabled = enabled; + this.filesystemName = filesystemName; + this.accountName = accountName; + + LOG.debug("LatencyTracker configuration: {}", enabled); + + if(enabled) { + try { + hostName = InetAddress.getLocalHost().getHostName(); + } catch (UnknownHostException e) { + hostName = "UnknownHost"; + } + + singletonLatencyReportingFormat = "h=" + hostName + " t=%s a=" + accountName + " c=" + filesystemName + " cr=%s ce=%s r=%s l=%s%s"; + aggregateLatencyReportingFormat = "h=" + hostName + " t=%s a=" + accountName + " c=" + filesystemName + " cr=%s ce=%s r=%s l=%s ls=%s lc=%s%s"; + } + } + + public void recordClientLatency( + Instant operationStart, + String callerName, + String calleeName, + boolean success, + AbfsHttpOperation res) { + if(!enabled) return; + + Instant operationStop = getLatencyInstant(); + + recordClientLatency(operationStart, operationStop, callerName, calleeName, success, res); + } + + public void recordClientLatency( + Instant operationStart, + Instant operationStop, + String callerName, + String calleeName, + boolean success, + AbfsHttpOperation res) { + if(!enabled) return; + + Instant trackerStart = Instant.now(); + long latency = isValidInstant(operationStart) && isValidInstant(operationStop) ? + Duration.between(operationStart, operationStop).toMillis() : -1; + + String latencyDetails = String.format(singletonLatencyReportingFormat, + Instant.now(), + callerName, + calleeName, + success ? "Succeeded" : "Failed", + latency, + res == null ? "" : (" " + res.toKvpString())); + + this.offerToQueue(trackerStart, latencyDetails); + } + + public void recordClientLatency( + Instant operationStart, + String callerName, + String calleeName, + boolean success, + Instant aggregateStart, + long aggregateCount, + AbfsHttpOperation res) { + if(!enabled) return; + + Instant operationStop = getLatencyInstant(); + + recordClientLatency(operationStart, operationStop, callerName, calleeName, success, aggregateStart, aggregateCount, res); + } + + public void recordClientLatency( + Instant operationStart, + Instant operationStop, + String callerName, + String calleeName, + boolean success, + Instant aggregateStart, + long aggregateCount, + AbfsHttpOperation res){ + if(!enabled) return; + + Instant trackerStart = Instant.now(); + long latency = isValidInstant(operationStart) && isValidInstant(operationStop) ? + Duration.between(operationStart, operationStop).toMillis() : -1; + long aggregateLatency = isValidInstant(aggregateStart) && isValidInstant(operationStop) ? + Duration.between(aggregateStart, operationStop).toMillis() : -1; + + String latencyDetails = String.format(aggregateLatencyReportingFormat, + Instant.now(), + callerName, + calleeName, + success ? "Succeeded" : "Failed", + latency, + aggregateLatency, + aggregateCount, + res == null ? "" : (" " + res.toKvpString())); + + offerToQueue(trackerStart, latencyDetails); + } + + public String getClientLatency() { + if (!enabled) return null; + + Instant trackerStart = Instant.now(); + String latencyDetails = Q.poll(); // non-blocking pop + + if(LOG.isDebugEnabled()) { + Instant stop = Instant.now(); + long elapsed = Duration.between(trackerStart, stop).toMillis(); + LOG.debug(String.format("Dequeued latency info [%s ms]: %s", elapsed, latencyDetails)); + } + + return latencyDetails; + } + + public Instant getLatencyInstant() { + if (!enabled) return null; + return Instant.now(); + } + + private void offerToQueue(Instant trackerStart, String latencyDetails) { + Q.offer(latencyDetails); // non-blocking append + + if(LOG.isDebugEnabled()) { + Instant trackerStop = Instant.now(); + long elapsed = Duration.between(trackerStart, trackerStop).toMillis(); + LOG.debug(String.format("Queued latency info [%s ms]: %s", elapsed, latencyDetails)); + } + } + + private boolean isValidInstant(Instant testInstant) { + return testInstant != null && testInstant != Instant.MIN && testInstant != Instant.MAX; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index b9a2315669626..74c8803e4f6a8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.fs.azurebfs; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.UUID; @@ -24,6 +25,7 @@ import org.junit.Test; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.services.AuthType; @@ -67,10 +69,10 @@ public void testFailedRequestWhenFSNotExist() throws Exception { + testUri.substring(testUri.indexOf("@")); AzureBlobFileSystem fs = this.getFileSystem(nonExistingFsUrl); - intercept(AbfsRestOperationException.class, + intercept(FileNotFoundException.class, "\"The specified filesystem does not exist.\", 404", ()-> { - fs.getIsNamespaceEnabled(); + fs.getFileStatus(new Path("/")); // Run a dummy FS call }); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 228e385410a7b..7df9fb1a35289 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -43,7 +43,7 @@ private void validateUserAgent(String expectedPattern, AbfsConfiguration config, boolean includeSSLProvider) { AbfsClient client = new AbfsClient(baseUrl, null, - config, null, null); + config, null, null, null); String sslProviderName = null; if (includeSSLProvider) { sslProviderName = DelegatingSSLSocketFactory.getDefaultFactory().getProviderName(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestLatencyTracker.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestLatencyTracker.java new file mode 100644 index 0000000000000..4f3d17fa10980 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestLatencyTracker.java @@ -0,0 +1,473 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.services; + +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URL; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.regex.*; + +/** + * Test the latency tracker for abfs + * + */ +public final class TestLatencyTracker { + private static final Logger LOG = LoggerFactory.getLogger(TestLatencyTracker.class); + private final String filesystemName = "bogusFilesystemName"; + private final String accountName = "bogusAccountName"; + private final URL url; + + public TestLatencyTracker() throws Exception + { + this.url = new URL("http", "www.microsoft.com", "/bogusFile"); + } + + @Test + public void verifyDisablingOfTracker() throws Exception { + // verify that disabling of the tracker works + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, false); + + String latencyDetails = latencyTracker.getClientLatency(); + Assert.assertNull("LatencyTracker should be empty", latencyDetails); + + latencyTracker.recordClientLatency(Instant.now(), "disablingCaller", "disablingCallee", true, + new AbfsHttpOperation(url, "GET", new ArrayList())); + + latencyDetails = latencyTracker.getClientLatency(); + Assert.assertNull("LatencyTracker should return no record", latencyDetails); + } + + @Test + public void verifyTrackingForSingletonLatencyRecords() throws Exception { + // verify that tracking for singleton latency records works as expected + final int numTasks = 100; + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, true); + + String latencyDetails = latencyTracker.getClientLatency(); + Assert.assertNull("LatencyTracker should be empty", latencyDetails); + + ExecutorService executorService = Executors.newCachedThreadPool(); + List> tasks = new ArrayList>(); + AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList()); + + for(int i=0; i < numTasks; i++) { + Callable c = new Callable() { + @Override + public Integer call() throws Exception { + latencyTracker.recordClientLatency(Instant.now(), "oneOperationCaller", "oneOperationCallee", true, httpOperation); + return 0; + } + }; + tasks.add(c); + } + + for(Future fr: executorService.invokeAll(tasks)) { + fr.get(); + } + + for(int i=0; i < numTasks; i++) { + latencyDetails = latencyTracker.getClientLatency(); + Assert.assertNotNull("LatencyTracker should return non-null record", latencyDetails); + Assert.assertTrue ("Latency record should be in the correct format", Pattern.matches( + "h=[^ ]* t=[^ ]* a=bogusFilesystemName c=bogusAccountName cr=oneOperationCaller ce=oneOperationCallee r=Succeeded l=[0-9]+ s=0 e= ci=[^ ]* ri=[^ ]* bs=0 br=0 m=GET u=http%3A%2F%2Fwww.microsoft.com%2FbogusFile", latencyDetails)); + } + } + + @Test + public void verifyTrackingForAggregateLatencyRecords() throws Exception { + // verify that tracking of aggregate latency records works as expected + final int numTasks = 100; + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, true); + + String latencyDetails = latencyTracker.getClientLatency(); + Assert.assertNull("LatencyTracker should be empty", latencyDetails); + + ExecutorService executorService = Executors.newCachedThreadPool(); + List> tasks = new ArrayList>(); + AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList()); + + for(int i=0; i < numTasks; i++) { + Callable c = new Callable() { + @Override + public Integer call() throws Exception { + // test latency tracking when aggregate latency numbers are also passed + latencyTracker.recordClientLatency(Instant.now(), "oneOperationCaller", "oneOperationCallee", true, Instant.now(), 42, httpOperation); + return 0; + } + }; + tasks.add(c); + } + + for(Future fr: executorService.invokeAll(tasks)) { + fr.get(); + } + + for(int i=0; i < numTasks; i++) { + latencyDetails = latencyTracker.getClientLatency(); + Assert.assertNotNull("LatencyTracker should return non-null record", latencyDetails); + Assert.assertTrue ("Latency record should be in the correct format", Pattern.matches( + "h=[^ ]* t=[^ ]* a=bogusFilesystemName c=bogusAccountName cr=oneOperationCaller ce=oneOperationCallee r=Succeeded l=[0-9]+ ls=[0-9]+ lc=42 s=0 e= ci=[^ ]* ri=[^ ]* bs=0 br=0 m=GET u=http%3A%2F%2Fwww.microsoft.com%2FbogusFile", latencyDetails)); + } + } + + @Test + public void verifyRecordingSingletonLatencyIsCheapWhenDisabled() throws Exception { + // when latency tracker is disabled, we expect it to take time equivalent to checking a boolean value + final long maxLatencyWhenDisabledMs = 1; + final long minLatencyWhenDisabledMs = 0; + final long numTasks = 1000; + long aggregateLatency = 0; + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, false); + + ExecutorService executorService = Executors.newCachedThreadPool(); + List> tasks = new ArrayList>(); + final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList()); + + for(int i=0; i < numTasks; i++) { + Callable c = new Callable() { + @Override + public Long call() throws Exception { + Instant startRecord = Instant.now(); + + try{ + ; // placeholder try block + } finally { + latencyTracker.recordClientLatency(startRecord, "oneOperationCaller", "oneOperationCallee", true, httpOperation); + } + + long latencyRecord = Duration.between(startRecord, Instant.now()).toMillis(); + LOG.debug("Spent {} ms in recording latency.", latencyRecord); + return latencyRecord; + } + }; + tasks.add(c); + } + + for(Future fr: executorService.invokeAll(tasks)) { + aggregateLatency += fr.get(); + } + + double averageRecordLatency = aggregateLatency/numTasks; + Assert.assertTrue(String.format("Average time for recording singleton latencies, %s ms should be in the range [%s, %s).", averageRecordLatency, minLatencyWhenDisabledMs, maxLatencyWhenDisabledMs), + averageRecordLatency < maxLatencyWhenDisabledMs && averageRecordLatency >= minLatencyWhenDisabledMs); + } + + @Test + public void verifyRecordingAggregateLatencyIsCheapWhenDisabled() throws Exception { + // when latency tracker is disabled, we expect it to take time equivalent to checking a boolean value + final long maxLatencyWhenDisabledMs = 1; + final long minLatencyWhenDisabledMs = 0; + final long numTasks = 1000; + long aggregateLatency = 0; + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, false); + + ExecutorService executorService = Executors.newCachedThreadPool(); + List> tasks = new ArrayList>(); + final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList()); + + for(int i=0; i < numTasks; i++) { + Callable c = new Callable() { + @Override + public Long call() throws Exception { + Instant startRecord = Instant.now(); + + try { + // placeholder try block + } finally { + latencyTracker.recordClientLatency(startRecord, "oneOperationCaller", "oneOperationCallee", true, startRecord, 42, httpOperation); + } + + long latencyRecord = Duration.between(startRecord, Instant.now()).toMillis(); + LOG.debug("Spent {} ms in recording latency.", latencyRecord); + return latencyRecord; + } + }; + tasks.add(c); + } + + for(Future fr: executorService.invokeAll(tasks)) { + aggregateLatency += fr.get(); + } + + double averageRecordLatency = aggregateLatency/numTasks; + Assert.assertTrue(String.format("Average time for recording singleton latencies, %s ms should be in the range [%s, %s).", averageRecordLatency, minLatencyWhenDisabledMs, maxLatencyWhenDisabledMs), + averageRecordLatency < maxLatencyWhenDisabledMs && averageRecordLatency >= minLatencyWhenDisabledMs); + } + + @Test + public void verifyGettingLatencyRecordsIsCheapWhenDisabled() throws Exception { + // when latency tracker is disabled, we expect it to take time equivalent to checking a boolean value + final long maxLatencyWhenDisabledMs = 1; + final long minLatencyWhenDisabledMs = 0; + final long numTasks = 1000; + long aggregateLatency = 0; + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, false); + + ExecutorService executorService = Executors.newCachedThreadPool(); + List> tasks = new ArrayList>(); + final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList()); + + for(int i=0; i < numTasks; i++) { + Callable c = new Callable() { + @Override + public Long call() throws Exception { + Instant startGet = Instant.now(); + latencyTracker.getClientLatency(); + long latencyGet = Duration.between(startGet, Instant.now()).toMillis(); + LOG.debug("Spent {} ms in retrieving latency record.", latencyGet); + return latencyGet; + } + }; + tasks.add(c); + } + + for(Future fr: executorService.invokeAll(tasks)) { + aggregateLatency += fr.get(); + } + + double averageRecordLatency = aggregateLatency/numTasks; + Assert.assertTrue(String.format("Average time for getting latency records, %s ms should be in the range [%s, %s).", averageRecordLatency, minLatencyWhenDisabledMs, maxLatencyWhenDisabledMs), + averageRecordLatency < maxLatencyWhenDisabledMs && averageRecordLatency >= minLatencyWhenDisabledMs); + } + + @Test + public void verifyRecordingSingletonLatencyIsCheapWhenEnabled() throws Exception { + final long maxLatencyWhenDisabledMs = 50; + final long minLatencyWhenDisabledMs = 0; + final long numTasks = 1000; + long aggregateLatency = 0; + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, true); + + ExecutorService executorService = Executors.newCachedThreadPool(); + List> tasks = new ArrayList>(); + final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList()); + + for(int i=0; i < numTasks; i++) { + Callable c = new Callable() { + @Override + public Long call() throws Exception { + Instant startRecord = Instant.now(); + + try { + // placeholder try block + } finally { + latencyTracker.recordClientLatency(startRecord, "oneOperationCaller", "oneOperationCallee", true, httpOperation); + } + + long latencyRecord = Duration.between(startRecord, Instant.now()).toMillis(); + LOG.debug("Spent {} ms in recording latency.", latencyRecord); + return latencyRecord; + } + }; + tasks.add(c); + } + + for(Future fr: executorService.invokeAll(tasks)) { + aggregateLatency += fr.get(); + } + + double averageRecordLatency = aggregateLatency/numTasks; + Assert.assertTrue(String.format("Average time for recording singleton latencies, %s ms should be in the range [%s, %s).", averageRecordLatency, minLatencyWhenDisabledMs, maxLatencyWhenDisabledMs), + averageRecordLatency < maxLatencyWhenDisabledMs && averageRecordLatency >= minLatencyWhenDisabledMs); + } + + @Test + public void verifyRecordingAggregateLatencyIsCheapWhenEnabled() throws Exception { + final long maxLatencyWhenDisabledMs = 50; + final long minLatencyWhenDisabledMs = 0; + final long numTasks = 1000; + long aggregateLatency = 0; + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, true); + + ExecutorService executorService = Executors.newCachedThreadPool(); + List> tasks = new ArrayList>(); + final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList()); + + for(int i=0; i < numTasks; i++) { + Callable c = new Callable() { + @Override + public Long call() throws Exception { + Instant startRecord = Instant.now(); + + try { + // placeholder try block + } finally { + latencyTracker.recordClientLatency(startRecord, "oneOperationCaller", "oneOperationCallee", true, startRecord, 42, httpOperation); + } + + long latencyRecord = Duration.between(startRecord, Instant.now()).toMillis(); + LOG.debug("Spent {} ms in recording latency.", latencyRecord); + return latencyRecord; + } + }; + tasks.add(c); + } + + for(Future fr: executorService.invokeAll(tasks)) { + aggregateLatency += fr.get(); + } + + double averageRecordLatency = aggregateLatency/numTasks; + Assert.assertTrue(String.format("Average time for recording singleton latencies, %s ms should be in the range [%s, %s).", averageRecordLatency, minLatencyWhenDisabledMs, maxLatencyWhenDisabledMs), + averageRecordLatency < maxLatencyWhenDisabledMs && averageRecordLatency >= minLatencyWhenDisabledMs); + } + + @Test + public void verifyGettingLatencyRecordsIsCheapWhenEnabled() throws Exception { + final long maxLatencyWhenDisabledMs = 50; + final long minLatencyWhenDisabledMs = 0; + final long numTasks = 1000; + long aggregateLatency = 0; + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, true); + + ExecutorService executorService = Executors.newCachedThreadPool(); + List> tasks = new ArrayList>(); + final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList()); + + for(int i=0; i < numTasks; i++) { + Callable c = new Callable() { + @Override + public Long call() throws Exception { + Instant startRecord = Instant.now(); + latencyTracker.getClientLatency(); + long latencyRecord = Duration.between(startRecord, Instant.now()).toMillis(); + LOG.debug("Spent {} ms in recording latency.", latencyRecord); + return latencyRecord; + } + }; + tasks.add(c); + } + + for(Future fr: executorService.invokeAll(tasks)) { + aggregateLatency += fr.get(); + } + + double averageRecordLatency = aggregateLatency/numTasks; + Assert.assertTrue(String.format("Average time for recording singleton latencies, %s ms should be in the range [%s, %s).", averageRecordLatency, minLatencyWhenDisabledMs, maxLatencyWhenDisabledMs), + averageRecordLatency < maxLatencyWhenDisabledMs && averageRecordLatency >= minLatencyWhenDisabledMs); + } + + @Test + public void verifyNoExceptionOnInvalidInputWhenDisabled() throws Exception { + Instant testInstant = Instant.now(); + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, false); + final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList()); + + try { + latencyTracker.recordClientLatency(null, null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), "test", null, false, null); + latencyTracker.recordClientLatency(Instant.now(), "test", "test", false, null); + latencyTracker.recordClientLatency(Instant.now(), "test", "test", false, httpOperation); + + latencyTracker.recordClientLatency(null, null, null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), null, null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", null, false, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", "test", false, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", "test", false, httpOperation); + + latencyTracker.recordClientLatency(testInstant, Instant.now(), null, null, false, null); + latencyTracker.recordClientLatency(Instant.MAX, Instant.now(), null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.MIN, null, null, false, null); + + latencyTracker.recordClientLatency(null, null, null, false, null, 0,null); + latencyTracker.recordClientLatency(Instant.now(), null, null, false, null, 0,null); + latencyTracker.recordClientLatency(Instant.now(), "test", null, false, null, 0,null); + latencyTracker.recordClientLatency(Instant.now(), "test", "test", false, null, 0,null); + latencyTracker.recordClientLatency(Instant.now(), "test", "test", false, Instant.now(), 0,null); + latencyTracker.recordClientLatency(Instant.now(), "test", "test", false, Instant.now(), 42, httpOperation); + + latencyTracker.recordClientLatency(null, null, null, null, false, null, 0,null); + latencyTracker.recordClientLatency(Instant.now(), null, null, null, false, null, 0,null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), null, null, false, null, 0,null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", null, false, null, 0,null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", "test", false, null, 0,null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", "test", false, Instant.now(), 0,null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", "test", false, Instant.now(), 42, httpOperation); + + latencyTracker.recordClientLatency(testInstant, Instant.now(), null, null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.MAX, Instant.now(), null, null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.MIN, null, null, false, null, 0, null); + + + } catch (Exception e) { + Assert.assertTrue("There should be no exception", false); + } + } + + @Test + public void verifyNoExceptionOnInvalidInputWhenEnabled() throws Exception { + Instant testInstant = Instant.now(); + LatencyTracker latencyTracker = new LatencyTracker(accountName, filesystemName, true); + final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList()); + + try { + latencyTracker.recordClientLatency(null, null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), "test", null, false, null); + latencyTracker.recordClientLatency(Instant.now(), "test", "test", false, null); + latencyTracker.recordClientLatency(Instant.now(), "test", "test", false, httpOperation); + + latencyTracker.recordClientLatency(null, null, null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), null, null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", null, false, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", "test", false, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", "test", false, httpOperation); + + latencyTracker.recordClientLatency(testInstant, Instant.now(), null, null, false, null); + latencyTracker.recordClientLatency(Instant.MAX, Instant.now(), null, null, false, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.MIN, null, null, false, null); + + latencyTracker.recordClientLatency(null, null, null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), null, null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), "test", null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), "test", "test", false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), "test", "test", false, Instant.now(), 0, null); + latencyTracker.recordClientLatency(Instant.now(), "test", "test", false, Instant.now(), 42, httpOperation); + + latencyTracker.recordClientLatency(null, null, null, null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), null, null, null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), null, null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", "test", false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", "test", false, Instant.now(), 0, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.now(), "test", "test", false, Instant.now(), 42, httpOperation); + + latencyTracker.recordClientLatency(testInstant, Instant.now(), null, null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.MAX, Instant.now(), null, null, false, null, 0, null); + latencyTracker.recordClientLatency(Instant.now(), Instant.MIN, null, null, false, null, 0, null); + } catch (Exception e){ + Assert.assertTrue("There should be no exception", false); + } + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml index 24d444a88d743..d833cfbe017fa 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml +++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml @@ -33,6 +33,11 @@ false + + fs.azure.abfs.latency.track + false + + diff --git a/hadoop-tools/hadoop-distcp/pom.xml b/hadoop-tools/hadoop-distcp/pom.xml index 019ae366606de..fe1681b48e6d6 100644 --- a/hadoop-tools/hadoop-distcp/pom.xml +++ b/hadoop-tools/hadoop-distcp/pom.xml @@ -103,6 +103,11 @@ mockito-core test + + org.assertj + assertj-core + test + diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListing.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListing.java index ed7650f965909..ce7d00d2bd732 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListing.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListing.java @@ -38,6 +38,7 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.AfterClass; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -205,8 +206,8 @@ public void testBuildListing() { Assert.fail("Duplicates not detected"); } catch (DuplicateFileException ignore) { } - Assert.assertEquals(listing.getBytesToCopy(), 10); - Assert.assertEquals(listing.getNumberOfPaths(), 3); + assertThat(listing.getBytesToCopy()).isEqualTo(10); + assertThat(listing.getNumberOfPaths()).isEqualTo(3); TestDistCpUtils.delete(fs, "/tmp"); try { diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java index 9361fc18f3c0a..b48355af25ba5 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java @@ -19,7 +19,9 @@ package org.apache.hadoop.tools; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; +import static org.assertj.core.api.Assertions.within; import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; import org.junit.Assert; import org.junit.Test; @@ -106,14 +108,14 @@ public void testParsebandwidth() { DistCpOptions options = OptionsParser.parse(new String[] { "hdfs://localhost:8020/source/first", "hdfs://localhost:8020/target/"}); - Assert.assertEquals(options.getMapBandwidth(), 0, DELTA); + assertThat(options.getMapBandwidth()).isCloseTo(0f, within(DELTA)); options = OptionsParser.parse(new String[] { "-bandwidth", "11.2", "hdfs://localhost:8020/source/first", "hdfs://localhost:8020/target/"}); - Assert.assertEquals(options.getMapBandwidth(), 11.2, DELTA); + assertThat(options.getMapBandwidth()).isCloseTo(11.2f, within(DELTA)); } @Test(expected=IllegalArgumentException.class) @@ -256,21 +258,21 @@ public void testParseMaps() { DistCpOptions options = OptionsParser.parse(new String[] { "hdfs://localhost:8020/source/first", "hdfs://localhost:8020/target/"}); - Assert.assertEquals(options.getMaxMaps(), DistCpConstants.DEFAULT_MAPS); + assertThat(options.getMaxMaps()).isEqualTo(DistCpConstants.DEFAULT_MAPS); options = OptionsParser.parse(new String[] { "-m", "1", "hdfs://localhost:8020/source/first", "hdfs://localhost:8020/target/"}); - Assert.assertEquals(options.getMaxMaps(), 1); + assertThat(options.getMaxMaps()).isEqualTo(1); options = OptionsParser.parse(new String[] { "-m", "0", "hdfs://localhost:8020/source/first", "hdfs://localhost:8020/target/"}); - Assert.assertEquals(options.getMaxMaps(), 1); + assertThat(options.getMaxMaps()).isEqualTo(1); try { OptionsParser.parse(new String[] { @@ -389,13 +391,13 @@ public void testCopyStrategy() { "-f", "hdfs://localhost:8020/source/first", "hdfs://localhost:8020/target/"}); - Assert.assertEquals(options.getCopyStrategy(), "dynamic"); + assertThat(options.getCopyStrategy()).isEqualTo("dynamic"); options = OptionsParser.parse(new String[] { "-f", "hdfs://localhost:8020/source/first", "hdfs://localhost:8020/target/"}); - Assert.assertEquals(options.getCopyStrategy(), DistCpConstants.UNIFORMSIZE); + assertThat(options.getCopyStrategy()).isEqualTo(DistCpConstants.UNIFORMSIZE); } @Test @@ -563,7 +565,7 @@ public void testOptionsAppendToConf() { conf = new Configuration(); Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false)); Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.DELETE_MISSING.getConfigLabel(), false)); - Assert.assertEquals(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel()), null); + assertThat(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel())).isNull(); options = OptionsParser.parse(new String[] { "-update", "-delete", @@ -575,8 +577,9 @@ public void testOptionsAppendToConf() { options.appendToConf(conf); Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false)); Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.DELETE_MISSING.getConfigLabel(), false)); - Assert.assertEquals(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel()), "U"); - Assert.assertEquals(conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), 11.2, DELTA); + assertThat(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel())).isEqualTo("U"); + assertThat(conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1)) + .isCloseTo(11.2f, within(DELTA)); } @Test @@ -588,9 +591,8 @@ public void testOptionsAppendToConfDoesntOverwriteBandwidth() { "hdfs://localhost:8020/source/first", "hdfs://localhost:8020/target/"}); options.appendToConf(conf); - Assert.assertEquals( - conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), -1.0, - DELTA); + assertThat(conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1)) + .isCloseTo(-1.0f,within(DELTA)); conf = new Configuration(); Assert.assertEquals( @@ -800,6 +802,6 @@ public void testExclusionsOption() { "/tmp/filters.txt", "hdfs://localhost:8020/source/first", "hdfs://localhost:8020/target/"}); - Assert.assertEquals(options.getFiltersFile(), "/tmp/filters.txt"); + assertThat(options.getFiltersFile()).isEqualTo("/tmp/filters.txt"); } } diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java index 51eebbb2a3069..bf3165765d9cd 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java @@ -67,6 +67,7 @@ import static org.apache.hadoop.test.MetricsAsserts.assertCounter; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; +import static org.assertj.core.api.Assertions.assertThat; public class TestCopyMapper { private static final Logger LOG = LoggerFactory.getLogger(TestCopyMapper.class); @@ -769,7 +770,7 @@ public Integer run() { new CopyListingFileStatus(tmpFS.getFileStatus( new Path(SOURCE_PATH + "/src/file"))), context); - Assert.assertEquals(stubContext.getWriter().values().size(), 1); + assertThat(stubContext.getWriter().values().size()).isEqualTo(1); Assert.assertTrue(stubContext.getWriter().values().get(0).toString().startsWith("SKIP")); Assert.assertTrue(stubContext.getWriter().values().get(0).toString(). contains(SOURCE_PATH + "/src/file")); diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestDistCpUtils.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestDistCpUtils.java index 6ce8e3e1eabb2..4acb022786c63 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestDistCpUtils.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestDistCpUtils.java @@ -67,6 +67,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; public class TestDistCpUtils { private static final Logger LOG = LoggerFactory.getLogger(TestDistCpUtils.class); @@ -98,39 +99,39 @@ public static void destroy() { public void testGetRelativePathRoot() { Path root = new Path("/"); Path child = new Path("/a"); - Assert.assertEquals(DistCpUtils.getRelativePath(root, child), "/a"); + assertThat(DistCpUtils.getRelativePath(root, child)).isEqualTo("/a"); } @Test public void testGetRelativePath() { Path root = new Path("/tmp/abc"); Path child = new Path("/tmp/abc/xyz/file"); - Assert.assertEquals(DistCpUtils.getRelativePath(root, child), "/xyz/file"); + assertThat(DistCpUtils.getRelativePath(root, child)).isEqualTo("/xyz/file"); } @Test public void testPackAttributes() { EnumSet attributes = EnumSet.noneOf(FileAttribute.class); - Assert.assertEquals(DistCpUtils.packAttributes(attributes), ""); + assertThat(DistCpUtils.packAttributes(attributes)).isEqualTo(""); attributes.add(FileAttribute.REPLICATION); - Assert.assertEquals(DistCpUtils.packAttributes(attributes), "R"); + assertThat(DistCpUtils.packAttributes(attributes)).isEqualTo("R"); attributes.add(FileAttribute.BLOCKSIZE); - Assert.assertEquals(DistCpUtils.packAttributes(attributes), "RB"); + assertThat(DistCpUtils.packAttributes(attributes)).isEqualTo("RB"); attributes.add(FileAttribute.USER); attributes.add(FileAttribute.CHECKSUMTYPE); - Assert.assertEquals(DistCpUtils.packAttributes(attributes), "RBUC"); + assertThat(DistCpUtils.packAttributes(attributes)).isEqualTo("RBUC"); attributes.add(FileAttribute.GROUP); - Assert.assertEquals(DistCpUtils.packAttributes(attributes), "RBUGC"); + assertThat(DistCpUtils.packAttributes(attributes)).isEqualTo("RBUGC"); attributes.add(FileAttribute.PERMISSION); - Assert.assertEquals(DistCpUtils.packAttributes(attributes), "RBUGPC"); + assertThat(DistCpUtils.packAttributes(attributes)).isEqualTo("RBUGPC"); attributes.add(FileAttribute.TIMES); - Assert.assertEquals(DistCpUtils.packAttributes(attributes), "RBUGPCT"); + assertThat(DistCpUtils.packAttributes(attributes)).isEqualTo("RBUGPCT"); } @Test diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/BlockPlacementPolicyAlwaysSatisfied.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/BlockPlacementPolicyAlwaysSatisfied.java index 34c19512a738f..de12aec4137c8 100644 --- a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/BlockPlacementPolicyAlwaysSatisfied.java +++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/BlockPlacementPolicyAlwaysSatisfied.java @@ -42,6 +42,11 @@ public boolean isPlacementPolicySatisfied() { public String getErrorDescription() { return null; } + + @Override + public int getAdditionalReplicasRequired() { + return 0; + } } @Override diff --git a/hadoop-tools/hadoop-kafka/pom.xml b/hadoop-tools/hadoop-kafka/pom.xml index b675bd3751e9a..a227ad4efd029 100644 --- a/hadoop-tools/hadoop-kafka/pom.xml +++ b/hadoop-tools/hadoop-kafka/pom.xml @@ -111,5 +111,10 @@ mockito-core test + + org.assertj + assertj-core + test + diff --git a/hadoop-tools/hadoop-kafka/src/test/java/org/apache/hadoop/metrics2/impl/TestKafkaMetrics.java b/hadoop-tools/hadoop-kafka/src/test/java/org/apache/hadoop/metrics2/impl/TestKafkaMetrics.java index bfaef7b74f475..8d74bf247567f 100644 --- a/hadoop-tools/hadoop-kafka/src/test/java/org/apache/hadoop/metrics2/impl/TestKafkaMetrics.java +++ b/hadoop-tools/hadoop-kafka/src/test/java/org/apache/hadoop/metrics2/impl/TestKafkaMetrics.java @@ -42,11 +42,11 @@ import java.util.StringJoiner; import java.util.concurrent.Future; -import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.assertj.core.api.Assertions.assertThat; /** * This tests that the KafkaSink properly formats the Kafka message. @@ -147,7 +147,7 @@ public void visit(MetricsVisitor visitor) { if (LOG.isDebugEnabled()) { LOG.debug("kafka result: " + jsonResult); } - assertEquals(jsonLines.toString(), jsonResult); + assertThat(jsonLines.toString()).isEqualTo(jsonResult); } StringBuilder recordToJson(MetricsRecord record) { diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java index 6ed28d90d8cd4..f99038ebe5ff6 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.InputStreamReader; import java.io.Reader; +import java.security.Security; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -150,6 +151,10 @@ public enum TraceType { SLS, RUMEN, SYNTH } + public static final String NETWORK_CACHE_TTL = "networkaddress.cache.ttl"; + public static final String NETWORK_NEGATIVE_CACHE_TTL = + "networkaddress.cache.negative.ttl"; + private TraceType inputType; private SynthTraceJobProducer stjp; @@ -241,6 +246,9 @@ public void setSimulationParams(TraceType inType, String[] inTraces, public void start() throws IOException, ClassNotFoundException, YarnException, InterruptedException { + + enableDNSCaching(getConf()); + // start resource manager startRM(); // start node managers @@ -260,6 +268,23 @@ public void start() throws IOException, ClassNotFoundException, YarnException, runner.start(); } + /** + * Enables DNS Caching based on config. If DNS caching is enabled, then set + * the DNS cache to infinite time. Since in SLS random nodes are added, DNS + * resolution can take significant time which can cause erroneous results. + * For more details, check + * Java Networking Properties + * @param conf Configuration object. + */ + static void enableDNSCaching(Configuration conf) { + if (conf.getBoolean(SLSConfiguration.DNS_CACHING_ENABLED, + SLSConfiguration.DNS_CACHING_ENABLED_DEFAULT)) { + Security.setProperty(NETWORK_CACHE_TTL, "-1"); + Security.setProperty(NETWORK_NEGATIVE_CACHE_TTL, "-1"); + } + } + private void startRM() throws ClassNotFoundException, YarnException { Configuration rmConf = new YarnConfiguration(getConf()); String schedulerClass = rmConf.get(YarnConfiguration.RM_SCHEDULER); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java index 34b89b656a389..119960c92a494 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java @@ -28,6 +28,9 @@ public class SLSConfiguration { // sls public static final String PREFIX = "yarn.sls."; + public static final String DNS_CACHING_ENABLED = PREFIX + + "dns.caching.enabled"; + public static final boolean DNS_CACHING_ENABLED_DEFAULT = false; // runner public static final String RUNNER_PREFIX = PREFIX + "runner."; public static final String RUNNER_POOL_SIZE = RUNNER_PREFIX + "pool.size"; diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java index 668be145d70c4..bfbd592c42b58 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java +++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java @@ -64,7 +64,9 @@ public abstract class BaseSLSRunnerTest { @After public void tearDown() throws InterruptedException { - sls.stop(); + if (sls != null) { + sls.stop(); + } } public void runSLS(Configuration conf, long timeout) throws Exception { diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java index abb3b5e904abd..2463ccf06a93d 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java +++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java @@ -22,14 +22,18 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; +import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.*; +import java.security.Security; import java.util.*; +import static org.junit.Assert.assertEquals; + /** * This test performs simple runs of the SLS with different trace types and * schedulers. @@ -86,4 +90,39 @@ public void testSimulatorRunning() throws Exception { runSLS(conf, timeTillShutdownInsec); } + /** + * Test to check whether caching is enabled based on config. + */ + @Test + public void testEnableCaching() { + String networkCacheDefault = Security.getProperty( + SLSRunner.NETWORK_CACHE_TTL); + String networkNegativeCacheDefault = + Security.getProperty(SLSRunner.NETWORK_NEGATIVE_CACHE_TTL); + + try { + Configuration conf = new Configuration(false); + // check when dns caching is disabled + conf.setBoolean(SLSConfiguration.DNS_CACHING_ENABLED, false); + SLSRunner.enableDNSCaching(conf); + assertEquals(networkCacheDefault, + Security.getProperty(SLSRunner.NETWORK_CACHE_TTL)); + assertEquals(networkNegativeCacheDefault, + Security.getProperty(SLSRunner.NETWORK_NEGATIVE_CACHE_TTL)); + + // check when dns caching is enabled + conf.setBoolean(SLSConfiguration.DNS_CACHING_ENABLED, true); + SLSRunner.enableDNSCaching(conf); + assertEquals("-1", + Security.getProperty(SLSRunner.NETWORK_CACHE_TTL)); + assertEquals("-1", + Security.getProperty(SLSRunner.NETWORK_NEGATIVE_CACHE_TTL)); + } finally { + // set security settings back to default + Security.setProperty(SLSRunner.NETWORK_CACHE_TTL, + String.valueOf(networkCacheDefault)); + Security.setProperty(SLSRunner.NETWORK_NEGATIVE_CACHE_TTL, + String.valueOf(networkNegativeCacheDefault)); + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml index 24c2167a434ee..814736645dbae 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml @@ -68,6 +68,7 @@ org.apache.maven.plugins maven-surefire-plugin + 1800 ${java.home} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java index 4475caf5df8dc..d51afa26989b0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java @@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol; import org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl; @@ -77,7 +76,6 @@ import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator; import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext; -import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils; import java.io.IOException; import java.net.InetSocketAddress; @@ -103,7 +101,7 @@ public class OpportunisticContainerAllocatorAMService private final NodeQueueLoadMonitor nodeMonitor; private final OpportunisticContainerAllocator oppContainerAllocator; - private final int k; + private final int numNodes; private final long cacheRefreshInterval; private volatile List cachedNodes; @@ -176,7 +174,7 @@ public void allocate(ApplicationAttemptId appAttemptId, if (!appAttempt.getApplicationAttemptId().equals(appAttemptId)){ LOG.error("Calling allocate on previous or removed or non " - + "existent application attempt " + appAttemptId); + + "existent application attempt {}", appAttemptId); return; } @@ -238,7 +236,7 @@ public OpportunisticContainerAllocatorAMService(RMContext rmContext, new DistributedOpportunisticContainerAllocator( rmContext.getContainerTokenSecretManager(), maxAllocationsPerAMHeartbeat); - this.k = rmContext.getYarnConfiguration().getInt( + this.numNodes = rmContext.getYarnConfiguration().getInt( YarnConfiguration.OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED, YarnConfiguration.DEFAULT_OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED); long nodeSortInterval = rmContext.getYarnConfiguration().getLong( @@ -440,12 +438,12 @@ public void handle(SchedulerEvent event) { // <-- IGNORED EVENTS : END --> default: LOG.error("Unknown event arrived at" + - "OpportunisticContainerAllocatorAMService: " + event.toString()); + "OpportunisticContainerAllocatorAMService: {}", event); } } - public QueueLimitCalculator getNodeManagerQueueLimitCalculator() { + QueueLimitCalculator getNodeManagerQueueLimitCalculator() { return nodeMonitor.getThresholdCalculator(); } @@ -455,7 +453,7 @@ synchronized List getLeastLoadedNodes() { if ((currTime - lastCacheUpdateTime > cacheRefreshInterval) || (cachedNodes == null)) { cachedNodes = convertToRemoteNodes( - this.nodeMonitor.selectLeastLoadedNodes(this.k)); + this.nodeMonitor.selectLeastLoadedNodes(this.numNodes)); if (cachedNodes.size() > 0) { lastCacheUpdateTime = currTime; } @@ -485,12 +483,4 @@ private RemoteNode convertToRemoteNode(NodeId nodeId) { } return null; } - - private static ApplicationAttemptId getAppAttemptId() throws YarnException { - AMRMTokenIdentifier amrmTokenIdentifier = - YarnServerSecurityUtils.authorizeRequest(); - ApplicationAttemptId applicationAttemptId = - amrmTokenIdentifier.getApplicationAttemptId(); - return applicationAttemptId; - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md index a6d7971b63f1a..8f0b46430ca05 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md @@ -5292,6 +5292,87 @@ Response Header: Content-Type: application/xml Transfer-Encoding: chunked +**Adding Node Labels to a queue** + +Assuming we are using the capacity scheduler and the current queue configuration is two queues root.default, and root.a, this example adds a Node Label x to the queue root.a. Create a Node Label x and assign the nodes with below commands. + +```yarn rmadmin -addToClusterNodeLabels "x(exclusive=true)"``` + +```yarn rmadmin -replaceLabelsOnNode "=x"``` + +HTTP Request: + +```xml + Accept: application/xml + PUT http://rm-http-address:port/ws/v1/cluster/scheduler-conf + Content-Type: application/xml + + + + root.a + + + accessible-node-labels + x + + + accessible-node-labels.x.capacity + 100 + + + + + root + + + accessible-node-labels.x.capacity + 100 + + + + +``` + + +Response Header: + + HTTP/1.1 200 OK + Content-Type: application/xml + Transfer-Encoding: chunked + +**Removing Node Labels from a queue** + +Assuming we are using the capacity scheduler and the current queue configuration is two queues root.default, and root.a and Node Label x is assigned to queue root.a. This example unsets the Node Label x from the queue root.a and removes it. + +HTTP Request: + +```xml + Accept: application/xml + PUT http://rm-http-address:port/ws/v1/cluster/scheduler-conf + Content-Type: application/xml + + + + root.a + + + accessible-node-labels + + + + + +``` + + +Response Header: + + HTTP/1.1 200 OK + Content-Type: application/xml + Transfer-Encoding: chunked + +```yarn rmadmin -removeFromClusterNodeLabels x``` + Cluster Container Signal API --------------------------------