Merge master to EC branch HDDS-3816-ec
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/ECXceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/ECXceiverClientGrpc.java
new file mode 100644
index 0000000..4873fdd
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/ECXceiverClientGrpc.java
@@ -0,0 +1,60 @@
+/*
+ * 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;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+
+import java.security.cert.X509Certificate;
+import java.util.List;
+
+/**
+ * {@link XceiverClientSpi} implementation to work specifically with EC
+ * related requests. The only difference at the moment from the basic
+ * {@link XceiverClientGrpc} is that this implementation does async calls when
+ * a write request is posted via the sendCommandAsync method.
+ *
+ * @see https://issues.apache.org/jira/browse/HDDS-5954
+ */
+public class ECXceiverClientGrpc extends XceiverClientGrpc {
+
+ public ECXceiverClientGrpc(
+ Pipeline pipeline,
+ ConfigurationSource config,
+ List<X509Certificate> caCerts) {
+ super(pipeline, config, caCerts);
+ }
+
+ /**
+ * For EC writes, due to outside syncronization points during writes, it is
+ * not necessary to block any async requests that are
+ * arriving via the
+ * {@link #sendCommandAsync(ContainerProtos.ContainerCommandRequestProto)}
+ * method.
+ *
+ * @param request the request we need the decision about
+ * @return false always to do not block async requests.
+ */
+ @Override
+ protected boolean shouldBlockAndWaitAsyncReply(
+ ContainerProtos.ContainerCommandRequestProto request) {
+ return false;
+ }
+}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java
index 064ce6e..63dd511 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java
@@ -136,6 +136,32 @@
tags = ConfigTag.CLIENT)
private boolean checksumVerify = true;
+ @Config(key = "max.ec.stripe.write.retries",
+ defaultValue = "10",
+ description = "Ozone EC client to retry stripe to new block group on" +
+ " failures.",
+ tags = ConfigTag.CLIENT)
+ private int maxECStripeWriteRetries = 10;
+
+ @Config(key = "exclude.nodes.expiry.time",
+ defaultValue = "600000",
+ description = "Time after which an excluded node is reconsidered for" +
+ " writes in EC. If the value is zero, the node is excluded for the" +
+ " life of the client",
+ tags = ConfigTag.CLIENT)
+ private long excludeNodesExpiryTime = 10 * 60 * 1000;
+
+ @Config(key = "ec.reconstruct.stripe.read.pool.limit",
+ defaultValue = "30",
+ description = "Thread pool max size for parallelly read" +
+ " available ec chunks to reconstruct the whole stripe.",
+ tags = ConfigTag.CLIENT)
+ // For the largest recommended EC policy rs-10-4-1024k,
+ // 10 chunks are required at least for stripe reconstruction,
+ // so 1 core thread for each chunk and
+ // 3 concurrent stripe read should be enough.
+ private int ecReconstructStripeReadPoolLimit = 10 * 3;
+
@Config(key = "checksum.combine.mode",
defaultValue = "COMPOSITE_CRC",
description = "The combined checksum type [MD5MD5CRC / COMPOSITE_CRC] "
@@ -250,6 +276,14 @@
this.checksumVerify = checksumVerify;
}
+ public int getMaxECStripeWriteRetries() {
+ return this.maxECStripeWriteRetries;
+ }
+
+ public long getExcludeNodesExpiryTime() {
+ return excludeNodesExpiryTime;
+ }
+
public int getBufferIncrement() {
return bufferIncrement;
}
@@ -265,4 +299,12 @@
ChecksumCombineMode.COMPOSITE_CRC.name());
}
}
+
+ public void setEcReconstructStripeReadPoolLimit(int poolLimit) {
+ this.ecReconstructStripeReadPoolLimit = poolLimit;
+ }
+
+ public int getEcReconstructStripeReadPoolLimit() {
+ return ecReconstructStripeReadPoolLimit;
+ }
}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
index a268495..eea2bf3 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
@@ -460,7 +460,7 @@
// served out of order over XceiverClientGrpc. This needs to be fixed
// if this API is to be used for I/O path. Currently, this is not
// used for Read/Write Operation but for tests.
- if (!HddsUtils.isReadOnly(request)) {
+ if (shouldBlockAndWaitAsyncReply(request)) {
asyncReply.getResponse().get();
}
return asyncReply;
@@ -470,6 +470,21 @@
}
}
+ /**
+ * During data writes the ordering of WriteChunk and PutBlock is not ensured
+ * by any outside logic, therefore in this original implementation, all reads
+ * and writes are synchronized.
+ * This method is providing the possibility for subclasses to override this
+ * behaviour.
+ *
+ * @param request the request we need the decision about
+ * @return true if the request is a write request.
+ */
+ protected boolean shouldBlockAndWaitAsyncReply(
+ ContainerCommandRequestProto request) {
+ return !HddsUtils.isReadOnly(request);
+ }
+
@VisibleForTesting
public XceiverClientReply sendCommandAsync(
ContainerCommandRequestProto request, DatanodeDetails dn)
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
index f016972..ee4edec 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
@@ -29,6 +29,7 @@
import org.apache.hadoop.hdds.conf.ConfigGroup;
import org.apache.hadoop.hdds.conf.ConfigType;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -230,6 +231,9 @@
case STAND_ALONE:
client = new XceiverClientGrpc(pipeline, conf, caCerts);
break;
+ case EC:
+ client = new ECXceiverClientGrpc(pipeline, conf, caCerts);
+ break;
case CHAINED:
default:
throw new IOException("not implemented" + pipeline.getType());
@@ -246,9 +250,16 @@
private String getPipelineCacheKey(Pipeline pipeline, boolean forRead) {
String key = pipeline.getId().getId().toString() + pipeline.getType();
- if (topologyAwareRead && forRead) {
+ boolean isEC = pipeline.getReplicationConfig()
+ .getReplicationType() == HddsProtos.ReplicationType.EC;
+ if (topologyAwareRead && forRead || isEC) {
try {
key += pipeline.getClosestNode().getHostName();
+ if (isEC) {
+ // Currently EC uses standalone client.
+ key += pipeline.getClosestNode()
+ .getPort(DatanodeDetails.Port.Name.STANDALONE);
+ }
} catch (IOException e) {
LOG.error("Failed to get closest node to create pipeline cache key:" +
e.getMessage());
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java
new file mode 100644
index 0000000..5be2b07
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockExtendedInputStream.java
@@ -0,0 +1,34 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.storage;
+
+import org.apache.hadoop.hdds.client.BlockID;
+
+/**
+ * Abstract class used as an interface for input streams related to Ozone
+ * blocks.
+ */
+public abstract class BlockExtendedInputStream extends ExtendedInputStream {
+
+ public abstract BlockID getBlockID();
+
+ public abstract long getRemaining();
+
+ public abstract long getLength();
+
+}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
index a6f22d4..07a444a 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
@@ -21,7 +21,6 @@
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
-import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@@ -29,9 +28,6 @@
import java.util.function.Function;
import com.google.common.base.Preconditions;
-import org.apache.hadoop.fs.ByteBufferReadable;
-import org.apache.hadoop.fs.CanUnbuffer;
-import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
@@ -59,14 +55,11 @@
* This class encapsulates all state management for iterating
* through the sequence of chunks through {@link ChunkInputStream}.
*/
-public class BlockInputStream extends InputStream
- implements Seekable, CanUnbuffer, ByteBufferReadable {
+public class BlockInputStream extends BlockExtendedInputStream {
private static final Logger LOG =
LoggerFactory.getLogger(BlockInputStream.class);
- private static final int EOF = -1;
-
private final BlockID blockID;
private final long length;
private Pipeline pipeline;
@@ -219,7 +212,8 @@
protected List<ChunkInfo> getChunkInfos() throws IOException {
// irrespective of the container state, we will always read via Standalone
// protocol.
- if (pipeline.getType() != HddsProtos.ReplicationType.STAND_ALONE) {
+ if (pipeline.getType() != HddsProtos.ReplicationType.STAND_ALONE && pipeline
+ .getType() != HddsProtos.ReplicationType.EC) {
pipeline = Pipeline.newBuilder(pipeline)
.setReplicationConfig(StandaloneReplicationConfig.getInstance(
ReplicationConfig
@@ -269,46 +263,14 @@
xceiverClientFactory, () -> pipeline, verifyChecksum, token);
}
+ @Override
public synchronized long getRemaining() {
return length - getPos();
}
- /**
- * {@inheritDoc}
- */
@Override
- public synchronized int read() throws IOException {
- byte[] buf = new byte[1];
- if (read(buf, 0, 1) == EOF) {
- return EOF;
- }
- return Byte.toUnsignedInt(buf[0]);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public synchronized int read(byte[] b, int off, int len) throws IOException {
- ByteReaderStrategy strategy = new ByteArrayReader(b, off, len);
- if (len == 0) {
- return 0;
- }
- return readWithStrategy(strategy);
- }
-
- @Override
- public synchronized int read(ByteBuffer byteBuffer) throws IOException {
- ByteReaderStrategy strategy = new ByteBufferReader(byteBuffer);
- int len = strategy.getTargetLength();
- if (len == 0) {
- return 0;
- }
- return readWithStrategy(strategy);
- }
-
- synchronized int readWithStrategy(ByteReaderStrategy strategy) throws
- IOException {
+ protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+ throws IOException {
Preconditions.checkArgument(strategy != null);
if (!initialized) {
initialize();
@@ -478,10 +440,6 @@
}
}
- public synchronized void resetPosition() {
- this.blockPosition = 0;
- }
-
/**
* Checks if the stream is open. If not, throw an exception.
*
@@ -493,10 +451,12 @@
}
}
+ @Override
public BlockID getBlockID() {
return blockID;
}
+ @Override
public long getLength() {
return length;
}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
index 8b3f817..b86464c 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java
@@ -119,6 +119,7 @@
//current buffer allocated to write
private ChunkBuffer currentBuffer;
private final Token<? extends TokenIdentifier> token;
+ private int replicationIndex;
/**
* Creates a new BlockOutputStream.
@@ -148,6 +149,8 @@
this.bufferPool = bufferPool;
this.token = token;
+ replicationIndex = pipeline.getReplicaIndex(pipeline.getClosestNode());
+
//number of buffers used before doing a flush
refreshCurrentBuffer();
flushPeriod = (int) (config.getStreamBufferFlushSize() / config
@@ -210,6 +213,22 @@
return ioException.get();
}
+ XceiverClientSpi getXceiverClientSpi() {
+ return this.xceiverClient;
+ }
+
+ BlockData.Builder getContainerBlockData() {
+ return this.containerBlockData;
+ }
+
+ Token<? extends TokenIdentifier> getToken() {
+ return this.token;
+ }
+
+ ExecutorService getResponseExecutor() {
+ return this.responseExecutor;
+ }
+
@Override
public void write(int b) throws IOException {
checkOpen();
@@ -396,7 +415,7 @@
* @param force true if no data was written since most recent putBlock and
* stream is being closed
*/
- private CompletableFuture<ContainerProtos.
+ CompletableFuture<ContainerProtos.
ContainerCommandResponseProto> executePutBlock(boolean close,
boolean force) throws IOException {
checkOpen();
@@ -568,7 +587,7 @@
void waitOnFlushFutures() throws InterruptedException, ExecutionException {
}
- private void validateResponse(
+ void validateResponse(
ContainerProtos.ContainerCommandResponseProto responseProto)
throws IOException {
try {
@@ -587,7 +606,7 @@
}
- private void setIoException(Exception e) {
+ public void setIoException(Exception e) {
IOException ioe = getIoException();
if (ioe == null) {
IOException exception = new IOException(EXCEPTION_MSG + e.toString(), e);
@@ -623,7 +642,7 @@
*
* @throws IOException if stream is closed
*/
- private void checkOpen() throws IOException {
+ void checkOpen() throws IOException {
if (isClosed()) {
throw new IOException("BlockOutputStream has been closed.");
} else if (getIoException() != null) {
@@ -643,8 +662,10 @@
* @throws IOException if there is an I/O error while performing the call
* @throws OzoneChecksumException if there is an error while computing
* checksum
+ * @return
*/
- private void writeChunkToContainer(ChunkBuffer chunk) throws IOException {
+ CompletableFuture<ContainerCommandResponseProto> writeChunkToContainer(
+ ChunkBuffer chunk) throws IOException {
int effectiveChunkSize = chunk.remaining();
final long offset = chunkOffset.getAndAdd(effectiveChunkSize);
final ByteString data = chunk.toByteString(
@@ -664,31 +685,34 @@
try {
XceiverClientReply asyncReply = writeChunkAsync(xceiverClient, chunkInfo,
- blockID.get(), data, token);
- CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
- asyncReply.getResponse();
- future.thenApplyAsync(e -> {
- try {
- validateResponse(e);
- } catch (IOException sce) {
- future.completeExceptionally(sce);
- }
- return e;
- }, responseExecutor).exceptionally(e -> {
- String msg = "Failed to write chunk " + chunkInfo.getChunkName() + " " +
- "into block " + blockID;
- LOG.debug("{}, exception: {}", msg, e.getLocalizedMessage());
- CompletionException ce = new CompletionException(msg, e);
- setIoException(ce);
- throw ce;
- });
+ blockID.get(), data, token, replicationIndex);
+ CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+ respFuture = asyncReply.getResponse();
+ CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+ validateFuture = respFuture.thenApplyAsync(e -> {
+ try {
+ validateResponse(e);
+ } catch (IOException sce) {
+ respFuture.completeExceptionally(sce);
+ }
+ return e;
+ }, responseExecutor).exceptionally(e -> {
+ String msg = "Failed to write chunk " + chunkInfo.getChunkName() +
+ " into block " + blockID;
+ LOG.debug("{}, exception: {}", msg, e.getLocalizedMessage());
+ CompletionException ce = new CompletionException(msg, e);
+ setIoException(ce);
+ throw ce;
+ });
+ containerBlockData.addChunks(chunkInfo);
+ return validateFuture;
} catch (IOException | ExecutionException e) {
throw new IOException(EXCEPTION_MSG + e.toString(), e);
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();
handleInterruptedException(ex, false);
}
- containerBlockData.addChunks(chunkInfo);
+ return null;
}
@VisibleForTesting
@@ -704,7 +728,7 @@
* handle ExecutionException else skip it.
* @throws IOException
*/
- private void handleInterruptedException(Exception ex,
+ void handleInterruptedException(Exception ex,
boolean processExecutionException)
throws IOException {
LOG.error("Command execution was interrupted.");
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java
new file mode 100644
index 0000000..b2f86ac
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java
@@ -0,0 +1,177 @@
+/*
+ * 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.storage;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * Handles the chunk EC writes for an EC internal block.
+ */
+public class ECBlockOutputStream extends BlockOutputStream {
+
+ private final DatanodeDetails datanodeDetails;
+ private CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+ currentChunkRspFuture = null;
+
+ private CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+ putBlkRspFuture = null;
+ /**
+ * Creates a new ECBlockOutputStream.
+ *
+ * @param blockID block ID
+ * @param xceiverClientManager client manager that controls client
+ * @param pipeline pipeline where block will be written
+ * @param bufferPool pool of buffers
+ */
+ public ECBlockOutputStream(
+ BlockID blockID,
+ XceiverClientFactory xceiverClientManager,
+ Pipeline pipeline,
+ BufferPool bufferPool,
+ OzoneClientConfig config,
+ Token<? extends TokenIdentifier> token
+ ) throws IOException {
+ super(blockID, xceiverClientManager,
+ pipeline, bufferPool, config, token);
+ // In EC stream, there will be only one node in pipeline.
+ this.datanodeDetails = pipeline.getClosestNode();
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ this.currentChunkRspFuture =
+ writeChunkToContainer(ChunkBuffer.wrap(ByteBuffer.wrap(b, off, len)));
+ }
+
+ /**
+ * @param close whether putBlock is happening as part of closing the stream
+ * @param force true if no data was written since most recent putBlock and
+ * stream is being closed
+ */
+ public CompletableFuture<ContainerProtos.
+ ContainerCommandResponseProto> executePutBlock(boolean close,
+ boolean force) throws IOException {
+ checkOpen();
+
+ CompletableFuture<ContainerProtos.
+ ContainerCommandResponseProto> flushFuture = null;
+ try {
+ ContainerProtos.BlockData blockData = getContainerBlockData().build();
+ XceiverClientReply asyncReply =
+ putBlockAsync(getXceiverClient(), blockData, close, getToken());
+ CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
+ asyncReply.getResponse();
+ flushFuture = future.thenApplyAsync(e -> {
+ try {
+ validateResponse(e);
+ } catch (IOException sce) {
+ throw new CompletionException(sce);
+ }
+ // if the ioException is not set, putBlock is successful
+ if (getIoException() == null) {
+ BlockID responseBlockID = BlockID.getFromProtobuf(
+ e.getPutBlock().getCommittedBlockLength().getBlockID());
+ Preconditions.checkState(getBlockID().getContainerBlockID()
+ .equals(responseBlockID.getContainerBlockID()));
+ }
+ return e;
+ }, getResponseExecutor()).exceptionally(e -> {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("putBlock failed for blockID {} with exception {}",
+ getBlockID(), e.getLocalizedMessage());
+ }
+ CompletionException ce = new CompletionException(e);
+ setIoException(ce);
+ throw ce;
+ });
+ } catch (IOException | ExecutionException e) {
+ throw new IOException(EXCEPTION_MSG + e.toString(), e);
+ } catch (InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ handleInterruptedException(ex, false);
+ }
+ this.putBlkRspFuture = flushFuture;
+ return flushFuture;
+ }
+
+ @Override
+ public void close() throws IOException {
+ super.close();
+ cleanup(false);
+ }
+
+ /**
+ * @return The current chunk writer response future.
+ */
+ public CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+ getCurrentChunkResponseFuture() {
+ return this.currentChunkRspFuture;
+ }
+
+ /**
+ * @return The current chunk putBlock response future.
+ */
+ public CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+ getCurrentPutBlkResponseFuture() {
+ return this.putBlkRspFuture;
+ }
+
+ /**
+ * Gets the target data node used in the current stream.
+ * @return DatanodeDetails
+ */
+ public DatanodeDetails getDatanodeDetails() {
+ return datanodeDetails;
+ }
+
+ @Override
+ void validateResponse(
+ ContainerProtos.ContainerCommandResponseProto responseProto)
+ throws IOException {
+ try {
+ // if the ioException is already set, it means a prev request has failed
+ // just throw the exception. The current operation will fail with the
+ // original error
+ IOException exception = getIoException();
+ if (exception != null) {
+ return;
+ }
+ ContainerProtocolCalls.validateContainerResponse(responseProto);
+ } catch (IOException sce) {
+ setIoException(sce);
+ }
+ }
+}
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ExtendedInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ExtendedInputStream.java
new file mode 100644
index 0000000..de868c7
--- /dev/null
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ExtendedInputStream.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.storage;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.fs.ByteBufferReadable;
+import org.apache.hadoop.fs.CanUnbuffer;
+import org.apache.hadoop.fs.Seekable;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+/**
+ * Abstact class which extends InputStream and some common interfaces used by
+ * various Ozone InputStream classes.
+ */
+public abstract class ExtendedInputStream extends InputStream
+ implements Seekable, CanUnbuffer, ByteBufferReadable {
+
+ protected static final int EOF = -1;
+
+ @Override
+ public synchronized int read() throws IOException {
+ byte[] buf = new byte[1];
+ if (read(buf, 0, 1) == EOF) {
+ return EOF;
+ }
+ return Byte.toUnsignedInt(buf[0]);
+ }
+
+ @Override
+ public synchronized int read(byte[] b, int off, int len) throws IOException {
+ ByteReaderStrategy strategy = new ByteArrayReader(b, off, len);
+ int bufferLen = strategy.getTargetLength();
+ if (bufferLen == 0) {
+ return 0;
+ }
+ return readWithStrategy(strategy);
+ }
+
+ @Override
+ public synchronized int read(ByteBuffer byteBuffer) throws IOException {
+ ByteReaderStrategy strategy = new ByteBufferReader(byteBuffer);
+ int bufferLen = strategy.getTargetLength();
+ if (bufferLen == 0) {
+ return 0;
+ }
+ return readWithStrategy(strategy);
+ }
+
+ /**
+ * This must be overridden by the extending classes to call read on the
+ * underlying stream they are reading from. The last stream in the chain (the
+ * one which provides the actual data) needs to provide a real read via the
+ * read methods. For example if a test is extending this class, then it will
+ * need to override both read methods above and provide a dummy
+ * readWithStrategy implementation, as it will never be called by the tests.
+ *
+ * @param strategy
+ * @return
+ * @throws IOException
+ */
+ protected abstract int readWithStrategy(ByteReaderStrategy strategy) throws
+ IOException;
+
+ @Override
+ public synchronized void seek(long l) throws IOException {
+ throw new NotImplementedException("Seek is not implemented");
+ }
+
+ @Override
+ public synchronized boolean seekToNewSource(long l) throws IOException {
+ return false;
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DefaultReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DefaultReplicationConfig.java
new file mode 100644
index 0000000..851cae9
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DefaultReplicationConfig.java
@@ -0,0 +1,134 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
+import java.util.Objects;
+
+/**
+ * Replication configuration for EC replication.
+ */
+public class DefaultReplicationConfig {
+
+ private ReplicationType type;
+ private ReplicationFactor factor;
+ private ECReplicationConfig ecReplicationConfig;
+
+ public DefaultReplicationConfig(ReplicationType type,
+ ReplicationFactor factor) {
+ this.type = type;
+ this.factor = factor;
+ this.ecReplicationConfig = null;
+ }
+
+ public DefaultReplicationConfig(ReplicationConfig replicationConfig) {
+ this.type =
+ ReplicationType.fromProto(replicationConfig.getReplicationType());
+ if (replicationConfig instanceof ECReplicationConfig) {
+ this.ecReplicationConfig = (ECReplicationConfig) replicationConfig;
+ } else {
+ this.factor =
+ ReplicationFactor.valueOf(replicationConfig.getRequiredNodes());
+ }
+ }
+
+ public DefaultReplicationConfig(ReplicationType type,
+ ECReplicationConfig ecReplicationConfig) {
+ this.type = type;
+ this.factor = null;
+ this.ecReplicationConfig = ecReplicationConfig;
+ }
+
+ public DefaultReplicationConfig(ReplicationType type,
+ ReplicationFactor factor, ECReplicationConfig ecReplicationConfig) {
+ this.type = type;
+ this.factor = factor;
+ this.ecReplicationConfig = ecReplicationConfig;
+ }
+
+ public DefaultReplicationConfig(
+ org.apache.hadoop.hdds.protocol.proto.HddsProtos.DefaultReplicationConfig
+ defaultReplicationConfig) {
+ this.type = ReplicationType.fromProto(defaultReplicationConfig.getType());
+ if (defaultReplicationConfig.hasEcReplicationConfig()) {
+ this.ecReplicationConfig = new ECReplicationConfig(
+ defaultReplicationConfig.getEcReplicationConfig());
+ } else {
+ this.factor =
+ ReplicationFactor.fromProto(defaultReplicationConfig.getFactor());
+ }
+ }
+
+ public ReplicationType getType() {
+ return this.type;
+ }
+
+ public ReplicationFactor getFactor() {
+ return this.factor;
+ }
+
+ public DefaultReplicationConfig copy() {
+ return new DefaultReplicationConfig(this.type, this.factor,
+ this.ecReplicationConfig);
+ }
+
+ public ECReplicationConfig getEcReplicationConfig() {
+ return this.ecReplicationConfig;
+ }
+
+ public int getRequiredNodes() {
+ if (this.type == ReplicationType.EC) {
+ return ecReplicationConfig.getRequiredNodes();
+ }
+ return this.factor.getValue();
+ }
+
+ public HddsProtos.DefaultReplicationConfig toProto() {
+ final HddsProtos.DefaultReplicationConfig.Builder builder =
+ HddsProtos.DefaultReplicationConfig.newBuilder()
+ .setType(ReplicationType.toProto(this.type));
+ if (this.factor != null) {
+ builder.setFactor(ReplicationFactor.toProto(this.factor));
+ }
+ if (this.ecReplicationConfig != null) {
+ builder.setEcReplicationConfig(this.ecReplicationConfig.toProto());
+ }
+ return builder.build();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ DefaultReplicationConfig that = (DefaultReplicationConfig) o;
+ return Objects.equals(type, that.type) && Objects
+ .equals(factor, that.factor) && Objects
+ .equals(ecReplicationConfig, ecReplicationConfig);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(type, factor, ecReplicationConfig);
+ }
+}
+
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ECReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ECReplicationConfig.java
new file mode 100644
index 0000000..deb17cd
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ECReplicationConfig.java
@@ -0,0 +1,207 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
+import java.util.EnumSet;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Replication configuration for EC replication.
+ */
+public class ECReplicationConfig implements ReplicationConfig {
+
+ public static final String EC_REPLICATION_PARAMS_DELIMITER = "-";
+
+ /**
+ * Enum defining the allowed list of ECCodecs.
+ */
+ public enum EcCodec {
+ RS, XOR;
+
+ @Override
+ public String toString() {
+ return name().toLowerCase();
+ }
+
+ public static String allValuesAsString() {
+ return EnumSet.allOf(EcCodec.class)
+ .stream()
+ .map(Enum::toString)
+ .collect(Collectors.joining(","));
+ }
+ }
+
+ // Acceptable patterns are like:
+ // rs-3-2-1024k
+ // RS-3-2-2048
+ // XOR-10-4-4096K
+ private static final Pattern STRING_FORMAT
+ = Pattern.compile("([a-zA-Z]+)-(\\d+)-(\\d+)-(\\d+)((?:k|K))?");
+
+ private int data;
+
+ private int parity;
+
+ private int ecChunkSize = 1024 * 1024;
+
+ private EcCodec codec = EcCodec.RS;
+
+ public ECReplicationConfig(int data, int parity) {
+ this.data = data;
+ this.parity = parity;
+ }
+
+ public ECReplicationConfig(int data, int parity, EcCodec codec,
+ int ecChunkSize) {
+ this.data = data;
+ this.parity = parity;
+ this.codec = codec;
+ this.ecChunkSize = ecChunkSize;
+ }
+
+ /**
+ * Create an ECReplicationConfig object from a string representing the
+ * various parameters. Acceptable patterns are like:
+ * rs-3-2-1024k
+ * RS-3-2-2048
+ * XOR-10-4-4096K
+ * IllegalArgumentException will be thrown if the passed string does not
+ * match the defined pattern.
+ * @param string
+ */
+ public ECReplicationConfig(String string) {
+ final Matcher matcher = STRING_FORMAT.matcher(string);
+ if (!matcher.matches()) {
+ throw new IllegalArgumentException("EC replication config should be " +
+ "defined in the form rs-3-2-1024k, rs-6-3-1024; or rs-10-4-1024k." +
+ " Provided configuration was: " + string);
+ }
+
+ try {
+ codec = EcCodec.valueOf(matcher.group(1).toUpperCase());
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgumentException("The codec " + matcher.group(1) +
+ " is invalid. It must be one of " + EcCodec.allValuesAsString() + ".",
+ e);
+ }
+
+ data = Integer.parseInt(matcher.group(2));
+ parity = Integer.parseInt(matcher.group(3));
+ if (data <= 0 || parity <= 0) {
+ throw new IllegalArgumentException("Data and parity part in EC " +
+ "replication config supposed to be positive numbers");
+ }
+
+ int chunkSize = Integer.parseInt((matcher.group(4)));
+ if (chunkSize <= 0) {
+ throw new IllegalArgumentException("The ecChunkSize (" + chunkSize +
+ ") be greater than zero");
+ }
+ if (matcher.group(5) != null) {
+ // The "k" modifier is present, so multiple by 1024
+ chunkSize = chunkSize * 1024;
+ }
+ ecChunkSize = chunkSize;
+ }
+
+ public ECReplicationConfig(
+ HddsProtos.ECReplicationConfig ecReplicationConfig) {
+ this.data = ecReplicationConfig.getData();
+ this.parity = ecReplicationConfig.getParity();
+ this.codec = EcCodec.valueOf(ecReplicationConfig.getCodec().toUpperCase());
+ this.ecChunkSize = ecReplicationConfig.getEcChunkSize();
+ }
+
+ @Override
+ public HddsProtos.ReplicationType getReplicationType() {
+ return HddsProtos.ReplicationType.EC;
+ }
+
+ @Override
+ public int getRequiredNodes() {
+ return data + parity;
+ }
+
+ @Override
+ @JsonIgnore
+ public String getReplication() {
+ return getCodec() + EC_REPLICATION_PARAMS_DELIMITER
+ + getData() + EC_REPLICATION_PARAMS_DELIMITER
+ + getParity() + EC_REPLICATION_PARAMS_DELIMITER
+ + getEcChunkSize();
+ }
+
+ public HddsProtos.ECReplicationConfig toProto() {
+ return HddsProtos.ECReplicationConfig.newBuilder()
+ .setData(data)
+ .setParity(parity)
+ .setCodec(codec.toString())
+ .setEcChunkSize(ecChunkSize)
+ .build();
+ }
+
+ public int getData() {
+ return data;
+ }
+
+ public int getParity() {
+ return parity;
+ }
+
+ public int getEcChunkSize() {
+ return ecChunkSize;
+ }
+
+ public EcCodec getCodec() {
+ return codec;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ECReplicationConfig that = (ECReplicationConfig) o;
+ return data == that.data && parity == that.parity
+ && codec == that.getCodec() && ecChunkSize == that.getEcChunkSize();
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(data, parity, codec, ecChunkSize);
+ }
+
+ @Override
+ public String toString() {
+ return HddsProtos.ReplicationType.EC + "/ECReplicationConfig{"
+ + "data=" + data
+ + ", parity=" + parity
+ + ", ecChunkSize=" + ecChunkSize
+ + ", codec=" + codec + '}';
+ }
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java
index 6d294d1..b0c1dd2 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdds.client;
+import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@@ -92,6 +93,12 @@
}
@Override
+ @JsonIgnore
+ public String getReplication() {
+ return String.valueOf(replicationFactor);
+ }
+
+ @Override
public boolean equals(Object o) {
if (this == o) {
return true;
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java
index bcf4ea6..c9a12a6 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java
@@ -76,6 +76,28 @@
return parse(null, replication, config);
}
+ /**
+ * Helper method to serialize from proto.
+ * <p>
+ * This uses either the old type/factor or the new ecConfig depends on the
+ * type.
+ */
+ static ReplicationConfig fromProto(
+ HddsProtos.ReplicationType type,
+ HddsProtos.ReplicationFactor factor,
+ HddsProtos.ECReplicationConfig ecConfig) {
+ switch (type) {
+ case EC:
+ return new ECReplicationConfig(ecConfig);
+ case RATIS:
+ case STAND_ALONE:
+ return fromProtoTypeAndFactor(type, factor);
+ default:
+ throw new UnsupportedOperationException(
+ "Not supported replication: " + type);
+ }
+ }
+
static HddsProtos.ReplicationFactor getLegacyFactor(
ReplicationConfig replicationConfig) {
if (replicationConfig instanceof ReplicatedReplicationConfig) {
@@ -101,9 +123,12 @@
*/
static ReplicationConfig adjustReplication(
ReplicationConfig config, short replication, ConfigurationSource conf) {
- return parse(
- ReplicationType.valueOf(config.getReplicationType().toString()),
- Short.toString(replication), conf);
+ ReplicationType replicationType =
+ ReplicationType.valueOf(config.getReplicationType().toString());
+ if (replicationType.equals(ReplicationType.EC)) {
+ return config;
+ }
+ return parse(replicationType, Short.toString(replication), conf);
}
/**
@@ -134,6 +159,21 @@
replication = Objects.toString(replication,
config.get(OZONE_REPLICATION, OZONE_REPLICATION_DEFAULT));
+ return parseWithoutFallback(type, replication, config);
+ }
+
+ static ReplicationConfig parseWithoutFallback(ReplicationType type,
+ String replication, ConfigurationSource config) {
+
+ if (replication == null) {
+ throw new IllegalArgumentException(
+ "Replication can't be null. Replication type passed was : " + type);
+ }
+ if (type == null) {
+ throw new IllegalArgumentException(
+ "Replication type must be specified for: " + replication);
+ }
+
ReplicationConfig replicationConfig;
switch (type) {
case RATIS:
@@ -146,6 +186,8 @@
}
replicationConfig = fromTypeAndFactor(type, factor);
break;
+ case EC:
+ return new ECReplicationConfig(replication);
default:
throw new RuntimeException("Replication type" + type + " can not"
+ "be parsed.");
@@ -168,4 +210,9 @@
*/
int getRequiredNodes();
+ /**
+ * Returns the replication option in string format.
+ */
+ String getReplication();
+
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationFactor.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationFactor.java
index 8623a0e..a249040 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationFactor.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationFactor.java
@@ -71,6 +71,22 @@
}
}
+ public static HddsProtos.ReplicationFactor toProto(
+ ReplicationFactor replicationFactor) {
+ if (replicationFactor == null) {
+ return null;
+ }
+ switch (replicationFactor) {
+ case ONE:
+ return HddsProtos.ReplicationFactor.ONE;
+ case THREE:
+ return HddsProtos.ReplicationFactor.THREE;
+ default:
+ throw new IllegalArgumentException(
+ "Unsupported ProtoBuf replication factor: " + replicationFactor);
+ }
+ }
+
/**
* Returns integer representation of ReplicationFactor.
* @return replication value
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java
index 11a8218..64969eac 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java
@@ -26,7 +26,8 @@
public enum ReplicationType {
RATIS,
STAND_ALONE,
- CHAINED;
+ CHAINED,
+ EC;
public static ReplicationType fromProto(
HddsProtos.ReplicationType replicationType) {
@@ -40,9 +41,31 @@
return ReplicationType.STAND_ALONE;
case CHAINED:
return ReplicationType.CHAINED;
+ case EC:
+ return ReplicationType.EC;
default:
throw new IllegalArgumentException(
"Unsupported ProtoBuf replication type: " + replicationType);
}
}
+
+ public static HddsProtos.ReplicationType toProto(
+ ReplicationType replicationType) {
+ if (replicationType == null) {
+ return null;
+ }
+ switch (replicationType) {
+ case RATIS:
+ return HddsProtos.ReplicationType.RATIS;
+ case STAND_ALONE:
+ return HddsProtos.ReplicationType.STAND_ALONE;
+ case CHAINED:
+ return HddsProtos.ReplicationType.CHAINED;
+ case EC:
+ return HddsProtos.ReplicationType.EC;
+ default:
+ throw new IllegalArgumentException(
+ "Unsupported replication type: " + replicationType);
+ }
+ }
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java
index 213e6c4..7befffe 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/StandaloneReplicationConfig.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdds.client;
+import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@@ -78,6 +79,12 @@
}
@Override
+ @JsonIgnore
+ public String getReplication() {
+ return String.valueOf(this.replicationFactor);
+ }
+
+ @Override
public ReplicationType getReplicationType() {
return ReplicationType.STAND_ALONE;
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
index fe4ec99..2a962d7 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
@@ -341,6 +341,8 @@
public static final String OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY =
"ozone.scm.container.placement.impl";
+ public static final String OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY =
+ "ozone.scm.container.placement.ec.impl";
public static final String OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT =
"ozone.scm.pipeline.owner.container.count";
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
index e2a5c8c..dd56903 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
@@ -19,6 +19,7 @@
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.hdds.annotation.InterfaceStability;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto;
import org.apache.hadoop.hdds.scm.DatanodeAdminError;
import org.apache.hadoop.hdds.scm.container.ContainerReplicaInfo;
@@ -127,12 +128,14 @@
* @param startContainerID start containerID.
* @param count count must be {@literal >} 0.
* @param state Container of this state will be returned.
- * @param factor container factor.
+ * @param replicationConfig container replication Config.
* @return a list of pipeline.
* @throws IOException
*/
List<ContainerInfo> listContainer(long startContainerID, int count,
- HddsProtos.LifeCycleState state, HddsProtos.ReplicationFactor factor)
+ HddsProtos.LifeCycleState state,
+ HddsProtos.ReplicationType replicationType,
+ ReplicationConfig replicationConfig)
throws IOException;
/**
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 0071924..e99b6a5 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
@@ -26,6 +26,7 @@
import java.util.Comparator;
import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
@@ -114,8 +115,8 @@
public static ContainerInfo fromProtobuf(HddsProtos.ContainerInfoProto info) {
ContainerInfo.Builder builder = new ContainerInfo.Builder();
final ReplicationConfig config = ReplicationConfig
- .fromProtoTypeAndFactor(
- info.getReplicationType(), info.getReplicationFactor());
+ .fromProto(info.getReplicationType(), info.getReplicationFactor(),
+ info.getEcReplicationConfig());
builder.setUsedBytes(info.getUsedBytes())
.setNumberOfKeys(info.getNumberOfKeys())
.setState(info.getState())
@@ -159,10 +160,12 @@
return replicationConfig;
}
+ @JsonIgnore
public HddsProtos.ReplicationType getReplicationType() {
return replicationConfig.getReplicationType();
}
+ @JsonIgnore
public HddsProtos.ReplicationFactor getReplicationFactor() {
return ReplicationConfig.getLegacyFactor(replicationConfig);
}
@@ -171,6 +174,28 @@
return pipelineID;
}
+ /**
+ * Returns the usedBytes for the container. The value returned is derived
+ * from the replicas reported from the datanodes for the container.
+ *
+ * The size of a container can change over time. For an open container we
+ * assume the size of the container will grow, and hence the value will be
+ * the maximum of the values reported from its replicas.
+ *
+ * A closed container can only reduce in size as its blocks are removed. For
+ * a closed container, the value will be the minimum of the values reported
+ * from its replicas.
+ *
+ * An EC container, is made from a group data and parity containers where the
+ * first data and all parity containers should be the same size. The other
+ * data containers can be smaller or the same size. When calculating the EC
+ * container size, we use the min / max of the first data and parity
+ * containers,ignoring the others. For EC containers, this value actually
+ * represents the size of the largest container in the container group, rather
+ * than the total space used by all containers in the group.
+ *
+ * @return bytes used in the container.
+ */
public long getUsedBytes() {
return usedBytes;
}
@@ -232,10 +257,17 @@
.setContainerID(getContainerID())
.setDeleteTransactionId(getDeleteTransactionId())
.setOwner(getOwner())
- .setSequenceId(getSequenceId());
+ .setSequenceId(getSequenceId())
+ .setReplicationType(getReplicationType());
- builder.setReplicationFactor(
- ReplicationConfig.getLegacyFactor(replicationConfig));
+ if (replicationConfig instanceof ECReplicationConfig) {
+ builder.setEcReplicationConfig(((ECReplicationConfig) replicationConfig)
+ .toProto());
+ } else {
+ builder.setReplicationFactor(
+ ReplicationConfig.getLegacyFactor(replicationConfig));
+ }
+
builder.setReplicationType(replicationConfig.getReplicationType());
if (getPipelineID() != null) {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaInfo.java
index b30dff7..5a81f6b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaInfo.java
@@ -34,6 +34,7 @@
private long sequenceId;
private long keyCount;
private long bytesUsed;
+ private int replicaIndex = -1;
public static ContainerReplicaInfo fromProto(
HddsProtos.SCMContainerReplicaProto proto) {
@@ -45,7 +46,9 @@
.setPlaceOfBirth(UUID.fromString(proto.getPlaceOfBirth()))
.setSequenceId(proto.getSequenceID())
.setKeyCount(proto.getKeyCount())
- .setBytesUsed(proto.getBytesUsed());
+ .setBytesUsed(proto.getBytesUsed())
+ .setReplicaIndex(
+ proto.hasReplicaIndex() ? (int)proto.getReplicaIndex() : -1);
return builder.build();
}
@@ -80,6 +83,10 @@
return bytesUsed;
}
+ public int getReplicaIndex() {
+ return replicaIndex;
+ }
+
/**
* Builder for ContainerReplicaInfo class.
*/
@@ -122,6 +129,11 @@
return this;
}
+ public Builder setReplicaIndex(int replicaIndex) {
+ subject.replicaIndex = replicaIndex;
+ return this;
+ }
+
public ContainerReplicaInfo build() {
return subject;
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java
index 824a1f5..026b313 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ExcludeList.java
@@ -17,16 +17,20 @@
package org.apache.hadoop.hdds.scm.container.common.helpers;
-
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.ozone.common.MonotonicClock;
+import java.time.ZoneOffset;
import java.util.Collection;
import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
/**
* This class contains set of dns and containers which ozone client provides
@@ -34,15 +38,24 @@
*/
public class ExcludeList {
- private final Set<DatanodeDetails> datanodes;
+ private final Map<DatanodeDetails, Long> datanodes;
private final Set<ContainerID> containerIds;
private final Set<PipelineID> pipelineIds;
+ private long expiryTime = 0;
+ private java.time.Clock clock;
public ExcludeList() {
- datanodes = new HashSet<>();
+ datanodes = new ConcurrentHashMap<>();
containerIds = new HashSet<>();
pipelineIds = new HashSet<>();
+ clock = new MonotonicClock(ZoneOffset.UTC);
+ }
+
+ public ExcludeList(long autoExpiryTime, java.time.Clock clock) {
+ this();
+ this.expiryTime = autoExpiryTime;
+ this.clock = clock;
}
public Set<ContainerID> getContainerIds() {
@@ -50,7 +63,23 @@
}
public Set<DatanodeDetails> getDatanodes() {
- return datanodes;
+ Set<DatanodeDetails> dns = new HashSet<>();
+ if (expiryTime > 0) {
+ Iterator<Map.Entry<DatanodeDetails, Long>> iterator =
+ datanodes.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Map.Entry<DatanodeDetails, Long> entry = iterator.next();
+ Long storedExpiryTime = entry.getValue();
+ if (clock.millis() > storedExpiryTime) {
+ iterator.remove(); // removing
+ } else {
+ dns.add(entry.getKey());
+ }
+ }
+ } else {
+ dns = datanodes.keySet();
+ }
+ return dns;
}
public void addDatanodes(Collection<DatanodeDetails> dns) {
@@ -58,7 +87,7 @@
}
public void addDatanode(DatanodeDetails dn) {
- datanodes.add(dn);
+ datanodes.put(dn, clock.millis() + expiryTime);
}
public void addConatinerId(ContainerID containerId) {
@@ -78,9 +107,7 @@
HddsProtos.ExcludeListProto.newBuilder();
containerIds
.forEach(id -> builder.addContainerIds(id.getId()));
- datanodes.forEach(dn -> {
- builder.addDatanodes(dn.getUuidString());
- });
+ getDatanodes().forEach(dn -> builder.addDatanodes(dn.getUuidString()));
pipelineIds.forEach(pipelineID -> {
builder.addPipelineIds(pipelineID.getProtobuf());
});
@@ -105,7 +132,7 @@
}
public boolean isEmpty() {
- return datanodes.isEmpty() && containerIds.isEmpty() && pipelineIds
+ return getDatanodes().isEmpty() && containerIds.isEmpty() && pipelineIds
.isEmpty();
}
@@ -118,9 +145,10 @@
@Override
public String toString() {
return "ExcludeList {" +
- "datanodes = " + datanodes +
+ "datanodes = " + getDatanodes() +
", containerIds = " + containerIds +
", pipelineIds = " + pipelineIds +
'}';
}
+
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
index f5c0b62..116e767 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
@@ -23,6 +23,7 @@
import java.time.ZoneId;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
@@ -34,6 +35,7 @@
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -55,6 +57,7 @@
private PipelineState state;
private Map<DatanodeDetails, Long> nodeStatus;
+ private Map<DatanodeDetails, Integer> replicaIndexes;
// nodes with ordered distance to client
private ThreadLocal<List<DatanodeDetails>> nodesInOrder = new ThreadLocal<>();
// Current reported Leader for the pipeline
@@ -78,6 +81,7 @@
this.nodeStatus = nodeStatus;
this.creationTimestamp = Instant.now();
this.suggestedLeaderId = suggestedLeaderId;
+ this.replicaIndexes = new HashMap<>();
}
/**
@@ -178,6 +182,21 @@
/**
+ * Return the replica index of the specific datanode in the datanode set.
+ * <p>
+ * For non-EC case all the replication should be exactly the same,
+ * therefore the replication index can always be zero. In case of EC
+ * different Datanodes can have different data for one specific block
+ * (parity and/or data parts) therefore the replicaIndex should be
+ * different for each of the pipeline members.
+ *
+ * @param dn datanode details
+ */
+ public int getReplicaIndex(DatanodeDetails dn) {
+ return replicaIndexes.getOrDefault(dn, 0);
+ }
+
+ /**
* Returns the leader if found else defaults to closest node.
*
* @return {@link DatanodeDetails}
@@ -245,6 +264,12 @@
}
public boolean isHealthy() {
+ // EC pipelines are not reported by the DN and do not have a leader. If a
+ // node goes stale or dead, EC pipelines will by closed like RATIS pipelines
+ // but at the current time there are not other health metrics for EC.
+ if (replicationConfig.getReplicationType() == ReplicationType.EC) {
+ return true;
+ }
for (Long reportedTime : nodeStatus.values()) {
if (reportedTime < 0) {
return false;
@@ -263,20 +288,30 @@
public HddsProtos.Pipeline getProtobufMessage(int clientVersion)
throws UnknownPipelineStateException {
+
List<HddsProtos.DatanodeDetailsProto> members = new ArrayList<>();
+ List<Integer> memberReplicaIndexes = new ArrayList<>();
+
for (DatanodeDetails dn : nodeStatus.keySet()) {
members.add(dn.toProto(clientVersion));
+ memberReplicaIndexes.add(replicaIndexes.getOrDefault(dn, 0));
}
HddsProtos.Pipeline.Builder builder = HddsProtos.Pipeline.newBuilder()
.setId(id.getProtobuf())
.setType(replicationConfig.getReplicationType())
- .setFactor(ReplicationConfig.getLegacyFactor(replicationConfig))
.setState(PipelineState.getProtobuf(state))
.setLeaderID(leaderId != null ? leaderId.toString() : "")
.setCreationTimeStamp(creationTimestamp.toEpochMilli())
- .addAllMembers(members);
+ .addAllMembers(members)
+ .addAllMemberReplicaIndexes(memberReplicaIndexes);
+ if (replicationConfig instanceof ECReplicationConfig) {
+ builder.setEcReplicationConfig(((ECReplicationConfig) replicationConfig)
+ .toProto());
+ } else {
+ builder.setFactor(ReplicationConfig.getLegacyFactor(replicationConfig));
+ }
if (leaderId != null) {
HddsProtos.UUID uuid128 = HddsProtos.UUID.newBuilder()
.setMostSigBits(leaderId.getMostSignificantBits())
@@ -317,9 +352,16 @@
throws UnknownPipelineStateException {
Preconditions.checkNotNull(pipeline, "Pipeline is null");
- List<DatanodeDetails> nodes = new ArrayList<>();
+ Map<DatanodeDetails, Integer> nodes = new LinkedHashMap<>();
+ int index = 0;
+ int repIndexListLength = pipeline.getMemberReplicaIndexesCount();
for (DatanodeDetailsProto member : pipeline.getMembersList()) {
- nodes.add(DatanodeDetails.getFromProtoBuf(member));
+ int repIndex = 0;
+ if (index < repIndexListLength) {
+ repIndex = pipeline.getMemberReplicaIndexes(index);
+ }
+ nodes.put(DatanodeDetails.getFromProtoBuf(member), repIndex);
+ index++;
}
UUID leaderId = null;
if (pipeline.hasLeaderID128()) {
@@ -338,11 +380,13 @@
}
final ReplicationConfig config = ReplicationConfig
- .fromProtoTypeAndFactor(pipeline.getType(), pipeline.getFactor());
+ .fromProto(pipeline.getType(), pipeline.getFactor(),
+ pipeline.getEcReplicationConfig());
return new Builder().setId(PipelineID.getFromProtobuf(pipeline.getId()))
.setReplicationConfig(config)
.setState(PipelineState.fromProtobuf(pipeline.getState()))
- .setNodes(nodes)
+ .setNodes(new ArrayList<>(nodes.keySet()))
+ .setReplicaIndexes(nodes)
.setLeaderId(leaderId)
.setSuggestedLeaderId(suggestedLeaderId)
.setNodesInOrder(pipeline.getMemberOrdersList())
@@ -401,6 +445,10 @@
return new Builder(pipeline);
}
+ private void setReplicaIndexes(Map<DatanodeDetails, Integer> replicaIndexes) {
+ this.replicaIndexes = replicaIndexes;
+ }
+
/**
* Builder class for Pipeline.
*/
@@ -414,6 +462,7 @@
private UUID leaderId = null;
private Instant creationTimestamp = null;
private UUID suggestedLeaderId = null;
+ private Map<DatanodeDetails, Integer> replicaIndexes = new HashMap<>();
public Builder() { }
@@ -426,6 +475,15 @@
this.leaderId = pipeline.getLeaderId();
this.creationTimestamp = pipeline.getCreationTimestamp();
this.suggestedLeaderId = pipeline.getSuggestedLeaderId();
+ this.replicaIndexes = new HashMap<>();
+ if (nodeStatus != null) {
+ for (DatanodeDetails dn : nodeStatus.keySet()) {
+ int index = pipeline.getReplicaIndex(dn);
+ if (index > 0) {
+ replicaIndexes.put(dn, index);
+ }
+ }
+ }
}
public Builder setId(PipelineID id1) {
@@ -447,7 +505,6 @@
this.leaderId = leaderId1;
return this;
}
-
public Builder setNodes(List<DatanodeDetails> nodes) {
this.nodeStatus = new LinkedHashMap<>();
nodes.forEach(node -> nodeStatus.put(node, -1L));
@@ -469,6 +526,12 @@
return this;
}
+
+ public Builder setReplicaIndexes(Map<DatanodeDetails, Integer> indexes) {
+ this.replicaIndexes = indexes;
+ return this;
+ }
+
public Pipeline build() {
Preconditions.checkNotNull(id);
Preconditions.checkNotNull(replicationConfig);
@@ -483,6 +546,8 @@
pipeline.setCreationTimestamp(creationTimestamp);
}
+ pipeline.setReplicaIndexes(replicaIndexes);
+
if (nodeOrder != null && !nodeOrder.isEmpty()) {
// This branch is for build from ProtoBuf
List<DatanodeDetails> nodesWithOrder = new ArrayList<>();
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
index cd94100..d915a20 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdds.scm.protocol;
import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type;
@@ -184,6 +185,28 @@
int count, HddsProtos.LifeCycleState state,
HddsProtos.ReplicationFactor factor) throws IOException;
+
+ /**
+ * Ask SCM for a list of containers with a range of container ID, state
+ * and replication config, and the limit of count.
+ * The containers are returned from startID (exclusive), and
+ * filtered by state and replication config. The returned list is limited to
+ * count entries.
+ *
+ * @param startContainerID start container ID.
+ * @param count count, if count {@literal <} 0, the max size is unlimited.(
+ * Usually the count will be replace with a very big
+ * value instead of being unlimited in case the db is very big)
+ * @param state Container with this state will be returned.
+ * @param replicationConfig Replication config for the containers
+ * @return a list of container.
+ * @throws IOException
+ */
+ List<ContainerInfo> listContainer(long startContainerID,
+ int count, HddsProtos.LifeCycleState state,
+ HddsProtos.ReplicationType replicationType,
+ ReplicationConfig replicationConfig) throws IOException;
+
/**
* Deletes a container in SCM.
*
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
index 7f2d2a8..2597870 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
@@ -282,17 +282,27 @@
*/
public static XceiverClientReply writeChunkAsync(
XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID,
- ByteString data, Token<? extends TokenIdentifier> token)
+ ByteString data, Token<? extends TokenIdentifier> token,
+ int replicationIndex
+ )
throws IOException, ExecutionException, InterruptedException {
WriteChunkRequestProto.Builder writeChunkRequest =
WriteChunkRequestProto.newBuilder()
- .setBlockID(blockID.getDatanodeBlockIDProtobuf())
- .setChunkData(chunk).setData(data);
+ .setBlockID(DatanodeBlockID.newBuilder()
+ .setContainerID(blockID.getContainerID())
+ .setLocalID(blockID.getLocalID())
+ .setBlockCommitSequenceId(blockID.getBlockCommitSequenceId())
+ .setReplicaIndex(replicationIndex)
+ .build())
+ .setChunkData(chunk)
+ .setData(data);
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto.Builder builder =
- ContainerCommandRequestProto.newBuilder().setCmdType(Type.WriteChunk)
+ ContainerCommandRequestProto.newBuilder()
+ .setCmdType(Type.WriteChunk)
.setContainerID(blockID.getContainerID())
- .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+ .setDatanodeUuid(id)
+ .setWriteChunk(writeChunkRequest);
if (token != null) {
builder.setEncodedToken(token.encodeToUrlString());
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java
index 2bc1a67..4217055 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java
@@ -31,7 +31,9 @@
INITIAL_VERSION(0, "Initial Layout Version"),
DATANODE_SCHEMA_V2(1, "Datanode RocksDB Schema Version 2 (with column " +
"families)"),
- SCM_HA(2, "Storage Container Manager HA");
+ SCM_HA(2, "Storage Container Manager HA"),
+ ERASURE_CODED_STORAGE_SUPPORT(3, "Ozone version with built in support for"
+ + " Erasure Coded block data storage.");
////////////////////////////// //////////////////////////////
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java
index ec00ca2..b8e2542 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java
@@ -35,6 +35,9 @@
VERSION_HANDLES_UNKNOWN_DN_PORTS(1,
"Client version that handles the REPLICATION port in DatanodeDetails."),
+ ERASURE_CODING_SUPPORT(2, "This client version has support for Erasure"
+ + " Coding."),
+
FUTURE_VERSION(-1, "Used internally when the server side is older and an"
+ " unknown client version has arrived from the client.");
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
index d5794e5..699d732 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -115,6 +115,11 @@
"ozone.scm.block.size";
public static final String OZONE_SCM_BLOCK_SIZE_DEFAULT = "256MB";
+ public static final String OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES =
+ "ozone.client.max.ec.stripe.write.retries";
+ public static final String OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES_DEFAULT =
+ "10";
+
/**
* Ozone administrator users delimited by comma.
* If not set, only the user who launches an ozone service will be the
@@ -455,6 +460,13 @@
public static final String OZONE_CLIENT_REQUIRED_OM_VERSION_MIN_DEFAULT =
OzoneManagerVersion.S3G_PERSISTENT_CONNECTIONS.name();
+ public static final String
+ OZONE_CLIENT_BUCKET_REPLICATION_CONFIG_REFRESH_PERIOD_MS =
+ "ozone.client.bucket.replication.config.refresh.time.ms";
+ public static final long
+ OZONE_CLIENT_BUCKET_REPLICATION_CONFIG_REFRESH_PERIOD_DEFAULT_MS =
+ 300 * 1000;
+
public static final String OZONE_AUDIT_LOG_DEBUG_CMD_LIST_OMAUDIT =
"ozone.audit.log.debug.cmd.list.omaudit";
/**
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 0430283..a9198bd 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -261,6 +261,7 @@
public static final String ORIGIN_PIPELINE_ID = "originPipelineId";
public static final String ORIGIN_NODE_ID = "originNodeId";
public static final String SCHEMA_VERSION = "schemaVersion";
+ public static final String REPLICA_INDEX = "replicaIndex";
// Supported .container datanode schema versions.
// Since containers in older schema versions are currently not reformatted to
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java
index ff3d112..c6872f2 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneManagerVersion.java
@@ -32,6 +32,8 @@
DEFAULT_VERSION(0, "Initial version"),
S3G_PERSISTENT_CONNECTIONS(1,
"New S3G persistent connection support is present in OM."),
+ ERASURE_CODED_STORAGE_SUPPORT(2, "OzoneManager version that supports"
+ + "ECReplicationConfig"),
FUTURE_VERSION(-1, "Used internally in the client when the server side is "
+ " newer and an unknown server version has arrived to the client.");
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 703e3c5..aaa7859 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -811,6 +811,20 @@
</description>
</property>
<property>
+ <name>ozone.scm.container.placement.ec.impl</name>
+ <value>
+ org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRackScatter
+ </value>
+ <tag>OZONE, MANAGEMENT</tag>
+ <description>
+ The full name of class which implements
+ org.apache.hadoop.hdds.scm.PlacementPolicy.
+ The class decides which datanode will be used to host the container replica in EC mode. If not set,
+ org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom will be used as default
+ value.
+ </description>
+ </property>
+ <property>
<name>ozone.scm.pipeline.owner.container.count</name>
<value>3</value>
<tag>OZONE, SCM, PIPELINE</tag>
@@ -1145,26 +1159,41 @@
</property>
<property>
- <name>ozone.replication</name>
+ <name>ozone.server.default.replication</name>
<value>3</value>
- <tag>OZONE, CLIENT</tag>
+ <tag>OZONE</tag>
<description>
Default replication value. The actual number of replications can be
specified when writing the key. The default is used if replication
- is not specified. Supported values: 1 and 3.
+ is not specified when creating key or no default replication set at
+ bucket. Supported values: 1, 3 and EC_3_2.
</description>
</property>
<property>
- <name>ozone.replication.type</name>
+ <name>ozone.server.default.replication.type</name>
<value>RATIS</value>
- <tag>OZONE, CLIENT</tag>
+ <tag>OZONE</tag>
<description>
Default replication type to be used while writing key into ozone. The
value can be specified when writing the key, default is used when
- nothing is specified. Supported values: RATIS, STAND_ALONE and CHAINED.
+ nothing is specified when creating key or no default value set at bucket.
+ Supported values: RATIS, STAND_ALONE, CHAINED and EC.
</description>
</property>
+
+ <property>
+ <name>ozone.client.bucket.replication.config.refresh.time.ms</name>
+ <value>30000</value>
+ <tag>OZONE</tag>
+ <description>
+ Default time period to refresh the bucket replication config in o3fs
+ clients. Until the bucket replication config refreshed, client will
+ continue to use existing replication config irrespective of whether bucket
+ replication config updated at OM or not.
+ </description>
+ </property>
+
<property>
<name>hdds.container.close.threshold</name>
<value>0.9f</value>
@@ -3090,6 +3119,18 @@
</property>
<property>
+ <name>ozone.client.max.ec.stripe.write.retries</name>
+ <value>10</value>
+ <tag>CLIENT</tag>
+ <description>
+ When EC stripe write failed, client will request to allocate new block group and write the failed stripe into new
+ block group. If the same stripe failure continued in newly acquired block group also, then it will retry by
+ requesting to allocate new block group again. This configuration is used to limit these number of retries. By
+ default the number of retries are 10.
+ </description>
+ </property>
+
+ <property>
<name>ozone.audit.log.debug.cmd.list.omaudit</name>
<value></value>
<tag>OM</tag>
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestECReplicationConfig.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestECReplicationConfig.java
new file mode 100644
index 0000000..9440a6e
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestECReplicationConfig.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec.RS;
+import static org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec.XOR;
+import static org.junit.Assert.fail;
+
+/**
+ * Unit test for ECReplicationConfig.
+ */
+public class TestECReplicationConfig {
+
+ @Test
+ public void testSuccessfulStringParsing() {
+ Map<String, ECReplicationConfig> valid = new HashMap();
+ valid.put("rs-3-2-1024", new ECReplicationConfig(3, 2, RS, 1024));
+ valid.put("RS-3-2-1024", new ECReplicationConfig(3, 2, RS, 1024));
+ valid.put("rs-3-2-1024k", new ECReplicationConfig(3, 2, RS, 1024 * 1024));
+ valid.put("rs-3-2-1024K", new ECReplicationConfig(3, 2, RS, 1024 * 1024));
+ valid.put("xor-10-4-1", new ECReplicationConfig(10, 4, XOR, 1));
+ valid.put("XOR-6-3-12345", new ECReplicationConfig(6, 3, XOR, 12345));
+
+ for (Map.Entry<String, ECReplicationConfig> e : valid.entrySet()) {
+ ECReplicationConfig ec = new ECReplicationConfig(e.getKey());
+ Assert.assertEquals(e.getValue().getData(), ec.getData());
+ Assert.assertEquals(e.getValue().getParity(), ec.getParity());
+ Assert.assertEquals(e.getValue().getCodec(), ec.getCodec());
+ Assert.assertEquals(e.getValue().getEcChunkSize(), ec.getEcChunkSize());
+ }
+ }
+
+ @Test
+ public void testUnsuccessfulStringParsing() {
+ String[] invalid = {
+ "3-2-1024",
+ "rss-3-2-1024",
+ "rs-3-0-1024",
+ "rs-3-2-0k",
+ "rs-3-2",
+ "x3-2"
+ };
+ for (String s : invalid) {
+ try {
+ new ECReplicationConfig(s);
+ fail(s + " should not parse correctly");
+ } catch (IllegalArgumentException e) {
+ // ignore, this expected
+ }
+ }
+ }
+
+
+ @Test
+ public void testSerializeToProtoAndBack() {
+ ECReplicationConfig orig = new ECReplicationConfig(6, 3,
+ ECReplicationConfig.EcCodec.XOR, 1024);
+
+ HddsProtos.ECReplicationConfig proto = orig.toProto();
+
+ ECReplicationConfig recovered = new ECReplicationConfig(proto);
+ Assert.assertEquals(orig.getData(), recovered.getData());
+ Assert.assertEquals(orig.getParity(), recovered.getParity());
+ Assert.assertEquals(orig.getCodec(), recovered.getCodec());
+ Assert.assertEquals(orig.getEcChunkSize(), recovered.getEcChunkSize());
+ Assert.assertTrue(orig.equals(recovered));
+ }
+
+}
\ No newline at end of file
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestReplicationConfig.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestReplicationConfig.java
index 1315ad5..ed8c8f7 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestReplicationConfig.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/client/TestReplicationConfig.java
@@ -17,10 +17,14 @@
*/
package org.apache.hadoop.hdds.client;
+import org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
+import org.junit.Assert;
+import org.junit.Assume;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@@ -36,8 +40,11 @@
@RunWith(Parameterized.class)
public class TestReplicationConfig {
+ private static final int MB = 1024 * 1024;
+ private static final int KB = 1024;
+
@SuppressWarnings("checkstyle:VisibilityModifier")
- @Parameterized.Parameter()
+ @Parameterized.Parameter
public String type;
@SuppressWarnings("checkstyle:VisibilityModifier")
@@ -46,24 +53,49 @@
@SuppressWarnings("checkstyle:VisibilityModifier")
@Parameterized.Parameter(2)
+ public String codec;
+
+ @SuppressWarnings("checkstyle:VisibilityModifier")
+ @Parameterized.Parameter(3)
+ public int data;
+
+ @SuppressWarnings("checkstyle:VisibilityModifier")
+ @Parameterized.Parameter(4)
+ public int parity;
+
+ @SuppressWarnings("checkstyle:VisibilityModifier")
+ @Parameterized.Parameter(5)
+ public int chunkSize;
+
+ @SuppressWarnings("checkstyle:VisibilityModifier")
+ @Parameterized.Parameter(6)
public Class<?> replicationConfigClass;
+ //NOTE: if a new cunckSize is used/added in the parameters other than KB or MB
+ // please revisit the method createECDescriptor, to handle the new chunkSize.
@Parameterized.Parameters(name = "{0}/{1}")
public static Object[][] parameters() {
return new Object[][] {
- {"RATIS", "ONE", RatisReplicationConfig.class },
- {"RATIS", "THREE", RatisReplicationConfig.class},
- {"STAND_ALONE", "ONE", StandaloneReplicationConfig.class},
- {"STAND_ALONE", "THREE", StandaloneReplicationConfig.class}
+ {"RATIS", "ONE", null, 0, 0, 0, RatisReplicationConfig.class },
+ {"RATIS", "THREE", null, 0, 0, 0, RatisReplicationConfig.class},
+ {"STAND_ALONE", "ONE", null, 0, 0, 0,
+ StandaloneReplicationConfig.class},
+ {"STAND_ALONE", "THREE", null, 0, 0, 0,
+ StandaloneReplicationConfig.class},
+ {"EC", "RS-3-2-1024K", "RS", 3, 2, MB, ECReplicationConfig.class},
+ {"EC", "RS-3-2-1024", "RS", 3, 2, KB, ECReplicationConfig.class},
+ {"EC", "RS-6-3-1024K", "RS", 6, 3, MB, ECReplicationConfig.class},
+ {"EC", "RS-6-3-1024", "RS", 6, 3, KB, ECReplicationConfig.class},
+ {"EC", "RS-10-4-1024K", "RS", 10, 4, MB, ECReplicationConfig.class},
+ {"EC", "RS-10-4-1024", "RS", 10, 4, KB, ECReplicationConfig.class},
};
}
@Test
- public void testGetDefaultShouldCreateReplicationConfigFromDefaultConf() {
+ public void testGetDefaultShouldReturnRatisThreeIfNotSetClientSide() {
OzoneConfiguration conf = new OzoneConfiguration();
ReplicationConfig replicationConfig = ReplicationConfig.getDefault(conf);
-
validate(replicationConfig,
org.apache.hadoop.hdds.client.ReplicationType.RATIS,
org.apache.hadoop.hdds.client.ReplicationFactor.THREE,
@@ -71,7 +103,8 @@
}
@Test
- public void testGetDefaultShouldCreateReplicationConfFromCustomConfValues() {
+ public void testGetDefaultShouldCreateReplicationConfFromConfValues() {
+ assumeRatisOrStandaloneType();
OzoneConfiguration conf = new OzoneConfiguration();
conf.set(OZONE_REPLICATION_TYPE, type);
conf.set(OZONE_REPLICATION, factor);
@@ -84,7 +117,22 @@
}
@Test
+ public void testGetDefaultShouldCreateECReplicationConfFromConfValues() {
+ assumeECType();
+
+ OzoneConfiguration conf = new OzoneConfiguration();
+ conf.set(OZONE_REPLICATION_TYPE, type);
+ conf.set(OZONE_REPLICATION, ecDescriptor());
+
+ ReplicationConfig replicationConfig = ReplicationConfig.getDefault(conf);
+
+ validate(replicationConfig,
+ EcCodec.valueOf(codec), data, parity, chunkSize);
+ }
+
+ @Test
public void deserialize() {
+ assumeRatisOrStandaloneType();
final ReplicationConfig replicationConfig =
ReplicationConfig.fromProtoTypeAndFactor(
ReplicationType.valueOf(type),
@@ -96,7 +144,53 @@
}
@Test
+ public void deserializeEC() {
+ assumeECType();
+ HddsProtos.ECReplicationConfig proto =
+ HddsProtos.ECReplicationConfig.newBuilder()
+ .setCodec(codec)
+ .setData(data)
+ .setParity(parity)
+ .setEcChunkSize(chunkSize)
+ .build();
+
+ ReplicationConfig config = ReplicationConfig
+ .fromProto(ReplicationType.EC, null, proto);
+
+ validate(config, EcCodec.valueOf(codec), data, parity, chunkSize);
+ }
+
+ @Test
+ public void testECReplicationConfigGetReplication() {
+ assumeECType();
+ HddsProtos.ECReplicationConfig proto =
+ HddsProtos.ECReplicationConfig.newBuilder().setCodec(codec)
+ .setData(data).setParity(parity).setEcChunkSize(chunkSize).build();
+
+ ReplicationConfig config =
+ ReplicationConfig.fromProto(ReplicationType.EC, null, proto);
+
+ Assert.assertEquals(EcCodec.valueOf(
+ codec) + ECReplicationConfig.EC_REPLICATION_PARAMS_DELIMITER
+ + data + ECReplicationConfig.EC_REPLICATION_PARAMS_DELIMITER
+ + parity + ECReplicationConfig.EC_REPLICATION_PARAMS_DELIMITER
+ + chunkSize, config.getReplication());
+ }
+
+ @Test
+ public void testReplicationConfigGetReplication() {
+ assumeRatisOrStandaloneType();
+ final ReplicationConfig replicationConfig = ReplicationConfig
+ .fromTypeAndFactor(
+ org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
+ org.apache.hadoop.hdds.client.ReplicationFactor.valueOf(factor));
+
+ Assert.assertEquals(factor, replicationConfig.getReplication());
+ }
+
+ @Test
public void fromJavaObjects() {
+ assumeRatisOrStandaloneType();
final ReplicationConfig replicationConfig =
ReplicationConfig.fromTypeAndFactor(
org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
@@ -109,6 +203,7 @@
@Test
public void testParseFromTypeAndFactorAsString() {
+ assumeRatisOrStandaloneType();
ConfigurationSource conf = new OzoneConfiguration();
ReplicationConfig replicationConfig = ReplicationConfig.parse(
org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
@@ -121,10 +216,11 @@
@Test
public void testParseFromTypeAndFactorAsStringifiedInteger() {
+ assumeRatisOrStandaloneType();
ConfigurationSource conf = new OzoneConfiguration();
String f =
- factor == "ONE" ? "1"
- : factor == "THREE" ? "3"
+ factor.equals("ONE") ? "1"
+ : factor.equals("THREE") ? "3"
: "Test adjustment needed!";
ReplicationConfig replicationConfig = ReplicationConfig.parse(
@@ -137,7 +233,27 @@
}
@Test
+ public void testParseECReplicationConfigFromString() {
+ assumeECType();
+
+ ConfigurationSource conf = new OzoneConfiguration();
+ ReplicationConfig repConfig = ReplicationConfig.parse(
+ org.apache.hadoop.hdds.client.ReplicationType.EC, ecDescriptor(), conf);
+
+ validate(repConfig, EcCodec.valueOf(codec), data, parity, chunkSize);
+ }
+
+ /**
+ * The adjustReplication is a method that is used by RootedOzoneFileSystem
+ * to adjust the bucket's default replication config if needed.
+ *
+ * As we define, if the bucket's default replication configuration is RATIS
+ * or STAND_ALONE, then replica count can be adjusted with the replication
+ * factor.
+ */
+ @Test
public void testAdjustReplication() {
+ assumeRatisOrStandaloneType();
ConfigurationSource conf = new OzoneConfiguration();
ReplicationConfig replicationConfig = ReplicationConfig.parse(
org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
@@ -155,6 +271,30 @@
}
/**
+ * The adjustReplication is a method that is used by RootedOzoneFileSystem
+ * to adjust the bucket's default replication config if needed.
+ *
+ * As we define, if the bucket's default replication configuration is EC,
+ * then the client can not adjust the configuration via the replication
+ * factor.
+ */
+ @Test
+ public void testAdjustECReplication() {
+ assumeECType();
+ ConfigurationSource conf = new OzoneConfiguration();
+ ReplicationConfig replicationConfig = ReplicationConfig.parse(
+ org.apache.hadoop.hdds.client.ReplicationType.EC, ecDescriptor(), conf);
+
+ validate(
+ ReplicationConfig.adjustReplication(replicationConfig, (short) 3, conf),
+ EcCodec.valueOf(codec), data, parity, chunkSize);
+
+ validate(
+ ReplicationConfig.adjustReplication(replicationConfig, (short) 1, conf),
+ EcCodec.valueOf(codec), data, parity, chunkSize);
+ }
+
+ /**
* This is a bit of a tricky test in the parametrized environment.
* The goal is to ensure that the following methods do validation while
* creating the ReplicationConfig: getDefault, adjustReplication, parse.
@@ -166,17 +306,21 @@
*/
@Test
public void testValidationBasedOnConfig() {
+ assumeRatisOrStandaloneType();
OzoneConfiguration conf = new OzoneConfiguration();
conf.set(OZONE_REPLICATION + ".allowed-configs",
"^STANDALONE/ONE|RATIS/THREE$");
conf.set(OZONE_REPLICATION, factor);
conf.set(OZONE_REPLICATION_TYPE, type);
+ // in case of allowed configurations
if ((type.equals("RATIS") && factor.equals("THREE"))
|| (type.equals("STAND_ALONE") && factor.equals("ONE"))) {
ReplicationConfig replicationConfig = ReplicationConfig.parse(
org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
factor, conf);
+ // check if adjust throws exception when adjusting to a config that is
+ // not allowed
if (type.equals("RATIS")) {
assertThrows(IllegalArgumentException.class,
() -> ReplicationConfig
@@ -186,27 +330,33 @@
() -> ReplicationConfig
.adjustReplication(replicationConfig, (short) 3, conf));
}
- ReplicationConfig.fromTypeAndFactor(
- org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
- org.apache.hadoop.hdds.client.ReplicationFactor.valueOf(factor));
- ReplicationConfig.fromProtoTypeAndFactor(
- ReplicationType.valueOf(type), ReplicationFactor.valueOf(factor));
ReplicationConfig.getDefault(conf);
} else {
+ // parse should fail in case of a configuration that is not allowed.
assertThrows(IllegalArgumentException.class,
() -> ReplicationConfig.parse(
org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
factor, conf));
+ // default can not be a configuration that is not allowed.
assertThrows(IllegalArgumentException.class,
() -> ReplicationConfig.getDefault(conf));
}
+
+ // From proto and java objects, we need to be able to create replication
+ // configs even though they are not allowed, as there might have been
+ // keys, that were created earlier when the config was allowed.
+ ReplicationConfig.fromTypeAndFactor(
+ org.apache.hadoop.hdds.client.ReplicationType.valueOf(type),
+ org.apache.hadoop.hdds.client.ReplicationFactor.valueOf(factor));
+ ReplicationConfig.fromProtoTypeAndFactor(
+ ReplicationType.valueOf(type), ReplicationFactor.valueOf(factor));
+
// CHAINED replication type is not supported by ReplicationConfig.
assertThrows(RuntimeException.class,
() -> ReplicationConfig.parse(
org.apache.hadoop.hdds.client.ReplicationType.CHAINED, "", conf));
}
-
private void validate(ReplicationConfig replicationConfig,
org.apache.hadoop.hdds.client.ReplicationType expectedType,
org.apache.hadoop.hdds.client.ReplicationFactor expectedFactor) {
@@ -232,4 +382,36 @@
((ReplicatedReplicationConfig) replicationConfig)
.getReplicationFactor().name());
}
+
+ private void validate(ReplicationConfig replicationConfig,
+ EcCodec expectedCodec,
+ int expectedData, int expectedParity, int expectedChunkSize) {
+
+ assertEquals(ECReplicationConfig.class, replicationConfig.getClass());
+ assertEquals(ReplicationType.EC, replicationConfig.getReplicationType());
+
+ ECReplicationConfig ecReplicationConfig =
+ (ECReplicationConfig) replicationConfig;
+
+ assertEquals(expectedCodec, ecReplicationConfig.getCodec());
+ assertEquals(expectedData, ecReplicationConfig.getData());
+ assertEquals(expectedParity, ecReplicationConfig.getParity());
+ assertEquals(expectedChunkSize, ecReplicationConfig.getEcChunkSize());
+
+ assertEquals(expectedData + expectedParity,
+ replicationConfig.getRequiredNodes());
+ }
+
+ private String ecDescriptor() {
+ return codec.toUpperCase() + "-" + data + "-" + parity + "-" +
+ (chunkSize == MB ? "1024K" : "1024");
+ }
+
+ private void assumeRatisOrStandaloneType() {
+ Assume.assumeTrue(type.equals("RATIS") || type.equals("STAND_ALONE"));
+ }
+
+ private void assumeECType() {
+ Assume.assumeTrue(type.equals("EC"));
+ }
}
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerInfo.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerInfo.java
new file mode 100644
index 0000000..9ec19bf
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerInfo.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.container;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Tests for the ContainerInfo class.
+ */
+
+public class TestContainerInfo {
+
+ @Test
+ public void getProtobufMessageEC() throws IOException {
+ ContainerInfo container =
+ createContainerInfo(RatisReplicationConfig.getInstance(THREE));
+ HddsProtos.ContainerInfoProto proto = container.getProtobuf();
+
+ // No EC Config
+ Assert.assertFalse(proto.hasEcReplicationConfig());
+ Assert.assertEquals(THREE, proto.getReplicationFactor());
+ Assert.assertEquals(RATIS, proto.getReplicationType());
+
+ // Reconstruct object from Proto
+ ContainerInfo recovered = ContainerInfo.fromProtobuf(proto);
+ Assert.assertEquals(RATIS, recovered.getReplicationType());
+ Assert.assertTrue(
+ recovered.getReplicationConfig() instanceof RatisReplicationConfig);
+
+ // EC Config
+ container = createContainerInfo(new ECReplicationConfig(3, 2));
+ proto = container.getProtobuf();
+
+ Assert.assertEquals(3, proto.getEcReplicationConfig().getData());
+ Assert.assertEquals(2, proto.getEcReplicationConfig().getParity());
+ Assert.assertFalse(proto.hasReplicationFactor());
+ Assert.assertEquals(EC, proto.getReplicationType());
+
+ // Reconstruct object from Proto
+ recovered = ContainerInfo.fromProtobuf(proto);
+ Assert.assertEquals(EC, recovered.getReplicationType());
+ Assert.assertTrue(
+ recovered.getReplicationConfig() instanceof ECReplicationConfig);
+ ECReplicationConfig config =
+ (ECReplicationConfig)recovered.getReplicationConfig();
+ Assert.assertEquals(3, config.getData());
+ Assert.assertEquals(2, config.getParity());
+ }
+
+ private ContainerInfo createContainerInfo(ReplicationConfig repConfig) {
+ ContainerInfo.Builder builder = new ContainerInfo.Builder();
+ builder.setContainerID(1234)
+ .setReplicationConfig(repConfig)
+ .setPipelineID(PipelineID.randomId())
+ .setState(HddsProtos.LifeCycleState.OPEN)
+ .setOwner("scm");
+ return builder.build();
+ }
+}
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReplicaInfo.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReplicaInfo.java
index 195baca..4f63ea7 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReplicaInfo.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReplicaInfo.java
@@ -55,5 +55,36 @@
proto.getDatanodeDetails()), info.getDatanodeDetails());
Assert.assertEquals(proto.getSequenceID(), info.getSequenceId());
Assert.assertEquals(proto.getState(), info.getState());
+ // If replicaIndex is not in the proto, then -1 should be returned
+ Assert.assertEquals(-1, info.getReplicaIndex());
+ }
+
+ @Test
+ public void testObjectCreatedFromProtoWithReplicaIndedx() {
+ HddsProtos.SCMContainerReplicaProto proto =
+ HddsProtos.SCMContainerReplicaProto.newBuilder()
+ .setKeyCount(10)
+ .setBytesUsed(12345)
+ .setContainerID(567)
+ .setPlaceOfBirth(UUID.randomUUID().toString())
+ .setSequenceID(5)
+ .setDatanodeDetails(MockDatanodeDetails.randomDatanodeDetails()
+ .getProtoBufMessage())
+ .setState("OPEN")
+ .setReplicaIndex(4)
+ .build();
+
+ ContainerReplicaInfo info = ContainerReplicaInfo.fromProto(proto);
+
+ Assert.assertEquals(proto.getContainerID(), info.getContainerID());
+ Assert.assertEquals(proto.getBytesUsed(), info.getBytesUsed());
+ Assert.assertEquals(proto.getKeyCount(), info.getKeyCount());
+ Assert.assertEquals(proto.getPlaceOfBirth(),
+ info.getPlaceOfBirth().toString());
+ Assert.assertEquals(DatanodeDetails.getFromProtoBuf(
+ proto.getDatanodeDetails()), info.getDatanodeDetails());
+ Assert.assertEquals(proto.getSequenceID(), info.getSequenceId());
+ Assert.assertEquals(proto.getState(), info.getState());
+ Assert.assertEquals(4, info.getReplicaIndex());
}
}
\ No newline at end of file
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/common/helpers/TestExcludeList.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/common/helpers/TestExcludeList.java
new file mode 100644
index 0000000..d3fb49b
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/common/helpers/TestExcludeList.java
@@ -0,0 +1,67 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.container.common.helpers;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.ozone.test.TestClock;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.util.UUID;
+
+/**
+ * Tests the exclude nodes list behavior at client.
+ */
+public class TestExcludeList {
+ private TestClock clock = new TestClock(Instant.now(), ZoneOffset.UTC);
+
+ @Test
+ public void excludeNodesShouldBeCleanedBasedOnGivenTime() {
+ ExcludeList list = new ExcludeList(10, clock);
+ list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
+ .setIpAddress("127.0.0.1").setHostName("localhost").addPort(
+ DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+ .build());
+ Assert.assertTrue(list.getDatanodes().size() == 1);
+ clock.fastForward(11);
+ Assert.assertTrue(list.getDatanodes().size() == 0);
+ list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
+ .setIpAddress("127.0.0.2").setHostName("localhost").addPort(
+ DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+ .build());
+ list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
+ .setIpAddress("127.0.0.3").setHostName("localhost").addPort(
+ DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+ .build());
+ Assert.assertTrue(list.getDatanodes().size() == 2);
+ }
+
+ @Test
+ public void excludeNodeShouldNotBeCleanedIfExpiryTimeIsZero() {
+ ExcludeList list = new ExcludeList(0, clock);
+ list.addDatanode(DatanodeDetails.newBuilder().setUuid(UUID.randomUUID())
+ .setIpAddress("127.0.0.1").setHostName("localhost").addPort(
+ DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, 2001))
+ .build());
+ Assert.assertTrue(list.getDatanodes().size() == 1);
+ clock.fastForward(1);
+ Assert.assertTrue(list.getDatanodes().size() == 1);
+ }
+}
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java
index 2f459fb..8193747 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java
@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
/**
Test cases for SCM container client classes.
*/
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipeline.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipeline.java
index 62917de..dc28102 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipeline.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipeline.java
@@ -19,9 +19,12 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.Objects;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -87,6 +90,33 @@
.build();
}
+ public static Pipeline createEcPipeline() {
+ return createEcPipeline(new ECReplicationConfig(3, 2));
+ }
+
+ public static Pipeline createEcPipeline(ECReplicationConfig repConfig) {
+
+ List<DatanodeDetails> nodes = new ArrayList<>();
+ for (int i = 0; i < repConfig.getRequiredNodes(); i++) {
+ nodes.add(MockDatanodeDetails.randomDatanodeDetails());
+ }
+ Map<DatanodeDetails, Integer> nodeIndexes = new HashMap<>();
+
+ int index = nodes.size() - 1;
+ for (DatanodeDetails dn : nodes) {
+ nodeIndexes.put(dn, index);
+ index--;
+ }
+
+ return Pipeline.newBuilder()
+ .setState(Pipeline.PipelineState.OPEN)
+ .setId(PipelineID.randomId())
+ .setReplicationConfig(repConfig)
+ .setNodes(nodes)
+ .setReplicaIndexes(nodeIndexes)
+ .build();
+ }
+
private MockPipeline() {
throw new UnsupportedOperationException("no instances");
}
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipeline.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipeline.java
index 20e8283..b0f9875 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipeline.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipeline.java
@@ -17,7 +17,9 @@
*/
package org.apache.hadoop.hdds.scm.pipeline;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
@@ -49,4 +51,65 @@
assertPorts(dn, ALL_PORTS);
}
}
+
+ @Test
+ public void getProtobufMessageEC() throws IOException {
+ Pipeline subject = MockPipeline.createPipeline(3);
+
+ //when EC config is empty/null
+ HddsProtos.Pipeline protobufMessage = subject.getProtobufMessage(1);
+ Assert.assertEquals(0, protobufMessage.getEcReplicationConfig().getData());
+
+
+ //when EC config is NOT empty
+ subject = MockPipeline.createEcPipeline();
+
+ protobufMessage = subject.getProtobufMessage(1);
+ Assert.assertEquals(3, protobufMessage.getEcReplicationConfig().getData());
+ Assert
+ .assertEquals(2, protobufMessage.getEcReplicationConfig().getParity());
+
+ }
+
+ @Test
+ public void testReplicaIndexesSerialisedCorrectly() throws IOException {
+ Pipeline pipeline = MockPipeline.createEcPipeline();
+ HddsProtos.Pipeline protobufMessage = pipeline.getProtobufMessage(1);
+ Pipeline reloadedPipeline = Pipeline.getFromProtobuf(protobufMessage);
+
+ for (DatanodeDetails dn : pipeline.getNodes()) {
+ Assert.assertEquals(pipeline.getReplicaIndex(dn),
+ reloadedPipeline.getReplicaIndex(dn));
+ }
+ }
+
+ @Test
+ public void testECPipelineIsAlwaysHealthy() throws IOException {
+ Pipeline pipeline = MockPipeline.createEcPipeline();
+ Assert.assertTrue(pipeline.isHealthy());
+ }
+
+ @Test
+ public void testBuilderCopiesAllFieldsFromOtherPipeline() {
+ Pipeline original = MockPipeline.createEcPipeline();
+ Pipeline copied = Pipeline.newBuilder(original).build();
+ Assert.assertEquals(original.getId(), copied.getId());
+ Assert.assertEquals(original.getReplicationConfig(),
+ copied.getReplicationConfig());
+ Assert.assertEquals(original.getPipelineState(), copied.getPipelineState());
+ Assert.assertEquals(original.getId(), copied.getId());
+ Assert.assertEquals(original.getId(), copied.getId());
+ Assert.assertEquals(original.getId(), copied.getId());
+ Assert.assertEquals(original.getNodeSet(), copied.getNodeSet());
+ Assert.assertEquals(original.getNodesInOrder(), copied.getNodesInOrder());
+ Assert.assertEquals(original.getLeaderId(), copied.getLeaderId());
+ Assert.assertEquals(original.getCreationTimestamp(),
+ copied.getCreationTimestamp());
+ Assert.assertEquals(original.getSuggestedLeaderId(),
+ copied.getSuggestedLeaderId());
+ for (DatanodeDetails dn : original.getNodes()) {
+ Assert.assertEquals(original.getReplicaIndex(dn),
+ copied.getReplicaIndex(dn));
+ }
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
index 2b63183..a8b83a1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
@@ -46,6 +46,7 @@
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.yaml.snakeyaml.Yaml;
@@ -193,7 +194,9 @@
String storedChecksum = containerData.getChecksum();
Yaml yaml = ContainerDataYaml.getYamlForContainerType(
- containerData.getContainerType());
+ containerData.getContainerType(),
+ containerData instanceof KeyValueContainerData &&
+ ((KeyValueContainerData)containerData).getReplicaIndex() > 0);
containerData.computeAndSetChecksum(yaml);
String computedChecksum = containerData.getChecksum();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
index b4e15db..d3caa31 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
@@ -27,6 +27,7 @@
import java.io.OutputStreamWriter;
import java.io.Writer;
import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -42,8 +43,11 @@
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import com.google.common.base.Preconditions;
+
+import static org.apache.hadoop.ozone.OzoneConsts.REPLICA_INDEX;
import static org.apache.hadoop.ozone.container.keyvalue
.KeyValueContainerData.KEYVALUE_YAML_TAG;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.yaml.snakeyaml.Yaml;
@@ -84,8 +88,12 @@
Writer writer = null;
FileOutputStream out = null;
try {
+ boolean withReplicaIndex =
+ containerData instanceof KeyValueContainerData &&
+ ((KeyValueContainerData) containerData).getReplicaIndex() > 0;
+
// Create Yaml for given container type
- Yaml yaml = getYamlForContainerType(containerType);
+ Yaml yaml = getYamlForContainerType(containerType, withReplicaIndex);
// Compute Checksum and update ContainerData
containerData.computeAndSetChecksum(yaml);
@@ -148,7 +156,8 @@
propertyUtils.setBeanAccess(BeanAccess.FIELD);
propertyUtils.setAllowReadOnlyProperties(true);
- Representer representer = new ContainerDataRepresenter();
+ Representer representer = new ContainerDataRepresenter(
+ KeyValueContainerData.getYamlFields());
representer.setPropertyUtils(propertyUtils);
Constructor containerDataConstructor = new ContainerDataConstructor();
@@ -166,39 +175,51 @@
* Given a ContainerType this method returns a Yaml representation of
* the container properties.
*
- * @param containerType type of container
+ * @param containerType type of container
+ * @param withReplicaIndex in the container yaml
* @return Yamal representation of container properties
- *
* @throws StorageContainerException if the type is unrecognized
*/
- public static Yaml getYamlForContainerType(ContainerType containerType)
+ public static Yaml getYamlForContainerType(ContainerType containerType,
+ boolean withReplicaIndex)
throws StorageContainerException {
PropertyUtils propertyUtils = new PropertyUtils();
propertyUtils.setBeanAccess(BeanAccess.FIELD);
propertyUtils.setAllowReadOnlyProperties(true);
- switch (containerType) {
- case KeyValueContainer:
- Representer representer = new ContainerDataRepresenter();
+ if (containerType == ContainerType.KeyValueContainer) {
+ List<String> yamlFields =
+ KeyValueContainerData.getYamlFields();
+ if (withReplicaIndex) {
+ yamlFields = new ArrayList<>(yamlFields);
+ yamlFields.add(REPLICA_INDEX);
+ }
+ Representer representer = new ContainerDataRepresenter(yamlFields);
representer.setPropertyUtils(propertyUtils);
representer.addClassTag(
KeyValueContainerData.class,
- KeyValueContainerData.KEYVALUE_YAML_TAG);
+ KEYVALUE_YAML_TAG);
Constructor keyValueDataConstructor = new ContainerDataConstructor();
return new Yaml(keyValueDataConstructor, representer);
- default:
- throw new StorageContainerException("Unrecognized container Type " +
- "format " + containerType, ContainerProtos.Result
- .UNKNOWN_CONTAINER_TYPE);
}
+ throw new StorageContainerException("Unrecognized container Type " +
+ "format " + containerType, ContainerProtos.Result
+ .UNKNOWN_CONTAINER_TYPE);
}
/**
* Representer class to define which fields need to be stored in yaml file.
*/
private static class ContainerDataRepresenter extends Representer {
+
+ private List<String> yamlFields;
+
+ ContainerDataRepresenter(List<String> yamlFields) {
+ this.yamlFields = yamlFields;
+ }
+
@Override
protected Set<Property> getProperties(Class<? extends Object> type) {
Set<Property> set = super.getProperties(type);
@@ -207,7 +228,7 @@
// When a new Container type is added, we need to add what fields need
// to be filtered here
if (type.equals(KeyValueContainerData.class)) {
- List<String> yamlFields = KeyValueContainerData.getYamlFields();
+
// filter properties
for (Property prop : set) {
String name = prop.getName();
@@ -282,7 +303,11 @@
.setState(ContainerProtos.ContainerDataProto.State.valueOf(state));
String schemaVersion = (String) nodes.get(OzoneConsts.SCHEMA_VERSION);
kvData.setSchemaVersion(schemaVersion);
-
+ final Object replicaIndex = nodes.get(REPLICA_INDEX);
+ if (replicaIndex != null) {
+ kvData.setReplicaIndex(
+ ((Long) replicaIndex).intValue());
+ }
return kvData;
}
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index 1edd046..83ba104 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -409,6 +409,11 @@
ContainerProtos.ContainerType.KeyValueContainer;
createRequest.setContainerType(containerType);
+ if (containerRequest.hasWriteChunk()) {
+ createRequest.setReplicaIndex(
+ containerRequest.getWriteChunk().getBlockID().getReplicaIndex());
+ }
+
ContainerCommandRequestProto.Builder requestBuilder =
ContainerCommandRequestProto.newBuilder()
.setCmdType(Type.CreateContainer)
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index ba131ff..fdf44ed 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -754,6 +754,7 @@
.setKeyCount(containerData.getBlockCount())
.setUsed(containerData.getBytesUsed())
.setState(getHddsState())
+ .setReplicaIndex(containerData.getReplicaIndex())
.setDeleteTransactionId(containerData.getDeleteTransactionId())
.setBlockCommitSequenceId(containerData.getBlockCommitSequenceId())
.setOriginNodeId(containerData.getOriginNodeId());
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index ccc252a..0bc6c3a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -83,6 +83,8 @@
private long blockCommitSequenceId;
+ private int replicaIndex;
+
static {
// Initialize YAML fields
KV_YAML_FIELDS = Lists.newArrayList();
@@ -303,4 +305,12 @@
db.getStore().getBatchHandler().commitBatchOperation(batchOperation);
}
+
+ public int getReplicaIndex() {
+ return replicaIndex;
+ }
+
+ public void setReplicaIndex(int replicaIndex) {
+ this.replicaIndex = replicaIndex;
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index b2e56c4..cce6c3e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -273,6 +273,8 @@
KeyValueContainerData newContainerData = new KeyValueContainerData(
containerID, layoutVersion, maxContainerSize, request.getPipelineID(),
getDatanodeId());
+ newContainerData.setReplicaIndex(request.getCreateContainer()
+ .getReplicaIndex());
// TODO: Add support to add metadataList to ContainerData. Add metadata
// to container during creation.
KeyValueContainer newContainer = new KeyValueContainer(
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java
index a80adca..7cee63d 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestSchemaOneBackwardsCompatibility.java
@@ -571,7 +571,8 @@
// Changing the paths above affects the checksum, so it was also removed
// from the container file and calculated at run time.
Yaml yaml = ContainerDataYaml.getYamlForContainerType(
- kvData.getContainerType());
+ kvData.getContainerType(),
+ kvData.getReplicaIndex() > 0);
kvData.computeAndSetChecksum(yaml);
KeyValueContainerUtil.parseKVContainerData(kvData, conf);
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
index 0bfdb17..c0843ab 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.ozone.container.common.impl;
+import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.fs.FileSystemTestHelper;
import org.apache.hadoop.fs.FileUtil;
@@ -28,6 +29,7 @@
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.junit.Assert;
import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
import org.apache.ozone.test.GenericTestUtils;
import org.junit.Test;
@@ -36,6 +38,7 @@
import java.io.File;
import java.io.IOException;
+import java.nio.charset.Charset;
import java.time.Instant;
import java.util.UUID;
@@ -77,7 +80,8 @@
* Creates a .container file. cleanup() should be called at the end of the
* test when container file is created.
*/
- private File createContainerFile(long containerID) throws IOException {
+ private File createContainerFile(long containerID, int replicaIndex)
+ throws IOException {
new File(testRoot).mkdirs();
String containerPath = containerID + ".container";
@@ -92,6 +96,7 @@
keyValueContainerData.updateDataScanTime(SCAN_TIME);
keyValueContainerData.setSchemaVersion(
VersionedDatanodeFeatures.SchemaV2.chooseSchemaVersion());
+ keyValueContainerData.setReplicaIndex(replicaIndex);
File containerFile = new File(testRoot, containerPath);
@@ -113,7 +118,7 @@
public void testCreateContainerFile() throws IOException {
long containerID = testContainerID++;
- File containerFile = createContainerFile(containerID);
+ File containerFile = createContainerFile(containerID, 7);
// Read from .container file, and verify data.
KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
@@ -136,7 +141,8 @@
assertEquals(SCAN_TIME.toEpochMilli(),
kvData.getDataScanTimestamp().longValue());
assertEquals(VersionedDatanodeFeatures.SchemaV2.chooseSchemaVersion(),
- kvData.getSchemaVersion());
+ kvData.getSchemaVersion());
+ assertEquals(7, kvData.getReplicaIndex());
// Update ContainerData.
kvData.addMetadata(OzoneConsts.VOLUME, VOLUME_OWNER);
@@ -174,6 +180,21 @@
}
@Test
+ public void testCreateContainerFileWithoutReplicaIndex() throws IOException {
+ long containerID = testContainerID++;
+
+ File containerFile = createContainerFile(containerID, 0);
+
+ final String content =
+ FileUtils.readFileToString(containerFile, Charset.defaultCharset());
+
+ Assert.assertFalse("ReplicaIndex shouldn't be persisted if zero",
+ content.contains("replicaIndex"));
+ cleanup();
+ }
+
+
+ @Test
public void testIncorrectContainerFile() throws IOException {
try {
String containerFile = "incorrect.container";
@@ -233,7 +254,24 @@
public void testChecksumInContainerFile() throws IOException {
long containerID = testContainerID++;
- File containerFile = createContainerFile(containerID);
+ File containerFile = createContainerFile(containerID, 0);
+
+ // Read from .container file, and verify data.
+ KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
+ .readContainerFile(containerFile);
+ ContainerUtils.verifyChecksum(kvData, conf);
+
+ cleanup();
+ }
+
+ /**
+ * Test to verify {@link ContainerUtils#verifyChecksum(ContainerData)}.
+ */
+ @Test
+ public void testChecksumInContainerFileWithReplicaIndex() throws IOException {
+ long containerID = testContainerID++;
+
+ File containerFile = createContainerFile(containerID, 10);
// Read from .container file, and verify data.
KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java
index d882ca4..34fb79a 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/upgrade/TestDatanodeUpgradeToScmHA.java
@@ -192,7 +192,7 @@
// restarted with SCM HA config and gets a different SCM ID.
conf.setBoolean(ScmConfigKeys.OZONE_SCM_HA_ENABLE_KEY, true);
changeScmID();
- restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion());
+ restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), true);
// Make sure the existing container can be read.
readChunk(exportWriteChunk2, pipeline);
@@ -289,7 +289,7 @@
/// FINALIZED: Restart datanode to upgrade the failed volume ///
- restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion());
+ restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion(), false);
Assert.assertEquals(1,
dsm.getContainer().getVolumeSet().getVolumesList().size());
@@ -344,7 +344,7 @@
changeScmID();
// A new volume is added that must be formatted.
File preFinVolume2 = addVolume();
- restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion());
+ restartDatanode(HDDSLayoutFeature.INITIAL_VERSION.layoutVersion(), true);
Assert.assertEquals(2,
dsm.getContainer().getVolumeSet().getVolumesList().size());
@@ -378,7 +378,7 @@
File finVolume = addVolume();
// Yet another SCM ID is received this time, but it should not matter.
changeScmID();
- restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion());
+ restartDatanode(HDDSLayoutFeature.SCM_HA.layoutVersion(), false);
Assert.assertEquals(3,
dsm.getContainer().getVolumeSet().getVolumesList().size());
Assert.assertEquals(0,
@@ -521,7 +521,7 @@
callVersionEndpointTask();
}
- public void restartDatanode(int expectedMlv)
+ public void restartDatanode(int expectedMlv, boolean exactMatch)
throws Exception {
// Stop existing datanode.
DatanodeDetails dd = dsm.getDatanodeDetails();
@@ -532,7 +532,12 @@
conf, null, null,
null);
int mlv = dsm.getLayoutVersionManager().getMetadataLayoutVersion();
- Assert.assertEquals(expectedMlv, mlv);
+ if (exactMatch) {
+ Assert.assertEquals(expectedMlv, mlv);
+ } else {
+ Assert.assertTrue("Expected minimum mlv(" + expectedMlv
+ + ") is smaller than mlv(" + mlv + ").", expectedMlv <= mlv);
+ }
callVersionEndpointTask();
}
diff --git a/hadoop-hdds/docs/content/feature/EC-Chunk-Layout.png b/hadoop-hdds/docs/content/feature/EC-Chunk-Layout.png
new file mode 100644
index 0000000..d7b6785
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/EC-Chunk-Layout.png
Binary files differ
diff --git a/hadoop-hdds/docs/content/feature/EC-Reads-With-No-Failures.png b/hadoop-hdds/docs/content/feature/EC-Reads-With-No-Failures.png
new file mode 100644
index 0000000..7dd4cab
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/EC-Reads-With-No-Failures.png
Binary files differ
diff --git a/hadoop-hdds/docs/content/feature/EC-Reconstructional-Read.png b/hadoop-hdds/docs/content/feature/EC-Reconstructional-Read.png
new file mode 100644
index 0000000..427e73a
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/EC-Reconstructional-Read.png
Binary files differ
diff --git a/hadoop-hdds/docs/content/feature/EC-Write-Block-Allocation-in-Containers.png b/hadoop-hdds/docs/content/feature/EC-Write-Block-Allocation-in-Containers.png
new file mode 100644
index 0000000..3945662
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/EC-Write-Block-Allocation-in-Containers.png
Binary files differ
diff --git a/hadoop-hdds/docs/content/feature/ErasureCoding.md b/hadoop-hdds/docs/content/feature/ErasureCoding.md
new file mode 100644
index 0000000..cced272
--- /dev/null
+++ b/hadoop-hdds/docs/content/feature/ErasureCoding.md
@@ -0,0 +1,212 @@
+---
+title: "Ozone Erasure Coding"
+weight: 1
+menu:
+ main:
+ parent: Features
+summary: Erasure Coding Support for Ozone.
+---
+<!---
+ 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.
+-->
+
+## Background
+
+Distributed systems basic expectation is to provide the data durability.
+To provide the higher data durability, many popular storage systems use replication
+approach which is expensive. The Apache Ozone supports `RATIS/THREE` replication scheme.
+The Ozone default replication scheme `RATIS/THREE` has 200% overhead in storage
+space and other resources (e.g., network bandwidth).
+However, for warm and cold datasets with relatively low I/O activities, additional
+block replicas rarely accessed during normal operations, but still consume the same
+amount of resources as the first replica.
+
+Therefore, a natural improvement is to use Erasure Coding (EC) in place of replication,
+which provides the same level of fault-tolerance with much less storage space.
+In typical EC setups, the storage overhead is no more than 50%. The replication factor of an EC file is meaningless.
+Instead of replication factor, we introduced ReplicationConfig interface to specify the required type of replication,
+either `RATIS/THREE` or `EC`.
+
+Integrating EC with Ozone can improve storage efficiency while still providing similar
+data durability as traditional replication-based Ozone deployments.
+As an example, a 3x replicated file with 6 blocks will consume 6*3 = `18` blocks of disk space.
+But with EC (6 data, 3 parity) deployment, it will only consume `9` blocks of disk space.
+
+## Architecture
+
+The storage data layout is a key factor in the implementation of EC. After deep analysis
+and several technical consideration, the most fitting data layout is striping model.
+The data striping layout is not new. The striping model already adapted by several other
+file systems(Ex: Quantcast File System, Hadoop Distributed File System etc) successfully before.
+
+For example, with the EC (6 data, 3 parity) scheme, the data chunks will be distributed to first 6 data nodes in order
+and then client generates the 3 parity chunks and transfer to remaining 3 nodes in order.
+These 9 chunks together we call as "Stripe". Next 6 chunks will be distributed to the same first 6 data nodes again
+and the parity to remaining 3 nodes. These 9 data nodes stored blocks together called as "BlockGroup".
+
+If the application is continuing to write beyond the size of `6 * BLOCK_SIZE`, then client will request new block group from Ozone Manager.
+
+### Erasure Coding Write
+
+The core logic of erasure coding writes are placed at ozone client.
+When client creates the file, ozone manager allocates the block group(`d + p`)
+number of nodes from the pipeline provider and return the same to client.
+As data is coming in from the application, client will write first d number of chunks
+to d number of data nodes in block group. It will also cache the d number chunks
+to generate the parity chunks. Once parity chunks generated, it will transfer the
+same to the remaining p nodes in order. Once all blocks reached their configured sizes,
+client will request the new block group nodes.
+
+Below diagram depicts the block allocation in containers as logical groups.
+For interest of space, we assumed EC(3, 2) Replication Config for the diagram.
+
+![EC Block Allocation in Containers](EC-Write-Block-Allocation-in-Containers.png)
+
+
+Let's zoom out the blockID: 1 data layout from the above picture, that showed in the following picture.
+This picture shows how the chunks will be layed out in data node blocks.
+
+![EC Chunk Layout](EC-Chunk-Layout.png)
+
+Currently, the EC client re-used the data transfer end-points to transfer the data to data nodes.
+The XceiverClientGRPC client used for writing data and putBlock info.
+The datanode side changes are minimal as we reused the same existing transfer protocols.
+The EC data block written at the datanode is same as any other block in non-EC mode.
+In a single block group, container id numbers are same in all nodes. A file can have multiple block groups.
+Each block group will have `d+p` number of block and all ids are same.
+
+**d** - Number of data blocks in a block group
+
+**p** - Number of parity blocks in a block group
+
+### Erasure Coding Read
+
+For reads, OM will provide the node location details as part of key lookup.
+If the key is erasure coded, Ozone client reads it in EC fashion. Since the data layout
+is different(see the previous section about write path), reads should consider the layout and do the reads accordingly.
+
+The EC client will open the connections to DNs based on the expected locations. When all data locations are available,
+it will attempt to do plain reads chunk by chunk in round robin fashion from d data blocks.
+
+Below picture shows the order when there are no failures while reading.
+
+![EC Reads With no Failures](EC-Reads-With-No-Failures.png)
+
+Until it sees read failures, there is no need of doing EC reconstruction.
+
+#### Erasure Coding On-the-fly Reconstruction Reads
+
+When client detects there are failures while reading or when starting the reads,
+Ozone EC client is capable of reconstructing/recovering the lost data by doing the EC decoding.
+To do the EC decoding it needs to read parity replicas. This is a degraded read as it needs to do reconstruction.
+This reconstruction is completely transparent to the applications.
+
+Below picture depicts how it uses parity replicas in reconstruction.
+
+![EC Reconstructional Reads](EC-Reconstructional-Read.png)
+
+ ### Erasure Coding Replication Config
+
+ Apache Ozone built with the pure 'Object Storage' semantics. However, many big data
+ eco system projects still uses file system APIs. To provide both worlds best access to Ozone,
+ it's provided both faces of interfaces. In both cases, keys/files would be written into buckets under the hood.
+ So, EC Replication Configs can be set at bucket level.
+ The EC policy encapsulates how to encode/decode a file.
+ Each EC Replication Config defined by the following pieces of information:
+ 1. **data:** Data blocks number in an EC block group.
+ 2. **parity:** Parity blocks number in an EC block group.
+ 3. **ecChunkSize:** The size of a striping chunk. This determines the granularity of striped reads and writes.
+ 4. **codec:** This is to indicate the type of EC algorithms (e.g., `RS`(Reed-Solomon), `XOR`).
+
+To pass the EC Replication Config in command line or configuration files, we need to use the following format:
+*codec*-*num data blocks*-*num parity blocks*-*ec chunk size*
+
+Currently, there are three built-in EC Replication Configs supported: `RS-3-2-1024k`, `RS-6-3-1024k`, `XOR-2-1-1024k`.
+The most recommended option is `RS-6-3-1024k`. When a key/file created without specifying the Replication Config,
+it inherits the EC Replication Config of its bucket if it's available.
+
+Changing the bucket level EC Replication Config only affect new files created within the bucket.
+Once a file has been created, its EC Replication Config cannot be changed currently.
+
+Deployment
+----------
+### Cluster and Hardware Configuration
+
+EC places additional demands on the cluster in terms of CPU and network.
+Encoding and decoding work consumes additional CPU on both Ozone clients and DataNodes.
+EC requires a minimum of as many DataNodes in the cluster as the configured EC stripe width. For the EC Replication Config `RS` (6,3), we need
+a minimum of 9 DataNodes.
+
+Erasure Coded keys/files also spread across racks for rack fault-tolerance.
+This means that when reading and writing striped files, most operations are off-rack.
+Network bisection bandwidth is thus very important.
+
+For rack fault-tolerance, it is also important to have enough number of racks,
+so that on average, each rack holds number of blocks no more than the number of EC parity blocks.
+A formula to calculate this would be (data blocks + parity blocks) / parity blocks, rounding up.
+For `RS` (6,3) EC Replication Config, this means minimally 3 racks (calculated by (6 + 3) / 3 = 3),
+and ideally 9 or more to handle planned and unplanned outages.
+For clusters with fewer racks than the number of the parity cells, Ozone cannot maintain rack fault-tolerance,
+but will still attempt to spread a striped file across multiple nodes to preserve node-level fault-tolerance.
+Due to this reason, it is recommended to setup racks with similar number of DataNodes.
+
+### Configurations
+
+EC Replication Config can be enabled at bucket level as discussed above.
+Cluster wide default Replication Config can be set with EC Replication Config by using
+the configuration keys `ozone.server.default.replication.type` and `ozone.server.default.replication`.
+
+```XML
+<property>
+ <name>ozone.server.default.replication.type</name>
+ <value>EC</value>
+</property>
+
+<property>
+ <name>ozone.server.default.replication</name>
+ <value>RS-6-3-1024k</value>
+</property>
+```
+
+Please note, the above configurations will be used only when client does not pass
+any replication config or bucket does not have any default values.
+
+#### Setting EC Replication Config On Bucket
+
+We can set the bucket EC Replication Config via ozone sh command. The EC Replication Config options can be passed while creating the bucket.
+
+```shell
+ozone sh bucket create <bucket path> --type EC --replication rs-6-3-1024k
+```
+
+We can also reset the EC Replication Config with the following command.
+
+```shell
+ozone sh bucket set-replication-config <bucket path> --type EC --replication rs-3-2-1024k
+```
+
+Once we reset, only newly created keys take effect of this new setting. Prior created keys in the bucket stay with same older setting.
+
+#### Setting EC Replication Config While Creating Keys/Files
+
+We can pass the EC Replication Config while creating the keys irrespective of bucket Replication Config.
+
+```shell
+ozone sh key put <Ozone Key Object Path> <Local File> --type EC --replication rs-6-3-1024k
+```
+
+In the case bucket already has default EC Replication Config, there is no need of passing EC Replication Config while creating key.
+
diff --git a/hadoop-hdds/erasurecode/pom.xml b/hadoop-hdds/erasurecode/pom.xml
new file mode 100644
index 0000000..ca2e3cb
--- /dev/null
+++ b/hadoop-hdds/erasurecode/pom.xml
@@ -0,0 +1,53 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+https://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.ozone</groupId>
+ <artifactId>hdds</artifactId>
+ <version>1.3.0-SNAPSHOT</version>
+ </parent>
+ <artifactId>hdds-erasurecode</artifactId>
+ <version>1.3.0-SNAPSHOT</version>
+ <description>Apache Ozone Distributed Data Store Earsurecode utils
+ </description>
+ <name>Apache Ozone HDDS Erasurecode</name>
+ <packaging>jar</packaging>
+
+ <dependencies>
+
+ <dependency>
+ <groupId>org.apache.ozone</groupId>
+ <artifactId>hdds-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.ozone</groupId>
+ <artifactId>hdds-test-utils</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.github.spotbugs</groupId>
+ <artifactId>spotbugs-annotations</artifactId>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+
+ </build>
+</project>
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/HadoopNativeECAccessorUtil.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/HadoopNativeECAccessorUtil.java
new file mode 100644
index 0000000..5314db3
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/HadoopNativeECAccessorUtil.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.io.erasurecode.rawcoder;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+
+/**
+ * This class is used to access some of the protected API
+ * from hadoop native EC java code.
+ * This is needed until we adapt the native ec code from hadoop.
+ */
+@InterfaceAudience.Private
+public final class HadoopNativeECAccessorUtil {
+ public static void performEncodeImpl(NativeRSRawEncoder encoder,
+ ByteBuffer[] inputs, int[] inputOffsets, int dataLen,
+ ByteBuffer[] outputs, int[] outputOffsets) throws IOException {
+ encoder.performEncodeImpl(inputs, inputOffsets, dataLen, outputs,
+ outputOffsets);
+ }
+
+ public static void performDecodeImpl(NativeRSRawDecoder decoder,
+ ByteBuffer[] inputs, int[] inputOffsets, int dataLen, int[] erased,
+ ByteBuffer[] outputs, int[] outputOffsets) throws IOException {
+ decoder.performDecodeImpl(inputs, inputOffsets, dataLen, erased, outputs,
+ outputOffsets);
+ }
+
+ public static void performEncodeImpl(NativeXORRawEncoder encoder,
+ ByteBuffer[] inputs, int[] inputOffsets, int dataLen,
+ ByteBuffer[] outputs, int[] outputOffsets) throws IOException {
+ encoder.performEncodeImpl(inputs, inputOffsets, dataLen, outputs,
+ outputOffsets);
+ }
+
+ public static void performDecodeImpl(NativeXORRawDecoder decoder,
+ ByteBuffer[] inputs, int[] inputOffsets, int dataLen, int[] erased,
+ ByteBuffer[] outputs, int[] outputOffsets) throws IOException {
+ decoder.performDecodeImpl(inputs, inputOffsets, dataLen, erased, outputs,
+ outputOffsets);
+ }
+
+ private HadoopNativeECAccessorUtil() {
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/package-info.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/package-info.java
new file mode 100644
index 0000000..42c6705
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/package-info.java
@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+/**
+ *
+ * Erasure coding rawcoder utilities for Apache Ozone.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/BufferAllocator.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/BufferAllocator.java
new file mode 100644
index 0000000..ed2c94e
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/BufferAllocator.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.ozone.erasurecode;
+
+
+import java.nio.ByteBuffer;
+
+/**
+ * An abstract buffer allocator used for test.
+ */
+public abstract class BufferAllocator {
+ private boolean usingDirect = false;
+
+ public BufferAllocator(boolean usingDirect) {
+ this.usingDirect = usingDirect;
+ }
+
+ protected boolean isUsingDirect() {
+ return usingDirect;
+ }
+
+ /**
+ * Allocate and return a ByteBuffer of specified length.
+ * @param bufferLen
+ * @return
+ */
+ public abstract ByteBuffer allocate(int bufferLen);
+
+ /**
+ * A simple buffer allocator that just uses ByteBuffer's
+ * allocate/allocateDirect API.
+ */
+ public static class SimpleBufferAllocator extends BufferAllocator {
+
+ public SimpleBufferAllocator(boolean usingDirect) {
+ super(usingDirect);
+ }
+
+ @Override
+ public ByteBuffer allocate(int bufferLen) {
+ return isUsingDirect() ? ByteBuffer.allocateDirect(bufferLen) :
+ ByteBuffer.allocate(bufferLen);
+ }
+ }
+
+ /**
+ * A buffer allocator that allocates a buffer from an existing large buffer by
+ * slice calling, but if no available space just degrades as
+ * SimpleBufferAllocator. So please ensure enough space for it.
+ */
+ public static class SlicedBufferAllocator extends BufferAllocator {
+ private ByteBuffer overallBuffer;
+
+ public SlicedBufferAllocator(boolean usingDirect, int totalBufferLen) {
+ super(usingDirect);
+ overallBuffer = isUsingDirect() ?
+ ByteBuffer.allocateDirect(totalBufferLen) :
+ ByteBuffer.allocate(totalBufferLen);
+ }
+
+ @Override
+ public ByteBuffer allocate(int bufferLen) {
+ if (bufferLen > overallBuffer.capacity() - overallBuffer.position()) {
+ // If no available space for the requested length, then allocate new
+ return isUsingDirect() ? ByteBuffer.allocateDirect(bufferLen) :
+ ByteBuffer.allocate(bufferLen);
+ }
+
+ overallBuffer.limit(overallBuffer.position() + bufferLen);
+ ByteBuffer result = overallBuffer.slice();
+ overallBuffer.position(overallBuffer.position() + bufferLen);
+ return result;
+ }
+ }
+
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/CodecRegistry.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/CodecRegistry.java
new file mode 100644
index 0000000..1c447a7
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/CodecRegistry.java
@@ -0,0 +1,170 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.NativeXORRawErasureCoderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * This class registers all coder implementations.
+ *
+ * {@link CodecRegistry} maps codec names to coder factories. All coder
+ * factories are dynamically identified and loaded using ServiceLoader.
+ */
+@InterfaceAudience.Private
+public final class CodecRegistry {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(CodecRegistry.class);
+
+ private static CodecRegistry instance = new CodecRegistry();
+ private Map<String, List<RawErasureCoderFactory>> coderMap;
+ private Map<String, String[]> coderNameMap;
+
+ private CodecRegistry() {
+ coderMap = new HashMap<>();
+ coderNameMap = new HashMap<>();
+ final ServiceLoader<RawErasureCoderFactory> coderFactories =
+ ServiceLoader.load(RawErasureCoderFactory.class);
+ updateCoders(coderFactories);
+ }
+
+ public static CodecRegistry getInstance() {
+ return instance;
+ }
+
+ /**
+ * Update coderMap and coderNameMap with iterable type of coder factories.
+ * @param coderFactories
+ */
+ @VisibleForTesting
+ void updateCoders(Iterable<RawErasureCoderFactory> coderFactories) {
+ for (RawErasureCoderFactory coderFactory : coderFactories) {
+ String codecName = coderFactory.getCodecName();
+ List<RawErasureCoderFactory> coders = coderMap.get(codecName);
+ if (coders == null) {
+ coders = new ArrayList<>();
+ coders.add(coderFactory);
+ coderMap.put(codecName, coders);
+ LOG.debug("Codec registered: codec = {}, coder = {}",
+ coderFactory.getCodecName(), coderFactory.getCoderName());
+ } else {
+ Boolean hasConflit = false;
+ for (RawErasureCoderFactory coder : coders) {
+ if (coder.getCoderName().equals(coderFactory.getCoderName())) {
+ hasConflit = true;
+ LOG.error("Coder {} cannot be registered because its coder name " +
+ "{} has conflict with {}",
+ coderFactory.getClass().getName(),
+ coderFactory.getCoderName(), coder.getClass().getName());
+ break;
+ }
+ }
+ if (!hasConflit) {
+ if (coderFactory instanceof NativeRSRawErasureCoderFactory
+ || coderFactory instanceof NativeXORRawErasureCoderFactory) {
+ coders.add(0, coderFactory);
+ } else {
+ coders.add(coderFactory);
+ }
+ LOG.debug("Codec registered: codec = {}, coder = {}",
+ coderFactory.getCodecName(), coderFactory.getCoderName());
+ }
+ }
+ }
+
+ // update coderNameMap accordingly
+ coderNameMap.clear();
+ for (Map.Entry<String, List<RawErasureCoderFactory>> entry :
+ coderMap.entrySet()) {
+ String codecName = entry.getKey();
+ List<RawErasureCoderFactory> coders = entry.getValue();
+ coderNameMap.put(codecName, coders.stream().
+ map(RawErasureCoderFactory::getCoderName).
+ collect(Collectors.toList()).toArray(new String[0]));
+ }
+ }
+
+ /**
+ * Get all coder names of the given codec.
+ * @param codecName the name of codec
+ * @return an array of all coder names, null if not exist
+ */
+ public String[] getCoderNames(String codecName) {
+ String[] coderNames = coderNameMap.get(codecName);
+ return coderNames;
+ }
+
+ /**
+ * Get all coder factories of the given codec.
+ * @param codecName the name of codec
+ * @return a list of all coder factories, null if not exist
+ */
+ public List<RawErasureCoderFactory> getCoders(String codecName) {
+ List<RawErasureCoderFactory> coders = coderMap.get(codecName);
+ return coders;
+ }
+
+ /**
+ * Get all codec names.
+ * @return a set of all codec names
+ */
+ public Set<String> getCodecNames() {
+ return coderMap.keySet();
+ }
+
+ /**
+ * Get a specific coder factory defined by codec name and coder name.
+ * @param codecName name of the codec
+ * @param coderName name of the coder
+ * @return the specific coder, null if not exist
+ */
+ public RawErasureCoderFactory getCoderByName(
+ String codecName, String coderName) {
+ List<RawErasureCoderFactory> coders = getCoders(codecName);
+
+ // find the RawErasureCoderFactory with the name of coderName
+ for (RawErasureCoderFactory coder : coders) {
+ if (coder.getCoderName().equals(coderName)) {
+ return coder;
+ }
+ }
+ return null;
+ }
+
+ public RawErasureCoderFactory getCodecFactory(String codecName) {
+ for (RawErasureCoderFactory factory : getCoders(codecName)) {
+ return factory;
+ }
+ throw new IllegalArgumentException("There is no registered codec " +
+ "factory for codec " + codecName);
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/ECChunk.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/ECChunk.java
new file mode 100644
index 0000000..1bf28b3
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/ECChunk.java
@@ -0,0 +1,113 @@
+/**
+ * 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.ozone.erasurecode;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A wrapper for ByteBuffer or bytes array for an erasure code chunk.
+ */
+public class ECChunk {
+
+ private ByteBuffer chunkBuffer;
+
+ // TODO: should be in a more general flags
+ private boolean allZero = false;
+
+ /**
+ * Wrapping a ByteBuffer.
+ *
+ * @param buffer buffer to be wrapped by the chunk
+ */
+ public ECChunk(ByteBuffer buffer) {
+ this.chunkBuffer = buffer;
+ }
+
+ public ECChunk(ByteBuffer buffer, int offset, int len) {
+ ByteBuffer tmp = buffer.duplicate();
+ tmp.position(offset);
+ tmp.limit(offset + len);
+ this.chunkBuffer = tmp.slice();
+ }
+
+ /**
+ * Wrapping a bytes array.
+ *
+ * @param buffer buffer to be wrapped by the chunk
+ */
+ public ECChunk(byte[] buffer) {
+ this.chunkBuffer = ByteBuffer.wrap(buffer);
+ }
+
+ public ECChunk(byte[] buffer, int offset, int len) {
+ this.chunkBuffer = ByteBuffer.wrap(buffer, offset, len);
+ }
+
+ public boolean isAllZero() {
+ return allZero;
+ }
+
+ public void setAllZero(boolean allZero) {
+ this.allZero = allZero;
+ }
+
+ /**
+ * Convert to ByteBuffer.
+ *
+ * @return ByteBuffer
+ */
+ public ByteBuffer getBuffer() {
+ return chunkBuffer;
+ }
+
+ /**
+ * Convert an array of this chunks to an array of ByteBuffers.
+ *
+ * @param chunks chunks to convert into buffers
+ * @return an array of ByteBuffers
+ */
+ public static ByteBuffer[] toBuffers(ECChunk[] chunks) {
+ ByteBuffer[] buffers = new ByteBuffer[chunks.length];
+
+ ECChunk chunk;
+ for (int i = 0; i < chunks.length; i++) {
+ chunk = chunks[i];
+ if (chunk == null) {
+ buffers[i] = null;
+ } else {
+ buffers[i] = chunk.getBuffer();
+ }
+ }
+
+ return buffers;
+ }
+
+ /**
+ * Convert to a bytes array, just for test usage.
+ * @return bytes array
+ */
+ public byte[] toBytesArray() {
+ byte[] bytesArr = new byte[chunkBuffer.remaining()];
+ // Avoid affecting the original one
+ chunkBuffer.mark();
+ chunkBuffer.get(bytesArr);
+ chunkBuffer.reset();
+
+ return bytesArr;
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/package-info.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/package-info.java
new file mode 100644
index 0000000..fe69252
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/package-info.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.
+ */
+
+/**
+ *
+ * Erasure coding utilities for Apache Ozone.
+ *
+ * Initial implementation is imported from Apache Hadoop see that repo for full
+ * history.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.ozone.erasurecode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawDecoder.java
new file mode 100644
index 0000000..537c2bb
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawDecoder.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.util.PerformanceAdvisory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * Abstract native raw decoder for all native coders to extend with.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("checkstyle:VisibilityModifier")
+abstract class AbstractNativeRawDecoder extends RawErasureDecoder {
+ public static final Logger LOG =
+ LoggerFactory.getLogger(AbstractNativeRawDecoder.class);
+
+ // Protect ISA-L coder data structure in native layer from being accessed and
+ // updated concurrently by the init, release and decode functions.
+ protected final ReentrantReadWriteLock decoderLock =
+ new ReentrantReadWriteLock();
+
+ AbstractNativeRawDecoder(ECReplicationConfig replicationConfig) {
+ super(replicationConfig);
+ }
+
+ @Override
+ protected void doDecode(ByteBufferDecodingState decodingState)
+ throws IOException {
+ decoderLock.readLock().lock();
+ try {
+ int[] inputOffsets = new int[decodingState.inputs.length];
+ int[] outputOffsets = new int[decodingState.outputs.length];
+
+ ByteBuffer buffer;
+ for (int i = 0; i < decodingState.inputs.length; ++i) {
+ buffer = decodingState.inputs[i];
+ if (buffer != null) {
+ inputOffsets[i] = buffer.position();
+ }
+ }
+
+ for (int i = 0; i < decodingState.outputs.length; ++i) {
+ buffer = decodingState.outputs[i];
+ outputOffsets[i] = buffer.position();
+ }
+
+ performDecodeImpl(decodingState.inputs, inputOffsets,
+ decodingState.decodeLength, decodingState.erasedIndexes,
+ decodingState.outputs, outputOffsets);
+ } finally {
+ decoderLock.readLock().unlock();
+ }
+ }
+
+ protected abstract void performDecodeImpl(ByteBuffer[] inputs,
+ int[] inputOffsets, int dataLen,
+ int[] erased, ByteBuffer[] outputs,
+ int[] outputOffsets)
+ throws IOException;
+
+ @Override
+ protected void doDecode(ByteArrayDecodingState decodingState)
+ throws IOException {
+ PerformanceAdvisory.LOG.debug("convertToByteBufferState is invoked, " +
+ "not efficiently. Please use direct ByteBuffer inputs/outputs");
+
+ ByteBufferDecodingState bbdState = decodingState.convertToByteBufferState();
+ doDecode(bbdState);
+
+ for (int i = 0; i < decodingState.outputs.length; i++) {
+ bbdState.outputs[i].get(decodingState.outputs[i],
+ decodingState.outputOffsets[i], decodingState.decodeLength);
+ }
+ }
+
+ @Override
+ public boolean preferDirectBuffer() {
+ return true;
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawEncoder.java
new file mode 100644
index 0000000..df6f36f
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/AbstractNativeRawEncoder.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.util.PerformanceAdvisory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * Abstract native raw encoder for all native coders to extend with.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("checkstyle:VisibilityModifier")
+abstract class AbstractNativeRawEncoder extends RawErasureEncoder {
+ public static final Logger LOG =
+ LoggerFactory.getLogger(AbstractNativeRawEncoder.class);
+
+ // Protect ISA-L coder data structure in native layer from being accessed and
+ // updated concurrently by the init, release and encode functions.
+ protected final ReentrantReadWriteLock encoderLock =
+ new ReentrantReadWriteLock();
+
+ AbstractNativeRawEncoder(ECReplicationConfig replicationConfig) {
+ super(replicationConfig);
+ }
+
+ @Override
+ protected void doEncode(ByteBufferEncodingState encodingState)
+ throws IOException {
+ encoderLock.readLock().lock();
+ try {
+ int[] inputOffsets = new int[encodingState.inputs.length];
+ int[] outputOffsets = new int[encodingState.outputs.length];
+ int dataLen = encodingState.inputs[0].remaining();
+
+ ByteBuffer buffer;
+ for (int i = 0; i < encodingState.inputs.length; ++i) {
+ buffer = encodingState.inputs[i];
+ inputOffsets[i] = buffer.position();
+ }
+
+ for (int i = 0; i < encodingState.outputs.length; ++i) {
+ buffer = encodingState.outputs[i];
+ outputOffsets[i] = buffer.position();
+ }
+
+ performEncodeImpl(encodingState.inputs, inputOffsets, dataLen,
+ encodingState.outputs, outputOffsets);
+ } finally {
+ encoderLock.readLock().unlock();
+ }
+ }
+
+ protected abstract void performEncodeImpl(
+ ByteBuffer[] inputs, int[] inputOffsets,
+ int dataLen, ByteBuffer[] outputs, int[] outputOffsets)
+ throws IOException;
+
+ @Override
+ protected void doEncode(ByteArrayEncodingState encodingState)
+ throws IOException {
+ PerformanceAdvisory.LOG.debug("convertToByteBufferState is invoked, " +
+ "not efficiently. Please use direct ByteBuffer inputs/outputs");
+
+ ByteBufferEncodingState bbeState = encodingState.convertToByteBufferState();
+ doEncode(bbeState);
+
+ for (int i = 0; i < encodingState.outputs.length; i++) {
+ bbeState.outputs[i].get(encodingState.outputs[i],
+ encodingState.outputOffsets[i], encodingState.encodeLength);
+ }
+ }
+
+ @Override
+ public boolean preferDirectBuffer() {
+ return true;
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayDecodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayDecodingState.java
new file mode 100644
index 0000000..3053539
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayDecodingState.java
@@ -0,0 +1,132 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+
+import java.nio.ByteBuffer;
+
+/**
+ * A utility class that maintains decoding state during a decode call using
+ * byte array inputs.
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+class ByteArrayDecodingState extends DecodingState {
+ byte[][] inputs;
+ int[] inputOffsets;
+ int[] erasedIndexes;
+ byte[][] outputs;
+ int[] outputOffsets;
+
+ ByteArrayDecodingState(RawErasureDecoder decoder, byte[][] inputs,
+ int[] erasedIndexes, byte[][] outputs) {
+ this.decoder = decoder;
+ this.inputs = inputs;
+ this.outputs = outputs;
+ this.erasedIndexes = erasedIndexes;
+ byte[] validInput = CoderUtil.findFirstValidInput(inputs);
+ this.decodeLength = validInput.length;
+
+ checkParameters(inputs, erasedIndexes, outputs);
+ checkInputBuffers(inputs);
+ checkOutputBuffers(outputs);
+
+ this.inputOffsets = new int[inputs.length]; // ALL ZERO
+ this.outputOffsets = new int[outputs.length]; // ALL ZERO
+ }
+
+ ByteArrayDecodingState(RawErasureDecoder decoder,
+ int decodeLength,
+ int[] erasedIndexes,
+ byte[][] inputs,
+ int[] inputOffsets,
+ byte[][] outputs,
+ int[] outputOffsets) {
+ this.decoder = decoder;
+ this.decodeLength = decodeLength;
+ this.erasedIndexes = erasedIndexes;
+ this.inputs = inputs;
+ this.outputs = outputs;
+ this.inputOffsets = inputOffsets;
+ this.outputOffsets = outputOffsets;
+ }
+
+ /**
+ * Convert to a ByteBufferDecodingState when it's backed by on-heap arrays.
+ */
+ ByteBufferDecodingState convertToByteBufferState() {
+ ByteBuffer[] newInputs = new ByteBuffer[inputs.length];
+ ByteBuffer[] newOutputs = new ByteBuffer[outputs.length];
+
+ for (int i = 0; i < inputs.length; i++) {
+ newInputs[i] = CoderUtil.cloneAsDirectByteBuffer(inputs[i],
+ inputOffsets[i], decodeLength);
+ }
+
+ for (int i = 0; i < outputs.length; i++) {
+ newOutputs[i] = ByteBuffer.allocateDirect(decodeLength);
+ }
+
+ ByteBufferDecodingState bbdState = new ByteBufferDecodingState(decoder,
+ decodeLength, erasedIndexes, newInputs, newOutputs);
+ return bbdState;
+ }
+
+ /**
+ * Check and ensure the buffers are of the desired length.
+ * @param buffers the buffers to check
+ */
+ void checkInputBuffers(byte[][] buffers) {
+ int validInputs = 0;
+
+ for (byte[] buffer : buffers) {
+ if (buffer == null) {
+ continue;
+ }
+
+ if (buffer.length != decodeLength) {
+ throw new IllegalArgumentException(
+ "Invalid buffer, not of length " + decodeLength);
+ }
+
+ validInputs++;
+ }
+
+ if (validInputs < decoder.getNumDataUnits()) {
+ throw new IllegalArgumentException(
+ "No enough valid inputs are provided, not recoverable");
+ }
+ }
+
+ /**
+ * Check and ensure the buffers are of the desired length.
+ * @param buffers the buffers to check
+ */
+ void checkOutputBuffers(byte[][] buffers) {
+ for (byte[] buffer : buffers) {
+ if (buffer == null) {
+ throw new IllegalArgumentException(
+ "Invalid buffer found, not allowing null");
+ }
+
+ if (buffer.length != decodeLength) {
+ throw new IllegalArgumentException(
+ "Invalid buffer not of length " + decodeLength);
+ }
+ }
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayEncodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayEncodingState.java
new file mode 100644
index 0000000..0e0def8
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteArrayEncodingState.java
@@ -0,0 +1,105 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A utility class that maintains encoding state during an encode call using
+ * byte array inputs.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("checkstyle:VisibilityModifier")
+class ByteArrayEncodingState extends EncodingState {
+ byte[][] inputs;
+ byte[][] outputs;
+ int[] inputOffsets;
+ int[] outputOffsets;
+
+ ByteArrayEncodingState(RawErasureEncoder encoder,
+ byte[][] inputs, byte[][] outputs) {
+ this.encoder = encoder;
+ byte[] validInput = CoderUtil.findFirstValidInput(inputs);
+ this.encodeLength = validInput.length;
+ this.inputs = inputs;
+ this.outputs = outputs;
+
+ checkParameters(inputs, outputs);
+ checkBuffers(inputs);
+ checkBuffers(outputs);
+
+ this.inputOffsets = new int[inputs.length]; // ALL ZERO
+ this.outputOffsets = new int[outputs.length]; // ALL ZERO
+ }
+
+ ByteArrayEncodingState(RawErasureEncoder encoder,
+ int encodeLength,
+ byte[][] inputs,
+ int[] inputOffsets,
+ byte[][] outputs,
+ int[] outputOffsets) {
+ this.encoder = encoder;
+ this.encodeLength = encodeLength;
+ this.inputs = inputs;
+ this.outputs = outputs;
+ this.inputOffsets = inputOffsets;
+ this.outputOffsets = outputOffsets;
+ }
+
+ /**
+ * Convert to a ByteBufferEncodingState when it's backed by on-heap arrays.
+ */
+ ByteBufferEncodingState convertToByteBufferState() {
+ ByteBuffer[] newInputs = new ByteBuffer[inputs.length];
+ ByteBuffer[] newOutputs = new ByteBuffer[outputs.length];
+
+ for (int i = 0; i < inputs.length; i++) {
+ newInputs[i] = CoderUtil.cloneAsDirectByteBuffer(inputs[i],
+ inputOffsets[i], encodeLength);
+ }
+
+ for (int i = 0; i < outputs.length; i++) {
+ newOutputs[i] = ByteBuffer.allocateDirect(encodeLength);
+ }
+
+ ByteBufferEncodingState bbeState = new ByteBufferEncodingState(encoder,
+ encodeLength, newInputs, newOutputs);
+ return bbeState;
+ }
+
+ /**
+ * Check and ensure the buffers are of the desired length.
+ * @param buffers the buffers to check
+ */
+ void checkBuffers(byte[][] buffers) {
+ for (byte[] buffer : buffers) {
+ if (buffer == null) {
+ throw new HadoopIllegalArgumentException(
+ "Invalid buffer found, not allowing null");
+ }
+
+ if (buffer.length != encodeLength) {
+ throw new HadoopIllegalArgumentException(
+ "Invalid buffer not of length " + encodeLength);
+ }
+ }
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferDecodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferDecodingState.java
new file mode 100644
index 0000000..7b9ebf6
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferDecodingState.java
@@ -0,0 +1,143 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A utility class that maintains decoding state during a decode call using
+ * ByteBuffer inputs.
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+class ByteBufferDecodingState extends DecodingState {
+ ByteBuffer[] inputs;
+ ByteBuffer[] outputs;
+ int[] erasedIndexes;
+ boolean usingDirectBuffer;
+
+ ByteBufferDecodingState(RawErasureDecoder decoder, ByteBuffer[] inputs,
+ int[] erasedIndexes, ByteBuffer[] outputs) {
+ this.decoder = decoder;
+ this.inputs = inputs;
+ this.outputs = outputs;
+ this.erasedIndexes = erasedIndexes;
+ ByteBuffer validInput = CoderUtil.findFirstValidInput(inputs);
+ this.decodeLength = validInput.remaining();
+ this.usingDirectBuffer = validInput.isDirect();
+
+ checkParameters(inputs, erasedIndexes, outputs);
+ checkInputBuffers(inputs);
+ checkOutputBuffers(outputs);
+ }
+
+ ByteBufferDecodingState(RawErasureDecoder decoder,
+ int decodeLength,
+ int[] erasedIndexes,
+ ByteBuffer[] inputs,
+ ByteBuffer[] outputs) {
+ this.decoder = decoder;
+ this.decodeLength = decodeLength;
+ this.erasedIndexes = erasedIndexes;
+ this.inputs = inputs;
+ this.outputs = outputs;
+ }
+
+ /**
+ * Convert to a ByteArrayDecodingState when it's backed by on-heap arrays.
+ */
+ ByteArrayDecodingState convertToByteArrayState() {
+ int[] inputOffsets = new int[inputs.length];
+ int[] outputOffsets = new int[outputs.length];
+ byte[][] newInputs = new byte[inputs.length][];
+ byte[][] newOutputs = new byte[outputs.length][];
+
+ ByteBuffer buffer;
+ for (int i = 0; i < inputs.length; ++i) {
+ buffer = inputs[i];
+ if (buffer != null) {
+ inputOffsets[i] = buffer.arrayOffset() + buffer.position();
+ newInputs[i] = buffer.array();
+ }
+ }
+
+ for (int i = 0; i < outputs.length; ++i) {
+ buffer = outputs[i];
+ outputOffsets[i] = buffer.arrayOffset() + buffer.position();
+ newOutputs[i] = buffer.array();
+ }
+
+ ByteArrayDecodingState baeState = new ByteArrayDecodingState(decoder,
+ decodeLength, erasedIndexes, newInputs,
+ inputOffsets, newOutputs, outputOffsets);
+ return baeState;
+ }
+
+ /**
+ * Check and ensure the buffers are of the desired length and type, direct
+ * buffers or not.
+ * @param buffers the buffers to check
+ */
+ void checkInputBuffers(ByteBuffer[] buffers) {
+ int validInputs = 0;
+
+ for (ByteBuffer buffer : buffers) {
+ if (buffer == null) {
+ continue;
+ }
+
+ if (buffer.remaining() != decodeLength) {
+ throw new IllegalArgumentException(
+ "Invalid buffer, not of length " + decodeLength);
+ }
+ if (buffer.isDirect() != usingDirectBuffer) {
+ throw new IllegalArgumentException(
+ "Invalid buffer, isDirect should be " + usingDirectBuffer);
+ }
+
+ validInputs++;
+ }
+
+ if (validInputs < decoder.getNumDataUnits()) {
+ throw new IllegalArgumentException(
+ "No enough valid inputs are provided, not recoverable");
+ }
+ }
+
+ /**
+ * Check and ensure the buffers are of the desired length and type, direct
+ * buffers or not.
+ * @param buffers the buffers to check
+ */
+ void checkOutputBuffers(ByteBuffer[] buffers) {
+ for (ByteBuffer buffer : buffers) {
+ if (buffer == null) {
+ throw new IllegalArgumentException(
+ "Invalid buffer found, not allowing null");
+ }
+
+ if (buffer.remaining() != decodeLength) {
+ throw new IllegalArgumentException(
+ "Invalid buffer, not of length " + decodeLength);
+ }
+ if (buffer.isDirect() != usingDirectBuffer) {
+ throw new IllegalArgumentException(
+ "Invalid buffer, isDirect should be " + usingDirectBuffer);
+ }
+ }
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferEncodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferEncodingState.java
new file mode 100644
index 0000000..4ca7a3f
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ByteBufferEncodingState.java
@@ -0,0 +1,110 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A utility class that maintains encoding state during an encode call using
+ * ByteBuffer inputs.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("checkstyle:VisibilityModifier")
+class ByteBufferEncodingState extends EncodingState {
+ ByteBuffer[] inputs;
+ ByteBuffer[] outputs;
+ boolean usingDirectBuffer;
+
+ ByteBufferEncodingState(RawErasureEncoder encoder,
+ ByteBuffer[] inputs, ByteBuffer[] outputs) {
+ this.encoder = encoder;
+ ByteBuffer validInput = CoderUtil.findFirstValidInput(inputs);
+ this.encodeLength = validInput.remaining();
+ this.usingDirectBuffer = validInput.isDirect();
+ this.inputs = inputs;
+ this.outputs = outputs;
+
+ checkParameters(inputs, outputs);
+ checkBuffers(inputs);
+ checkBuffers(outputs);
+ }
+
+ ByteBufferEncodingState(RawErasureEncoder encoder,
+ int encodeLength,
+ ByteBuffer[] inputs,
+ ByteBuffer[] outputs) {
+ this.encoder = encoder;
+ this.encodeLength = encodeLength;
+ this.inputs = inputs;
+ this.outputs = outputs;
+ }
+
+ /**
+ * Convert to a ByteArrayEncodingState when it's backed by on-heap arrays.
+ */
+ ByteArrayEncodingState convertToByteArrayState() {
+ int[] inputOffsets = new int[inputs.length];
+ int[] outputOffsets = new int[outputs.length];
+ byte[][] newInputs = new byte[inputs.length][];
+ byte[][] newOutputs = new byte[outputs.length][];
+
+ ByteBuffer buffer;
+ for (int i = 0; i < inputs.length; ++i) {
+ buffer = inputs[i];
+ inputOffsets[i] = buffer.arrayOffset() + buffer.position();
+ newInputs[i] = buffer.array();
+ }
+
+ for (int i = 0; i < outputs.length; ++i) {
+ buffer = outputs[i];
+ outputOffsets[i] = buffer.arrayOffset() + buffer.position();
+ newOutputs[i] = buffer.array();
+ }
+
+ ByteArrayEncodingState baeState = new ByteArrayEncodingState(encoder,
+ encodeLength, newInputs, inputOffsets, newOutputs, outputOffsets);
+ return baeState;
+ }
+
+ /**
+ * Check and ensure the buffers are of the desired length and type, direct
+ * buffers or not.
+ * @param buffers the buffers to check
+ */
+ void checkBuffers(ByteBuffer[] buffers) {
+ for (ByteBuffer buffer : buffers) {
+ if (buffer == null) {
+ throw new HadoopIllegalArgumentException(
+ "Invalid buffer found, not allowing null");
+ }
+
+ if (buffer.remaining() != encodeLength) {
+ throw new HadoopIllegalArgumentException(
+ "Invalid buffer remaining " + buffer.remaining()
+ + ", not of length " + encodeLength);
+ }
+ if (buffer.isDirect() != usingDirectBuffer) {
+ throw new HadoopIllegalArgumentException(
+ "Invalid buffer, isDirect should be " + usingDirectBuffer);
+ }
+ }
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/CoderUtil.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/CoderUtil.java
new file mode 100644
index 0000000..04bb980
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/CoderUtil.java
@@ -0,0 +1,174 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.ozone.erasurecode.ECChunk;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Helpful utilities for implementing some raw erasure coders.
+ */
+public final class CoderUtil {
+
+ private CoderUtil() {
+ // No called
+ }
+
+ private static byte[] emptyChunk = new byte[4096];
+
+ /**
+ * Make sure to return an empty chunk buffer for the desired length.
+ * @param leastLength
+ * @return empty chunk of zero bytes
+ */
+ static byte[] getEmptyChunk(int leastLength) {
+ if (emptyChunk.length >= leastLength) {
+ return emptyChunk; // In most time
+ }
+
+ synchronized (CoderUtil.class) {
+ emptyChunk = new byte[leastLength];
+ }
+
+ return emptyChunk;
+ }
+
+ /**
+ * Ensure a buffer filled with ZERO bytes from current readable/writable
+ * position.
+ * @param buffer a buffer ready to read / write certain size bytes
+ * @return the buffer itself, with ZERO bytes written, the position and limit
+ * are not changed after the call
+ */
+ static ByteBuffer resetBuffer(ByteBuffer buffer, int len) {
+ int pos = buffer.position();
+ buffer.put(getEmptyChunk(len), 0, len);
+ buffer.position(pos);
+
+ return buffer;
+ }
+
+ /**
+ * Ensure the buffer (either input or output) ready to read or write with ZERO
+ * bytes fully in specified length of len.
+ * @param buffer bytes array buffer
+ * @return the buffer itself
+ */
+ static byte[] resetBuffer(byte[] buffer, int offset, int len) {
+ byte[] empty = getEmptyChunk(len);
+ System.arraycopy(empty, 0, buffer, offset, len);
+
+ return buffer;
+ }
+
+ /**
+ * Initialize the output buffers with ZERO bytes.
+ */
+ static void resetOutputBuffers(ByteBuffer[] buffers, int dataLen) {
+ for (ByteBuffer buffer : buffers) {
+ resetBuffer(buffer, dataLen);
+ }
+ }
+
+ /**
+ * Initialize the output buffers with ZERO bytes.
+ */
+ static void resetOutputBuffers(byte[][] buffers, int[] offsets,
+ int dataLen) {
+ for (int i = 0; i < buffers.length; i++) {
+ resetBuffer(buffers[i], offsets[i], dataLen);
+ }
+ }
+
+
+ /**
+ * Convert an array of this chunks to an array of ByteBuffers.
+ *
+ * @param chunks chunks to convertToByteArrayState into buffers
+ * @return an array of ByteBuffers
+ */
+ static ByteBuffer[] toBuffers(ECChunk[] chunks) {
+ ByteBuffer[] buffers = new ByteBuffer[chunks.length];
+
+ ECChunk chunk;
+ for (int i = 0; i < chunks.length; i++) {
+ chunk = chunks[i];
+ if (chunk == null) {
+ buffers[i] = null;
+ } else {
+ buffers[i] = chunk.getBuffer();
+ if (chunk.isAllZero()) {
+ CoderUtil.resetBuffer(buffers[i], buffers[i].remaining());
+ }
+ }
+ }
+
+ return buffers;
+ }
+
+ /**
+ * Clone an input bytes array as direct ByteBuffer.
+ */
+ static ByteBuffer cloneAsDirectByteBuffer(byte[] input, int offset, int len) {
+ if (input == null) { // an input can be null, if erased or not to read
+ return null;
+ }
+
+ ByteBuffer directBuffer = ByteBuffer.allocateDirect(len);
+ directBuffer.put(input, offset, len);
+ directBuffer.flip();
+ return directBuffer;
+ }
+
+
+
+ /**
+ * Find the valid input from all the inputs.
+ * @param inputs input buffers to look for valid input
+ * @return the first valid input
+ */
+ static <T> T findFirstValidInput(T[] inputs) {
+ for (T input : inputs) {
+ if (input != null) {
+ return input;
+ }
+ }
+
+ throw new IllegalArgumentException(
+ "Invalid inputs are found, all being null");
+ }
+
+ /**
+ * Picking up indexes of valid inputs.
+ * @param inputs decoding input buffers
+ * @param <T>
+ */
+ static <T> int[] getValidIndexes(T[] inputs) {
+ int[] validIndexes = new int[inputs.length];
+ int idx = 0;
+ for (int i = 0; i < inputs.length; i++) {
+ if (inputs[i] != null) {
+ validIndexes[idx++] = i;
+ }
+ }
+
+ return Arrays.copyOf(validIndexes, idx);
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DecodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DecodingState.java
new file mode 100644
index 0000000..7a407b8
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DecodingState.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.ozone.erasurecode.rawcoder;
+
+/**
+ * A utility class that maintains decoding state during a decode call.
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+class DecodingState {
+ RawErasureDecoder decoder;
+ int decodeLength;
+
+ /**
+ * Check and validate decoding parameters, throw exception accordingly. The
+ * checking assumes it's a MDS code. Other code can override this.
+ * @param inputs input buffers to check
+ * @param erasedIndexes indexes of erased units in the inputs array
+ * @param outputs output buffers to check
+ */
+ <T> void checkParameters(T[] inputs, int[] erasedIndexes,
+ T[] outputs) {
+ if (inputs.length != decoder.getNumParityUnits() +
+ decoder.getNumDataUnits()) {
+ throw new IllegalArgumentException("Invalid inputs length");
+ }
+
+ if (erasedIndexes.length != outputs.length) {
+ throw new IllegalArgumentException(
+ "erasedIndexes and outputs mismatch in length");
+ }
+
+ if (erasedIndexes.length > decoder.getNumParityUnits()) {
+ throw new IllegalArgumentException(
+ "Too many erased, not recoverable");
+ }
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawDecoder.java
new file mode 100644
index 0000000..4a11dbc
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawDecoder.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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A dummy raw decoder that does no real computation.
+ * Instead, it just returns zero bytes.
+ * This decoder can be used to isolate the performance issue to HDFS side logic
+ * instead of codec, and is intended for test only.
+ */
+@InterfaceAudience.Private
+public class DummyRawDecoder extends RawErasureDecoder {
+
+ public DummyRawDecoder(ECReplicationConfig ecReplicationConfig) {
+ super(ecReplicationConfig);
+ }
+
+ @Override
+ protected void doDecode(ByteBufferDecodingState decodingState) {
+ // Nothing to do. Output buffers have already been reset
+ }
+
+ @Override
+ protected void doDecode(ByteArrayDecodingState decodingState) {
+ // Nothing to do. Output buffers have already been reset
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawEncoder.java
new file mode 100644
index 0000000..9b178e6
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawEncoder.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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A dummy raw encoder that does no real computation.
+ * Instead, it just returns zero bytes.
+ * This encoder can be used to isolate the performance issue to HDFS side logic
+ * instead of codec, and is intended for test only.
+ */
+@InterfaceAudience.Private
+public class DummyRawEncoder extends RawErasureEncoder {
+
+ public DummyRawEncoder(ECReplicationConfig ecReplicationConfig) {
+ super(ecReplicationConfig);
+ }
+
+ @Override
+ protected void doEncode(ByteArrayEncodingState encodingState) {
+ // Nothing to do. Output buffers have already been reset
+ }
+
+ @Override
+ protected void doEncode(ByteBufferEncodingState encodingState) {
+ // Nothing to do. Output buffers have already been reset
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawErasureCoderFactory.java
new file mode 100644
index 0000000..bf74871
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/DummyRawErasureCoderFactory.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A raw erasure coder factory for dummy raw coders.
+ */
+@InterfaceAudience.Private
+public class DummyRawErasureCoderFactory implements RawErasureCoderFactory {
+ public static final String CODER_NAME = "dummy_dummy";
+ public static final String DUMMY_CODEC_NAME = "dummy";
+
+ @Override
+ public RawErasureEncoder createEncoder(
+ ECReplicationConfig ecReplicationConfig) {
+ return new DummyRawEncoder(ecReplicationConfig);
+ }
+
+ @Override
+ public RawErasureDecoder createDecoder(
+ ECReplicationConfig ecReplicationConfig) {
+ return new DummyRawDecoder(ecReplicationConfig);
+ }
+
+ @Override
+ public String getCoderName() {
+ return CODER_NAME;
+ }
+
+ @Override
+ public String getCodecName() {
+ return DUMMY_CODEC_NAME;
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/EncodingState.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/EncodingState.java
new file mode 100644
index 0000000..2339342
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/EncodingState.java
@@ -0,0 +1,47 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A utility class that maintains encoding state during an encode call.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("checkstyle:VisibilityModifier")
+abstract class EncodingState {
+ RawErasureEncoder encoder;
+ int encodeLength;
+
+ /**
+ * Check and validate decoding parameters, throw exception accordingly.
+ * @param inputs input buffers to check
+ * @param outputs output buffers to check
+ */
+ <T> void checkParameters(T[] inputs, T[] outputs) {
+ if (inputs.length != encoder.getNumDataUnits()) {
+ throw new HadoopIllegalArgumentException("Invalid inputs length "
+ + inputs.length + " !=" + encoder.getNumDataUnits());
+ }
+ if (outputs.length != encoder.getNumParityUnits()) {
+ throw new HadoopIllegalArgumentException("Invalid outputs length "
+ + outputs.length + " !=" + encoder.getNumParityUnits());
+ }
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ErasureCodeNative.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ErasureCodeNative.java
new file mode 100644
index 0000000..4fcac54
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/ErasureCodeNative.java
@@ -0,0 +1,96 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Erasure code native libraries (for now, Intel ISA-L) related utilities.
+ *
+ * Note: This class should be modified to check native ozone libs loaded once
+ * ozone native libs started to include native erasure coded bits. For now, we
+ * will just depend on libhadoop lib to check whether native erasure coded libs
+ * available.
+ */
+public final class ErasureCodeNative {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ErasureCodeNative.class.getName());
+
+ /**
+ * The reason why ISA-L library is not available, or null if it is available.
+ */
+ private static final String LOADING_FAILURE_REASON;
+
+ static {
+ if (!NativeCodeLoader.isNativeCodeLoaded()) {
+ LOADING_FAILURE_REASON = "hadoop native library cannot be loaded.";
+ } else if (!NativeCodeLoader.buildSupportsIsal()) {
+ LOADING_FAILURE_REASON = "libhadoop was built without ISA-L support";
+ } else {
+ String problem = null;
+ try {
+ loadLibrary();
+ } catch (Throwable t) {
+ problem = "Loading ISA-L failed: " + t.getMessage();
+ LOG.warn(problem, t);
+ }
+ LOADING_FAILURE_REASON = problem;
+ }
+
+ if (LOADING_FAILURE_REASON != null) {
+ LOG.warn("ISA-L support is not available in your platform... " +
+ "using builtin-java codec where applicable");
+ }
+ }
+
+ private ErasureCodeNative() {
+ }
+
+ /**
+ * Are native libraries loaded?
+ */
+ public static boolean isNativeCodeLoaded() {
+ return LOADING_FAILURE_REASON == null;
+ }
+
+ /**
+ * Is the native ISA-L library loaded and initialized? Throw exception if not.
+ */
+ public static void checkNativeCodeLoaded() {
+ if (LOADING_FAILURE_REASON != null) {
+ throw new RuntimeException(LOADING_FAILURE_REASON);
+ }
+ }
+
+ /**
+ * Load native library available or supported.
+ */
+ public static native void loadLibrary();
+
+ /**
+ * Get the native library name that's available or supported.
+ */
+ public static native String getLibraryName();
+
+ public static String getLoadingFailureReason() {
+ return LOADING_FAILURE_REASON;
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawDecoder.java
new file mode 100644
index 0000000..98c5e8e
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawDecoder.java
@@ -0,0 +1,63 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.rawcoder.HadoopNativeECAccessorUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A Reed-Solomon raw decoder using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeRSRawDecoder extends AbstractNativeRawDecoder {
+
+ private org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawDecoder
+ hadoopNativeRSRawDecoder;
+
+ public NativeRSRawDecoder(ECReplicationConfig ecReplicationConfig) {
+ super(ecReplicationConfig);
+ hadoopNativeRSRawDecoder =
+ new org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawDecoder(
+ new ErasureCoderOptions(ecReplicationConfig.getData(),
+ ecReplicationConfig.getParity()));
+ }
+
+ @Override
+ protected void performDecodeImpl(ByteBuffer[] inputs, int[] inputOffsets,
+ int dataLen, int[] erased, ByteBuffer[] outputs, int[] outputOffsets)
+ throws IOException {
+ HadoopNativeECAccessorUtil
+ .performDecodeImpl(hadoopNativeRSRawDecoder, inputs,
+ inputOffsets, dataLen, erased, outputs, outputOffsets);
+ }
+
+ @Override
+ public void release() {
+ hadoopNativeRSRawDecoder.release();
+ }
+
+ @Override
+ public boolean preferDirectBuffer() {
+ return hadoopNativeRSRawDecoder.preferDirectBuffer();
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawEncoder.java
new file mode 100644
index 0000000..62db61e
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawEncoder.java
@@ -0,0 +1,63 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.rawcoder.HadoopNativeECAccessorUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A Reed-Solomon raw encoder using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeRSRawEncoder extends AbstractNativeRawEncoder {
+
+ private org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawEncoder
+ hadoopNativeRSRawEncoder;
+
+ public NativeRSRawEncoder(ECReplicationConfig ecReplicationConfig) {
+ super(ecReplicationConfig);
+ hadoopNativeRSRawEncoder =
+ new org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawEncoder(
+ new ErasureCoderOptions(ecReplicationConfig.getData(),
+ ecReplicationConfig.getParity()));
+ }
+
+ @Override
+ protected void performEncodeImpl(ByteBuffer[] inputs, int[] inputOffsets,
+ int dataLen, ByteBuffer[] outputs, int[] outputOffsets)
+ throws IOException {
+ HadoopNativeECAccessorUtil
+ .performEncodeImpl(hadoopNativeRSRawEncoder, inputs,
+ inputOffsets, dataLen, outputs, outputOffsets);
+ }
+
+ @Override
+ public void release() {
+ hadoopNativeRSRawEncoder.release();
+ }
+
+ @Override
+ public boolean preferDirectBuffer() {
+ return hadoopNativeRSRawEncoder.preferDirectBuffer();
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawErasureCoderFactory.java
new file mode 100644
index 0000000..b2088ec
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeRSRawErasureCoderFactory.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A raw coder factory for raw Reed-Solomon coder in native using Intel ISA-L.
+ */
+@InterfaceAudience.Private
+public class NativeRSRawErasureCoderFactory
+ implements RawErasureCoderFactory {
+
+ public static final String CODER_NAME = "rs_native";
+
+ @Override
+ public RawErasureEncoder createEncoder(
+ ECReplicationConfig ecReplicationConfig) {
+ return new NativeRSRawEncoder(ecReplicationConfig);
+ }
+
+ @Override
+ public RawErasureDecoder createDecoder(
+ ECReplicationConfig ecReplicationConfig) {
+ return new NativeRSRawDecoder(ecReplicationConfig);
+ }
+
+ @Override
+ public String getCoderName() {
+ return CODER_NAME;
+ }
+
+ @Override
+ public String getCodecName() {
+ return ECReplicationConfig.EcCodec.RS.name().toLowerCase();
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawDecoder.java
new file mode 100644
index 0000000..96f8581
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawDecoder.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.rawcoder.HadoopNativeECAccessorUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A XOR raw decoder using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeXORRawDecoder extends AbstractNativeRawDecoder {
+
+ private org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawDecoder
+ hadoopNativeXORRawDecoder;
+
+ public NativeXORRawDecoder(ECReplicationConfig ecReplicationConfig) {
+ super(ecReplicationConfig);
+ hadoopNativeXORRawDecoder =
+ new org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawDecoder(
+ new ErasureCoderOptions(ecReplicationConfig.getData(),
+ ecReplicationConfig.getParity()));
+ }
+
+ @Override
+ protected void performDecodeImpl(ByteBuffer[] inputs, int[] inputOffsets,
+ int dataLen, int[] erased, ByteBuffer[] outputs, int[] outputOffsets)
+ throws IOException {
+ HadoopNativeECAccessorUtil
+ .performDecodeImpl(hadoopNativeXORRawDecoder, inputs,
+ inputOffsets, dataLen, erased, outputs, outputOffsets);
+ }
+
+ @Override
+ public void release() {
+ hadoopNativeXORRawDecoder.release();
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawEncoder.java
new file mode 100644
index 0000000..91be762
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawEncoder.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+import org.apache.hadoop.io.erasurecode.rawcoder.HadoopNativeECAccessorUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A XOR raw encoder using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeXORRawEncoder extends AbstractNativeRawEncoder {
+
+ private org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawEncoder
+ hadoopNativeXORRawEncoder;
+
+ public NativeXORRawEncoder(ECReplicationConfig ecReplicationConfig) {
+ super(ecReplicationConfig);
+ hadoopNativeXORRawEncoder =
+ new org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawEncoder(
+ new ErasureCoderOptions(ecReplicationConfig.getData(),
+ ecReplicationConfig.getParity()));
+ }
+
+ @Override
+ protected void performEncodeImpl(ByteBuffer[] inputs, int[] inputOffsets,
+ int dataLen, ByteBuffer[] outputs, int[] outputOffsets)
+ throws IOException {
+ HadoopNativeECAccessorUtil
+ .performEncodeImpl(hadoopNativeXORRawEncoder, inputs,
+ inputOffsets, dataLen, outputs, outputOffsets);
+ }
+
+ @Override
+ public void release() {
+ hadoopNativeXORRawEncoder.release();
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawErasureCoderFactory.java
new file mode 100644
index 0000000..4a2eb50
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/NativeXORRawErasureCoderFactory.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A raw coder factory for xor coder in native using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeXORRawErasureCoderFactory
+ implements RawErasureCoderFactory {
+
+ public static final String CODER_NAME = "xor_native";
+
+ @Override
+ public RawErasureEncoder createEncoder(
+ ECReplicationConfig ecReplicationConfig) {
+ return new NativeXORRawEncoder(ecReplicationConfig);
+ }
+
+ @Override
+ public RawErasureDecoder createDecoder(
+ ECReplicationConfig ecReplicationConfig) {
+ return new NativeXORRawDecoder(ecReplicationConfig);
+ }
+
+ @Override
+ public String getCoderName() {
+ return CODER_NAME;
+ }
+
+ @Override
+ public String getCodecName() {
+ return ECReplicationConfig.EcCodec.XOR.name().toLowerCase();
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawDecoder.java
new file mode 100644
index 0000000..a61e1ea
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawDecoder.java
@@ -0,0 +1,177 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.rawcoder.util.DumpUtil;
+import org.apache.ozone.erasurecode.rawcoder.util.GF256;
+import org.apache.ozone.erasurecode.rawcoder.util.RSUtil;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * A raw erasure decoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible. This new Java coder is about 5X faster than the one originated
+ * from HDFS-RAID, and also compatible with the native/ISA-L coder.
+ */
+@InterfaceAudience.Private
+public class RSRawDecoder extends RawErasureDecoder {
+ //relevant to schema and won't change during decode calls
+ private byte[] encodeMatrix;
+
+ /**
+ * Below are relevant to schema and erased indexes, thus may change during
+ * decode calls.
+ */
+ private byte[] decodeMatrix;
+ private byte[] invertMatrix;
+ /**
+ * Array of input tables generated from coding coefficients previously.
+ * Must be of size 32*k*rows
+ */
+ private byte[] gfTables;
+ private int[] cachedErasedIndexes;
+ private int[] validIndexes;
+ private int numErasedDataUnits;
+ private boolean[] erasureFlags;
+
+ public RSRawDecoder(ECReplicationConfig ecReplicationConfig) {
+ super(ecReplicationConfig);
+
+ int numAllUnits = getNumAllUnits();
+ if (getNumAllUnits() >= RSUtil.GF.getFieldSize()) {
+ throw new HadoopIllegalArgumentException(
+ "Invalid getNumDataUnits() and numParityUnits");
+ }
+
+ encodeMatrix = new byte[numAllUnits * getNumDataUnits()];
+ RSUtil.genCauchyMatrix(encodeMatrix, numAllUnits, getNumDataUnits());
+ if (allowVerboseDump()) {
+ DumpUtil.dumpMatrix(encodeMatrix, getNumDataUnits(), numAllUnits);
+ }
+ }
+
+ @Override
+ protected void doDecode(ByteBufferDecodingState decodingState) {
+ CoderUtil.resetOutputBuffers(decodingState.outputs,
+ decodingState.decodeLength);
+ prepareDecoding(decodingState.inputs, decodingState.erasedIndexes);
+
+ ByteBuffer[] realInputs = new ByteBuffer[getNumDataUnits()];
+ for (int i = 0; i < getNumDataUnits(); i++) {
+ realInputs[i] = decodingState.inputs[validIndexes[i]];
+ }
+ RSUtil.encodeData(gfTables, realInputs, decodingState.outputs);
+ }
+
+ @Override
+ protected void doDecode(ByteArrayDecodingState decodingState) {
+ int dataLen = decodingState.decodeLength;
+ CoderUtil.resetOutputBuffers(decodingState.outputs,
+ decodingState.outputOffsets, dataLen);
+ prepareDecoding(decodingState.inputs, decodingState.erasedIndexes);
+
+ byte[][] realInputs = new byte[getNumDataUnits()][];
+ int[] realInputOffsets = new int[getNumDataUnits()];
+ for (int i = 0; i < getNumDataUnits(); i++) {
+ realInputs[i] = decodingState.inputs[validIndexes[i]];
+ realInputOffsets[i] = decodingState.inputOffsets[validIndexes[i]];
+ }
+ RSUtil.encodeData(gfTables, dataLen, realInputs, realInputOffsets,
+ decodingState.outputs, decodingState.outputOffsets);
+ }
+
+ private <T> void prepareDecoding(T[] inputs, int[] erasedIndexes) {
+ int[] tmpValidIndexes = CoderUtil.getValidIndexes(inputs);
+ if (Arrays.equals(this.cachedErasedIndexes, erasedIndexes) &&
+ Arrays.equals(this.validIndexes, tmpValidIndexes)) {
+ return; // Optimization. Nothing to do
+ }
+ this.cachedErasedIndexes =
+ Arrays.copyOf(erasedIndexes, erasedIndexes.length);
+ this.validIndexes =
+ Arrays.copyOf(tmpValidIndexes, tmpValidIndexes.length);
+
+ processErasures(erasedIndexes);
+ }
+
+ private void processErasures(int[] erasedIndexes) {
+ this.decodeMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+ this.invertMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+ this.gfTables = new byte[getNumAllUnits() * getNumDataUnits() * 32];
+
+ this.erasureFlags = new boolean[getNumAllUnits()];
+ this.numErasedDataUnits = 0;
+
+ for (int i = 0; i < erasedIndexes.length; i++) {
+ int index = erasedIndexes[i];
+ erasureFlags[index] = true;
+ if (index < getNumDataUnits()) {
+ numErasedDataUnits++;
+ }
+ }
+
+ generateDecodeMatrix(erasedIndexes);
+
+ RSUtil.initTables(getNumDataUnits(), erasedIndexes.length,
+ decodeMatrix, 0, gfTables);
+ if (allowVerboseDump()) {
+ System.out.println(DumpUtil.bytesToHex(gfTables, -1));
+ }
+ }
+
+ // Generate decode matrix from encode matrix
+ private void generateDecodeMatrix(int[] erasedIndexes) {
+ int i, j, r, p;
+ byte s;
+ byte[] tmpMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+
+ // Construct matrix tmpMatrix by removing error rows
+ for (i = 0; i < getNumDataUnits(); i++) {
+ r = validIndexes[i];
+ for (j = 0; j < getNumDataUnits(); j++) {
+ tmpMatrix[getNumDataUnits() * i + j] =
+ encodeMatrix[getNumDataUnits() * r + j];
+ }
+ }
+
+ GF256.gfInvertMatrix(tmpMatrix, invertMatrix, getNumDataUnits());
+
+ for (i = 0; i < numErasedDataUnits; i++) {
+ for (j = 0; j < getNumDataUnits(); j++) {
+ decodeMatrix[getNumDataUnits() * i + j] =
+ invertMatrix[getNumDataUnits() * erasedIndexes[i] + j];
+ }
+ }
+
+ for (p = numErasedDataUnits; p < erasedIndexes.length; p++) {
+ for (i = 0; i < getNumDataUnits(); i++) {
+ s = 0;
+ for (j = 0; j < getNumDataUnits(); j++) {
+ s ^= GF256.gfMul(invertMatrix[j * getNumDataUnits() + i],
+ encodeMatrix[getNumDataUnits() * erasedIndexes[p] + j]);
+ }
+ decodeMatrix[getNumDataUnits() * p + i] = s;
+ }
+ }
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawEncoder.java
new file mode 100644
index 0000000..086b1c5
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawEncoder.java
@@ -0,0 +1,77 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.rawcoder.util.DumpUtil;
+import org.apache.ozone.erasurecode.rawcoder.util.RSUtil;
+
+/**
+ * A raw erasure encoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible. This new Java coder is about 5X faster than the one originated
+ * from HDFS-RAID, and also compatible with the native/ISA-L coder.
+ */
+public class RSRawEncoder extends RawErasureEncoder {
+ // relevant to schema and won't change during encode calls.
+ private byte[] encodeMatrix;
+ /**
+ * Array of input tables generated from coding coefficients previously.
+ * Must be of size 32*k*rows
+ */
+ private byte[] gfTables;
+
+ public RSRawEncoder(ECReplicationConfig ecReplicationConfig) {
+ super(ecReplicationConfig);
+
+ if (getNumAllUnits() >= RSUtil.GF.getFieldSize()) {
+ throw new IllegalArgumentException(
+ "Invalid numDataUnits and numParityUnits");
+ }
+
+ encodeMatrix = new byte[getNumAllUnits() * getNumDataUnits()];
+ RSUtil.genCauchyMatrix(encodeMatrix, getNumAllUnits(), getNumDataUnits());
+ if (allowVerboseDump()) {
+ DumpUtil.dumpMatrix(encodeMatrix, getNumDataUnits(), getNumAllUnits());
+ }
+ gfTables = new byte[getNumAllUnits() * getNumDataUnits() * 32];
+ RSUtil.initTables(getNumDataUnits(), getNumParityUnits(), encodeMatrix,
+ getNumDataUnits() * getNumDataUnits(), gfTables);
+ if (allowVerboseDump()) {
+ System.out.println(DumpUtil.bytesToHex(gfTables, -1));
+ }
+ }
+
+ @Override
+ protected void doEncode(ByteBufferEncodingState encodingState) {
+ CoderUtil.resetOutputBuffers(encodingState.outputs,
+ encodingState.encodeLength);
+ RSUtil.encodeData(gfTables, encodingState.inputs, encodingState.outputs);
+ }
+
+ @Override
+ protected void doEncode(ByteArrayEncodingState encodingState) {
+ CoderUtil.resetOutputBuffers(encodingState.outputs,
+ encodingState.outputOffsets,
+ encodingState.encodeLength);
+ RSUtil.encodeData(gfTables, encodingState.encodeLength,
+ encodingState.inputs,
+ encodingState.inputOffsets, encodingState.outputs,
+ encodingState.outputOffsets);
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawErasureCoderFactory.java
new file mode 100644
index 0000000..ece5d9d
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RSRawErasureCoderFactory.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A raw coder factory for the new raw Reed-Solomon coder in Java.
+ */
+@InterfaceAudience.Private
+public class RSRawErasureCoderFactory implements RawErasureCoderFactory {
+
+ public static final String CODER_NAME = "rs_java";
+
+ @Override
+ public RawErasureEncoder createEncoder(
+ ECReplicationConfig ecReplicationConfig) {
+ return new RSRawEncoder(ecReplicationConfig);
+ }
+
+ @Override
+ public RawErasureDecoder createDecoder(
+ ECReplicationConfig ecReplicationConfig) {
+ return new RSRawDecoder(ecReplicationConfig);
+ }
+
+ @Override
+ public String getCoderName() {
+ return CODER_NAME;
+ }
+
+ @Override
+ public String getCodecName() {
+ return ECReplicationConfig.EcCodec.RS.name().toLowerCase();
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderFactory.java
new file mode 100644
index 0000000..27cc049
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderFactory.java
@@ -0,0 +1,56 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * Raw erasure coder factory that can be used to create raw encoder and decoder.
+ * It helps in configuration since only one factory class is needed to be
+ * configured.
+ */
+@InterfaceAudience.Private
+public interface RawErasureCoderFactory {
+
+ /**
+ * Create raw erasure encoder.
+ * @param ecReplicationConfig the config used to create the encoder
+ * @return raw erasure encoder
+ */
+ RawErasureEncoder createEncoder(ECReplicationConfig ecReplicationConfig);
+
+ /**
+ * Create raw erasure decoder.
+ * @param ecReplicationConfig the config used to create the encoder
+ * @return raw erasure decoder
+ */
+ RawErasureDecoder createDecoder(ECReplicationConfig ecReplicationConfig);
+
+ /**
+ * Get the name of the coder.
+ * @return coder name
+ */
+ String getCoderName();
+
+ /**
+ * Get the name of its codec.
+ * @return codec name
+ */
+ String getCodecName();
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureDecoder.java
new file mode 100644
index 0000000..144ba41
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureDecoder.java
@@ -0,0 +1,217 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.ECChunk;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * An abstract raw erasure decoder that's to be inherited by new decoders.
+ *
+ * Raw erasure coder is part of erasure codec framework, where erasure coder is
+ * used to encode/decode a group of blocks (BlockGroup) according to the codec
+ * specific BlockGroup layout and logic. An erasure coder extracts chunks of
+ * data from the blocks and can employ various low level raw erasure coders to
+ * perform encoding/decoding against the chunks.
+ *
+ * To distinguish from erasure coder, here raw erasure coder is used to mean the
+ * low level constructs, since it only takes care of the math calculation with
+ * a group of byte buffers.
+ *
+ * Note it mainly provides decode() calls, which should be stateless and may be
+ * made thread-safe in future.
+ */
+public abstract class RawErasureDecoder {
+
+ private final ECReplicationConfig coderOptions;
+
+ public RawErasureDecoder(ECReplicationConfig coderOptions) {
+ this.coderOptions = coderOptions;
+ }
+
+ /**
+ * Decode with inputs and erasedIndexes, generates outputs.
+ * How to prepare for inputs:
+ * 1. Create an array containing data units + parity units. Please note the
+ * data units should be first or before the parity units.
+ * 2. Set null in the array locations specified via erasedIndexes to indicate
+ * they're erased and no data are to read from;
+ * 3. Set null in the array locations for extra redundant items, as they're
+ * not necessary to read when decoding. For example in RS-6-3, if only 1
+ * unit is really erased, then we have 2 extra items as redundant. They can
+ * be set as null to indicate no data will be used from them.
+ *
+ * For an example using RS (6, 3), assuming sources (d0, d1, d2, d3, d4, d5)
+ * and parities (p0, p1, p2), d2 being erased. We can and may want to use only
+ * 6 units like (d1, d3, d4, d5, p0, p2) to recover d2. We will have:
+ * inputs = [null(d0), d1, null(d2), d3, d4, d5, p0, null(p1), p2]
+ * erasedIndexes = [2] // index of d2 into inputs array
+ * outputs = [a-writable-buffer]
+ *
+ * Note, for both inputs and outputs, no mixing of on-heap buffers and direct
+ * buffers are allowed.
+ *
+ * If the coder option ALLOW_CHANGE_INPUTS is set true (false by default), the
+ * content of input buffers may change after the call, subject to concrete
+ * implementation.
+ *
+ * @param inputs input buffers to read data from. The buffers' remaining will
+ * be 0 after decoding
+ * @param erasedIndexes indexes of erased units in the inputs array
+ * @param outputs output buffers to put decoded data into according to
+ * erasedIndexes, ready for read after the call
+ */
+ public synchronized void decode(ByteBuffer[] inputs, int[] erasedIndexes,
+ ByteBuffer[] outputs) throws IOException {
+ ByteBufferDecodingState decodingState = new ByteBufferDecodingState(this,
+ inputs, erasedIndexes, outputs);
+
+ boolean usingDirectBuffer = decodingState.usingDirectBuffer;
+ int dataLen = decodingState.decodeLength;
+ if (dataLen == 0) {
+ return;
+ }
+
+ int[] inputPositions = new int[inputs.length];
+ for (int i = 0; i < inputPositions.length; i++) {
+ if (inputs[i] != null) {
+ inputPositions[i] = inputs[i].position();
+ }
+ }
+
+ if (usingDirectBuffer) {
+ doDecode(decodingState);
+ } else {
+ ByteArrayDecodingState badState = decodingState.convertToByteArrayState();
+ doDecode(badState);
+ }
+
+ for (int i = 0; i < inputs.length; i++) {
+ if (inputs[i] != null) {
+ // dataLen bytes consumed
+ inputs[i].position(inputPositions[i] + dataLen);
+ }
+ }
+ }
+
+ /**
+ * Perform the real decoding using Direct ByteBuffer.
+ * @param decodingState the decoding state
+ */
+ protected abstract void doDecode(ByteBufferDecodingState decodingState)
+ throws IOException;
+
+ /**
+ * Decode with inputs and erasedIndexes, generates outputs. More see above.
+ *
+ * @param inputs input buffers to read data from
+ * @param erasedIndexes indexes of erased units in the inputs array
+ * @param outputs output buffers to put decoded data into according to
+ * erasedIndexes, ready for read after the call
+ * @throws IOException if the decoder is closed.
+ */
+ public synchronized void decode(byte[][] inputs, int[] erasedIndexes,
+ byte[][] outputs) throws IOException {
+ ByteArrayDecodingState decodingState = new ByteArrayDecodingState(this,
+ inputs, erasedIndexes, outputs);
+
+ if (decodingState.decodeLength == 0) {
+ return;
+ }
+
+ doDecode(decodingState);
+ }
+
+ /**
+ * Perform the real decoding using bytes array, supporting offsets and
+ * lengths.
+ * @param decodingState the decoding state
+ * @throws IOException if the decoder is closed.
+ */
+ protected abstract void doDecode(ByteArrayDecodingState decodingState)
+ throws IOException;
+
+ /**
+ * Decode with inputs and erasedIndexes, generates outputs. More see above.
+ *
+ * Note, for both input and output ECChunks, no mixing of on-heap buffers and
+ * direct buffers are allowed.
+ *
+ * @param inputs input buffers to read data from
+ * @param erasedIndexes indexes of erased units in the inputs array
+ * @param outputs output buffers to put decoded data into according to
+ * erasedIndexes, ready for read after the call
+ * @throws IOException if the decoder is closed
+ */
+ public synchronized void decode(ECChunk[] inputs, int[] erasedIndexes,
+ ECChunk[] outputs) throws IOException {
+ ByteBuffer[] newInputs = CoderUtil.toBuffers(inputs);
+ ByteBuffer[] newOutputs = CoderUtil.toBuffers(outputs);
+ decode(newInputs, erasedIndexes, newOutputs);
+ }
+
+ public int getNumDataUnits() {
+ return coderOptions.getData();
+ }
+
+ public int getNumParityUnits() {
+ return coderOptions.getParity();
+ }
+
+ protected int getNumAllUnits() {
+ return coderOptions.getData() + coderOptions.getParity();
+ }
+
+ /**
+ * Tell if direct buffer is preferred or not. It's for callers to
+ * decide how to allocate coding chunk buffers, using DirectByteBuffer or
+ * bytes array. It will return false by default.
+ * @return true if native buffer is preferred for performance consideration,
+ * otherwise false.
+ */
+ public boolean preferDirectBuffer() {
+ return false;
+ }
+
+ /**
+ * Allow change into input buffers or not while perform encoding/decoding.
+ * @return true if it's allowed to change inputs, false otherwise
+ */
+ public boolean allowChangeInputs() {
+ return false;
+ }
+
+ /**
+ * Allow to dump verbose info during encoding/decoding.
+ * @return true if it's allowed to do verbose dump, false otherwise.
+ */
+ public boolean allowVerboseDump() {
+ return false;
+ }
+
+ /**
+ * Should be called when release this coder. Good chance to release encoding
+ * or decoding buffers
+ */
+ public void release() {
+ // Nothing to do here.
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureEncoder.java
new file mode 100644
index 0000000..2d3e704
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/RawErasureEncoder.java
@@ -0,0 +1,193 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.ECChunk;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * An abstract raw erasure encoder that's to be inherited by new encoders.
+ *
+ * Raw erasure coder is part of erasure codec framework, where erasure coder is
+ * used to encode/decode a group of blocks (BlockGroup) according to the codec
+ * specific BlockGroup layout and logic. An erasure coder extracts chunks of
+ * data from the blocks and can employ various low level raw erasure coders to
+ * perform encoding/decoding against the chunks.
+ *
+ * To distinguish from erasure coder, here raw erasure coder is used to mean the
+ * low level constructs, since it only takes care of the math calculation with
+ * a group of byte buffers.
+ *
+ * Note it mainly provides encode() calls, which should be stateless and may be
+ * made thread-safe in future.
+ */
+public abstract class RawErasureEncoder {
+
+ private final ECReplicationConfig coderOptions;
+
+ public RawErasureEncoder(ECReplicationConfig coderOptions) {
+ this.coderOptions = coderOptions;
+ }
+
+ /**
+ * Encode with inputs and generates outputs.
+ *
+ * Note, for both inputs and outputs, no mixing of on-heap buffers and direct
+ * buffers are allowed.
+ *
+ * If the coder option ALLOW_CHANGE_INPUTS is set true (false by default), the
+ * content of input buffers may change after the call, subject to concrete
+ * implementation. Anyway the positions of input buffers will move forward.
+ *
+ * @param inputs input buffers to read data from. The buffers' remaining will
+ * be 0 after encoding
+ * @param outputs output buffers to put the encoded data into, ready to read
+ * after the call
+ * @throws IOException if the encoder is closed.
+ */
+ public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs)
+ throws IOException {
+ ByteBufferEncodingState bbeState = new ByteBufferEncodingState(
+ this, inputs, outputs);
+
+ boolean usingDirectBuffer = bbeState.usingDirectBuffer;
+ int dataLen = bbeState.encodeLength;
+ if (dataLen == 0) {
+ return;
+ }
+
+ int[] inputPositions = new int[inputs.length];
+ for (int i = 0; i < inputPositions.length; i++) {
+ if (inputs[i] != null) {
+ inputPositions[i] = inputs[i].position();
+ }
+ }
+
+ if (usingDirectBuffer) {
+ doEncode(bbeState);
+ } else {
+ ByteArrayEncodingState baeState = bbeState.convertToByteArrayState();
+ doEncode(baeState);
+ }
+
+ for (int i = 0; i < inputs.length; i++) {
+ if (inputs[i] != null) {
+ // dataLen bytes consumed
+ inputs[i].position(inputPositions[i] + dataLen);
+ }
+ }
+ }
+
+ /**
+ * Perform the real encoding work using direct ByteBuffer.
+ * @param encodingState the encoding state
+ */
+ protected abstract void doEncode(ByteBufferEncodingState encodingState)
+ throws IOException;
+
+ /**
+ * Encode with inputs and generates outputs. More see above.
+ *
+ * @param inputs input buffers to read data from
+ * @param outputs output buffers to put the encoded data into, read to read
+ * after the call
+ */
+ public void encode(byte[][] inputs, byte[][] outputs) throws IOException {
+ ByteArrayEncodingState baeState = new ByteArrayEncodingState(
+ this, inputs, outputs);
+
+ int dataLen = baeState.encodeLength;
+ if (dataLen == 0) {
+ return;
+ }
+
+ doEncode(baeState);
+ }
+
+ /**
+ * Perform the real encoding work using bytes array, supporting offsets
+ * and lengths.
+ * @param encodingState the encoding state
+ */
+ protected abstract void doEncode(ByteArrayEncodingState encodingState)
+ throws IOException;
+
+ /**
+ * Encode with inputs and generates outputs. More see above.
+ *
+ * @param inputs input buffers to read data from
+ * @param outputs output buffers to put the encoded data into, read to read
+ * after the call
+ * @throws IOException if the encoder is closed.
+ */
+ public void encode(ECChunk[] inputs, ECChunk[] outputs) throws IOException {
+ ByteBuffer[] newInputs = ECChunk.toBuffers(inputs);
+ ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs);
+ encode(newInputs, newOutputs);
+ }
+
+ public int getNumDataUnits() {
+ return coderOptions.getData();
+ }
+
+ public int getNumParityUnits() {
+ return coderOptions.getParity();
+ }
+
+ public int getNumAllUnits() {
+ return coderOptions.getData() + coderOptions.getParity();
+ }
+
+ /**
+ * Tell if direct buffer is preferred or not. It's for callers to
+ * decide how to allocate coding chunk buffers, using DirectByteBuffer or
+ * bytes array. It will return false by default.
+ * @return true if native buffer is preferred for performance consideration,
+ * otherwise false.
+ */
+ public boolean preferDirectBuffer() {
+ return false;
+ }
+
+ /**
+ * Allow change into input buffers or not while perform encoding/decoding.
+ * @return true if it's allowed to change inputs, false otherwise
+ */
+ public boolean allowChangeInputs() {
+ return false;
+ }
+
+ /**
+ * Allow to dump verbose info during encoding/decoding.
+ * @return true if it's allowed to do verbose dump, false otherwise.
+ */
+ public boolean allowVerboseDump() {
+ return false;
+ }
+
+ /**
+ * Should be called when release this coder. Good chance to release encoding
+ * or decoding buffers
+ */
+ public void release() {
+ // Nothing to do here.
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawDecoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawDecoder.java
new file mode 100644
index 0000000..1829b17
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawDecoder.java
@@ -0,0 +1,88 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ *
+ * XOR code is an important primitive code scheme in erasure coding and often
+ * used in advanced codes, like HitchHiker and LRC, though itself is rarely
+ * deployed independently.
+ */
+@InterfaceAudience.Private
+public class XORRawDecoder extends RawErasureDecoder {
+
+ public XORRawDecoder(ECReplicationConfig ecReplicationConfig) {
+ super(ecReplicationConfig);
+ }
+
+ @Override
+ protected void doDecode(ByteBufferDecodingState decodingState) {
+ CoderUtil.resetOutputBuffers(decodingState.outputs,
+ decodingState.decodeLength);
+ ByteBuffer output = decodingState.outputs[0];
+
+ int erasedIdx = decodingState.erasedIndexes[0];
+
+ // Process the inputs.
+ int iIdx, oIdx;
+ for (int i = 0; i < decodingState.inputs.length; i++) {
+ // Skip the erased location.
+ if (i == erasedIdx) {
+ continue;
+ }
+
+ for (iIdx = decodingState.inputs[i].position(), oIdx = output.position();
+ iIdx < decodingState.inputs[i].limit();
+ iIdx++, oIdx++) {
+ output.put(oIdx, (byte) (output.get(oIdx) ^
+ decodingState.inputs[i].get(iIdx)));
+ }
+ }
+ }
+
+ @Override
+ protected void doDecode(ByteArrayDecodingState decodingState) {
+ byte[] output = decodingState.outputs[0];
+ int dataLen = decodingState.decodeLength;
+ CoderUtil.resetOutputBuffers(decodingState.outputs,
+ decodingState.outputOffsets, dataLen);
+ int erasedIdx = decodingState.erasedIndexes[0];
+
+ // Process the inputs.
+ int iIdx, oIdx;
+ for (int i = 0; i < decodingState.inputs.length; i++) {
+ // Skip the erased location.
+ if (i == erasedIdx) {
+ continue;
+ }
+
+ for (iIdx = decodingState.inputOffsets[i],
+ oIdx = decodingState.outputOffsets[0];
+ iIdx < decodingState.inputOffsets[i] + dataLen; iIdx++, oIdx++) {
+ output[oIdx] ^= decodingState.inputs[i][iIdx];
+ }
+ }
+ }
+
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawEncoder.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawEncoder.java
new file mode 100644
index 0000000..c02e97c
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawEncoder.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ *
+ * XOR code is an important primitive code scheme in erasure coding and often
+ * used in advanced codes, like HitchHiker and LRC, though itself is rarely
+ * deployed independently.
+ */
+@InterfaceAudience.Private
+public class XORRawEncoder extends RawErasureEncoder {
+
+ public XORRawEncoder(ECReplicationConfig ecReplicationConfig) {
+ super(ecReplicationConfig);
+ }
+
+ protected void doEncode(ByteBufferEncodingState encodingState) {
+ CoderUtil.resetOutputBuffers(encodingState.outputs,
+ encodingState.encodeLength);
+ ByteBuffer output = encodingState.outputs[0];
+
+ // Get the first buffer's data.
+ int iIdx, oIdx;
+ for (iIdx = encodingState.inputs[0].position(), oIdx = output.position();
+ iIdx < encodingState.inputs[0].limit(); iIdx++, oIdx++) {
+ output.put(oIdx, encodingState.inputs[0].get(iIdx));
+ }
+
+ // XOR with everything else.
+ for (int i = 1; i < encodingState.inputs.length; i++) {
+ for (iIdx = encodingState.inputs[i].position(), oIdx = output.position();
+ iIdx < encodingState.inputs[i].limit();
+ iIdx++, oIdx++) {
+ output.put(oIdx, (byte) (output.get(oIdx) ^
+ encodingState.inputs[i].get(iIdx)));
+ }
+ }
+ }
+
+ @Override
+ protected void doEncode(ByteArrayEncodingState encodingState) {
+ int dataLen = encodingState.encodeLength;
+ CoderUtil.resetOutputBuffers(encodingState.outputs,
+ encodingState.outputOffsets, dataLen);
+ byte[] output = encodingState.outputs[0];
+
+ // Get the first buffer's data.
+ int iIdx, oIdx;
+ for (iIdx = encodingState.inputOffsets[0],
+ oIdx = encodingState.outputOffsets[0];
+ iIdx < encodingState.inputOffsets[0] + dataLen; iIdx++, oIdx++) {
+ output[oIdx] = encodingState.inputs[0][iIdx];
+ }
+
+ // XOR with everything else.
+ for (int i = 1; i < encodingState.inputs.length; i++) {
+ for (iIdx = encodingState.inputOffsets[i],
+ oIdx = encodingState.outputOffsets[0];
+ iIdx < encodingState.inputOffsets[i] + dataLen; iIdx++, oIdx++) {
+ output[oIdx] ^= encodingState.inputs[i][iIdx];
+ }
+ }
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawErasureCoderFactory.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawErasureCoderFactory.java
new file mode 100644
index 0000000..631af5b
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/XORRawErasureCoderFactory.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+
+/**
+ * A raw coder factory for raw XOR coder.
+ */
+@InterfaceAudience.Private
+public class XORRawErasureCoderFactory implements RawErasureCoderFactory {
+
+ public static final String CODER_NAME = "xor_java";
+
+
+ @Override
+ public RawErasureEncoder createEncoder(
+ ECReplicationConfig replicationConfig) {
+ return new XORRawEncoder(replicationConfig);
+ }
+
+ @Override
+ public RawErasureDecoder createDecoder(
+ ECReplicationConfig replicationConfig) {
+ return new XORRawDecoder(replicationConfig);
+ }
+
+ @Override
+ public String getCoderName() {
+ return CODER_NAME;
+ }
+
+ @Override
+ public String getCodecName() {
+ return ECReplicationConfig.EcCodec.XOR.name().toLowerCase();
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/package-info.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/package-info.java
new file mode 100644
index 0000000..cfad4a3
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/package-info.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.
+ */
+
+/**
+ *
+ * Raw erasure coders.
+ *
+ * Raw erasure coder is part of erasure codec framework, where erasure coder is
+ * used to encode/decode a group of blocks (BlockGroup) according to the codec
+ * specific BlockGroup layout and logic. An erasure coder extracts chunks of
+ * data from the blocks and can employ various low level raw erasure coders to
+ * perform encoding/decoding against the chunks.
+ *
+ * To distinguish from erasure coder, here raw erasure coder is used to mean the
+ * low level constructs, since it only takes care of the math calculation with
+ * a group of byte buffers.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/CodecUtil.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/CodecUtil.java
new file mode 100644
index 0000000..4b4ab41
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/CodecUtil.java
@@ -0,0 +1,111 @@
+/**
+ * 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.ozone.erasurecode.rawcoder.util;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.CodecRegistry;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A codec & coder utility to help create coders conveniently.
+ * <p>
+ * {@link CodecUtil} includes erasure coder configurations key and default
+ * values such as coder class name and erasure codec option values included
+ * by {@link ECReplicationConfig}.{@link RawErasureEncoder} and
+ * {@link RawErasureDecoder} are created by createRawEncoder and
+ * createRawDecoder.
+ */
+@InterfaceAudience.Private
+public final class CodecUtil {
+
+ public static final Logger LOG = LoggerFactory.getLogger(CodecUtil.class);
+
+ private CodecUtil() {
+ }
+
+ private static RawErasureCoderFactory createRawCoderFactory(String coderName,
+ String codecName) {
+ RawErasureCoderFactory fact;
+ fact = CodecRegistry.getInstance().
+ getCoderByName(codecName, coderName);
+
+ return fact;
+ }
+
+ public static RawErasureEncoder createRawEncoderWithFallback(
+ final ECReplicationConfig ecReplicationConfig) {
+ // Note: Coders can be configurable, but for now, we just use whats
+ // available.
+ String codecName = ecReplicationConfig.getCodec().name().toLowerCase();
+ String[] rawCoderNames =
+ CodecRegistry.getInstance().getCoderNames(codecName);
+ for (String rawCoderName : rawCoderNames) {
+ try {
+ if (rawCoderName != null) {
+ RawErasureCoderFactory fact =
+ createRawCoderFactory(rawCoderName, codecName);
+ return fact.createEncoder(ecReplicationConfig);
+ }
+ } catch (LinkageError | Exception e) {
+ // Fallback to next coder if possible
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(
+ "Failed to create raw erasure encoder " + rawCoderName
+ + ", fallback to next codec if possible",
+ e);
+ }
+ }
+ }
+ throw new IllegalArgumentException(
+ "Fail to create raw erasure " + "encoder with given codec: "
+ + codecName);
+ }
+
+ public static RawErasureDecoder createRawDecoderWithFallback(
+ final ECReplicationConfig ecReplicationConfig) {
+ // Note: Coders can be configurable, but for now, we just use whats
+ // available.
+ String codecName = ecReplicationConfig.getCodec().name().toLowerCase();
+ String[] coders = CodecRegistry.getInstance().getCoderNames(codecName);
+ for (String rawCoderName : coders) {
+ try {
+ if (rawCoderName != null) {
+ RawErasureCoderFactory fact =
+ createRawCoderFactory(rawCoderName, codecName);
+ return fact.createDecoder(ecReplicationConfig);
+ }
+ } catch (LinkageError | Exception e) {
+ // Fallback to next coder if possible
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(
+ "Failed to create raw erasure decoder " + rawCoderName
+ + ", fallback to next codec if possible",
+ e);
+ }
+ }
+ }
+ throw new IllegalArgumentException(
+ "Fail to create raw erasure " + "decoder with given codec: "
+ + codecName);
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/DumpUtil.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/DumpUtil.java
new file mode 100644
index 0000000..54e9d76
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/DumpUtil.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ozone.erasurecode.rawcoder.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.ozone.erasurecode.ECChunk;
+
+/**
+ * A dump utility class for debugging data erasure coding/decoding issues.
+ * Don't suggest they are used in runtime production codes.
+ */
+@InterfaceAudience.Private
+public final class DumpUtil {
+ private static final String HEX_CHARS_STR = "0123456789ABCDEF";
+ private static final char[] HEX_CHARS = HEX_CHARS_STR.toCharArray();
+
+ private DumpUtil() {
+ // No called
+ }
+
+ /**
+ * Convert bytes into format like 0x02 02 00 80.
+ * If limit is negative or too large, then all bytes will be converted.
+ */
+ public static String bytesToHex(byte[] bytes, int limit) {
+ if (limit <= 0 || limit > bytes.length) {
+ limit = bytes.length;
+ }
+ int len = limit * 2;
+ len += limit; // for ' ' appended for each char
+ len += 2; // for '0x' prefix
+ char[] hexChars = new char[len];
+ hexChars[0] = '0';
+ hexChars[1] = 'x';
+ for (int j = 0; j < limit; j++) {
+ int v = bytes[j] & 0xFF;
+ hexChars[j * 3 + 2] = HEX_CHARS[v >>> 4];
+ hexChars[j * 3 + 3] = HEX_CHARS[v & 0x0F];
+ hexChars[j * 3 + 4] = ' ';
+ }
+
+ return new String(hexChars);
+ }
+
+ public static void dumpMatrix(byte[] matrix,
+ int numDataUnits, int numAllUnits) {
+ for (int i = 0; i < numDataUnits; i++) {
+ for (int j = 0; j < numAllUnits; j++) {
+ System.out.print(" ");
+ System.out.print(0xff & matrix[i * numAllUnits + j]);
+ }
+ System.out.println();
+ }
+ }
+
+ /**
+ * Print data in hex format in an array of chunks.
+ * @param header
+ * @param chunks
+ */
+ public static void dumpChunks(String header, ECChunk[] chunks) {
+ System.out.println();
+ System.out.println(header);
+ for (int i = 0; i < chunks.length; i++) {
+ dumpChunk(chunks[i]);
+ }
+ System.out.println();
+ }
+
+ /**
+ * Print data in hex format in a chunk.
+ * @param chunk
+ */
+ public static void dumpChunk(ECChunk chunk) {
+ String str;
+ if (chunk == null) {
+ str = "<EMPTY>";
+ } else {
+ byte[] bytes = chunk.toBytesArray();
+ str = DumpUtil.bytesToHex(bytes, 16);
+ }
+ System.out.println(str);
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GF256.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GF256.java
new file mode 100644
index 0000000..4e0b2d2
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GF256.java
@@ -0,0 +1,333 @@
+/**
+ * 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.ozone.erasurecode.rawcoder.util;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A GaloisField utility class only caring of 256 fields for efficiency. Some
+ * of the codes are borrowed from ISA-L implementation (C or ASM codes).
+ */
+@InterfaceAudience.Private
+public final class GF256 {
+
+ private GF256() { }
+
+ private static final byte[] GF_BASE = new byte[]{
+ (byte) 0x01, (byte) 0x02, (byte) 0x04, (byte) 0x08, (byte) 0x10,
+ (byte) 0x20, (byte) 0x40, (byte) 0x80, (byte) 0x1d, (byte) 0x3a,
+ (byte) 0x74, (byte) 0xe8, (byte) 0xcd, (byte) 0x87, (byte) 0x13,
+ (byte) 0x26, (byte) 0x4c, (byte) 0x98, (byte) 0x2d, (byte) 0x5a,
+ (byte) 0xb4, (byte) 0x75, (byte) 0xea, (byte) 0xc9, (byte) 0x8f,
+ (byte) 0x03, (byte) 0x06, (byte) 0x0c, (byte) 0x18, (byte) 0x30,
+ (byte) 0x60, (byte) 0xc0, (byte) 0x9d, (byte) 0x27, (byte) 0x4e,
+ (byte) 0x9c, (byte) 0x25, (byte) 0x4a, (byte) 0x94, (byte) 0x35,
+ (byte) 0x6a, (byte) 0xd4, (byte) 0xb5, (byte) 0x77, (byte) 0xee,
+ (byte) 0xc1, (byte) 0x9f, (byte) 0x23, (byte) 0x46, (byte) 0x8c,
+ (byte) 0x05, (byte) 0x0a, (byte) 0x14, (byte) 0x28, (byte) 0x50,
+ (byte) 0xa0, (byte) 0x5d, (byte) 0xba, (byte) 0x69, (byte) 0xd2,
+ (byte) 0xb9, (byte) 0x6f, (byte) 0xde, (byte) 0xa1, (byte) 0x5f,
+ (byte) 0xbe, (byte) 0x61, (byte) 0xc2, (byte) 0x99, (byte) 0x2f,
+ (byte) 0x5e, (byte) 0xbc, (byte) 0x65, (byte) 0xca, (byte) 0x89,
+ (byte) 0x0f, (byte) 0x1e, (byte) 0x3c, (byte) 0x78, (byte) 0xf0,
+ (byte) 0xfd, (byte) 0xe7, (byte) 0xd3, (byte) 0xbb, (byte) 0x6b,
+ (byte) 0xd6, (byte) 0xb1, (byte) 0x7f, (byte) 0xfe, (byte) 0xe1,
+ (byte) 0xdf, (byte) 0xa3, (byte) 0x5b, (byte) 0xb6, (byte) 0x71,
+ (byte) 0xe2, (byte) 0xd9, (byte) 0xaf, (byte) 0x43, (byte) 0x86,
+ (byte) 0x11, (byte) 0x22, (byte) 0x44, (byte) 0x88, (byte) 0x0d,
+ (byte) 0x1a, (byte) 0x34, (byte) 0x68, (byte) 0xd0, (byte) 0xbd,
+ (byte) 0x67, (byte) 0xce, (byte) 0x81, (byte) 0x1f, (byte) 0x3e,
+ (byte) 0x7c, (byte) 0xf8, (byte) 0xed, (byte) 0xc7, (byte) 0x93,
+ (byte) 0x3b, (byte) 0x76, (byte) 0xec, (byte) 0xc5, (byte) 0x97,
+ (byte) 0x33, (byte) 0x66, (byte) 0xcc, (byte) 0x85, (byte) 0x17,
+ (byte) 0x2e, (byte) 0x5c, (byte) 0xb8, (byte) 0x6d, (byte) 0xda,
+ (byte) 0xa9, (byte) 0x4f, (byte) 0x9e, (byte) 0x21, (byte) 0x42,
+ (byte) 0x84, (byte) 0x15, (byte) 0x2a, (byte) 0x54, (byte) 0xa8,
+ (byte) 0x4d, (byte) 0x9a, (byte) 0x29, (byte) 0x52, (byte) 0xa4,
+ (byte) 0x55, (byte) 0xaa, (byte) 0x49, (byte) 0x92, (byte) 0x39,
+ (byte) 0x72, (byte) 0xe4, (byte) 0xd5, (byte) 0xb7, (byte) 0x73,
+ (byte) 0xe6, (byte) 0xd1, (byte) 0xbf, (byte) 0x63, (byte) 0xc6,
+ (byte) 0x91, (byte) 0x3f, (byte) 0x7e, (byte) 0xfc, (byte) 0xe5,
+ (byte) 0xd7, (byte) 0xb3, (byte) 0x7b, (byte) 0xf6, (byte) 0xf1,
+ (byte) 0xff, (byte) 0xe3, (byte) 0xdb, (byte) 0xab, (byte) 0x4b,
+ (byte) 0x96, (byte) 0x31, (byte) 0x62, (byte) 0xc4, (byte) 0x95,
+ (byte) 0x37, (byte) 0x6e, (byte) 0xdc, (byte) 0xa5, (byte) 0x57,
+ (byte) 0xae, (byte) 0x41, (byte) 0x82, (byte) 0x19, (byte) 0x32,
+ (byte) 0x64, (byte) 0xc8, (byte) 0x8d, (byte) 0x07, (byte) 0x0e,
+ (byte) 0x1c, (byte) 0x38, (byte) 0x70, (byte) 0xe0, (byte) 0xdd,
+ (byte) 0xa7, (byte) 0x53, (byte) 0xa6, (byte) 0x51, (byte) 0xa2,
+ (byte) 0x59, (byte) 0xb2, (byte) 0x79, (byte) 0xf2, (byte) 0xf9,
+ (byte) 0xef, (byte) 0xc3, (byte) 0x9b, (byte) 0x2b, (byte) 0x56,
+ (byte) 0xac, (byte) 0x45, (byte) 0x8a, (byte) 0x09, (byte) 0x12,
+ (byte) 0x24, (byte) 0x48, (byte) 0x90, (byte) 0x3d, (byte) 0x7a,
+ (byte) 0xf4, (byte) 0xf5, (byte) 0xf7, (byte) 0xf3, (byte) 0xfb,
+ (byte) 0xeb, (byte) 0xcb, (byte) 0x8b, (byte) 0x0b, (byte) 0x16,
+ (byte) 0x2c, (byte) 0x58, (byte) 0xb0, (byte) 0x7d, (byte) 0xfa,
+ (byte) 0xe9, (byte) 0xcf, (byte) 0x83, (byte) 0x1b, (byte) 0x36,
+ (byte) 0x6c, (byte) 0xd8, (byte) 0xad, (byte) 0x47, (byte) 0x8e,
+ (byte) 0x01
+ };
+
+ private static final byte[] GF_LOG_BASE = new byte[]{
+ (byte) 0x00, (byte) 0xff, (byte) 0x01, (byte) 0x19, (byte) 0x02,
+ (byte) 0x32, (byte) 0x1a, (byte) 0xc6, (byte) 0x03, (byte) 0xdf,
+ (byte) 0x33, (byte) 0xee, (byte) 0x1b, (byte) 0x68, (byte) 0xc7,
+ (byte) 0x4b, (byte) 0x04, (byte) 0x64, (byte) 0xe0, (byte) 0x0e,
+ (byte) 0x34, (byte) 0x8d, (byte) 0xef, (byte) 0x81, (byte) 0x1c,
+ (byte) 0xc1, (byte) 0x69, (byte) 0xf8, (byte) 0xc8, (byte) 0x08,
+ (byte) 0x4c, (byte) 0x71, (byte) 0x05, (byte) 0x8a, (byte) 0x65,
+ (byte) 0x2f, (byte) 0xe1, (byte) 0x24, (byte) 0x0f, (byte) 0x21,
+ (byte) 0x35, (byte) 0x93, (byte) 0x8e, (byte) 0xda, (byte) 0xf0,
+ (byte) 0x12, (byte) 0x82, (byte) 0x45, (byte) 0x1d, (byte) 0xb5,
+ (byte) 0xc2, (byte) 0x7d, (byte) 0x6a, (byte) 0x27, (byte) 0xf9,
+ (byte) 0xb9, (byte) 0xc9, (byte) 0x9a, (byte) 0x09, (byte) 0x78,
+ (byte) 0x4d, (byte) 0xe4, (byte) 0x72, (byte) 0xa6, (byte) 0x06,
+ (byte) 0xbf, (byte) 0x8b, (byte) 0x62, (byte) 0x66, (byte) 0xdd,
+ (byte) 0x30, (byte) 0xfd, (byte) 0xe2, (byte) 0x98, (byte) 0x25,
+ (byte) 0xb3, (byte) 0x10, (byte) 0x91, (byte) 0x22, (byte) 0x88,
+ (byte) 0x36, (byte) 0xd0, (byte) 0x94, (byte) 0xce, (byte) 0x8f,
+ (byte) 0x96, (byte) 0xdb, (byte) 0xbd, (byte) 0xf1, (byte) 0xd2,
+ (byte) 0x13, (byte) 0x5c, (byte) 0x83, (byte) 0x38, (byte) 0x46,
+ (byte) 0x40, (byte) 0x1e, (byte) 0x42, (byte) 0xb6, (byte) 0xa3,
+ (byte) 0xc3, (byte) 0x48, (byte) 0x7e, (byte) 0x6e, (byte) 0x6b,
+ (byte) 0x3a, (byte) 0x28, (byte) 0x54, (byte) 0xfa, (byte) 0x85,
+ (byte) 0xba, (byte) 0x3d, (byte) 0xca, (byte) 0x5e, (byte) 0x9b,
+ (byte) 0x9f, (byte) 0x0a, (byte) 0x15, (byte) 0x79, (byte) 0x2b,
+ (byte) 0x4e, (byte) 0xd4, (byte) 0xe5, (byte) 0xac, (byte) 0x73,
+ (byte) 0xf3, (byte) 0xa7, (byte) 0x57, (byte) 0x07, (byte) 0x70,
+ (byte) 0xc0, (byte) 0xf7, (byte) 0x8c, (byte) 0x80, (byte) 0x63,
+ (byte) 0x0d, (byte) 0x67, (byte) 0x4a, (byte) 0xde, (byte) 0xed,
+ (byte) 0x31, (byte) 0xc5, (byte) 0xfe, (byte) 0x18, (byte) 0xe3,
+ (byte) 0xa5, (byte) 0x99, (byte) 0x77, (byte) 0x26, (byte) 0xb8,
+ (byte) 0xb4, (byte) 0x7c, (byte) 0x11, (byte) 0x44, (byte) 0x92,
+ (byte) 0xd9, (byte) 0x23, (byte) 0x20, (byte) 0x89, (byte) 0x2e,
+ (byte) 0x37, (byte) 0x3f, (byte) 0xd1, (byte) 0x5b, (byte) 0x95,
+ (byte) 0xbc, (byte) 0xcf, (byte) 0xcd, (byte) 0x90, (byte) 0x87,
+ (byte) 0x97, (byte) 0xb2, (byte) 0xdc, (byte) 0xfc, (byte) 0xbe,
+ (byte) 0x61, (byte) 0xf2, (byte) 0x56, (byte) 0xd3, (byte) 0xab,
+ (byte) 0x14, (byte) 0x2a, (byte) 0x5d, (byte) 0x9e, (byte) 0x84,
+ (byte) 0x3c, (byte) 0x39, (byte) 0x53, (byte) 0x47, (byte) 0x6d,
+ (byte) 0x41, (byte) 0xa2, (byte) 0x1f, (byte) 0x2d, (byte) 0x43,
+ (byte) 0xd8, (byte) 0xb7, (byte) 0x7b, (byte) 0xa4, (byte) 0x76,
+ (byte) 0xc4, (byte) 0x17, (byte) 0x49, (byte) 0xec, (byte) 0x7f,
+ (byte) 0x0c, (byte) 0x6f, (byte) 0xf6, (byte) 0x6c, (byte) 0xa1,
+ (byte) 0x3b, (byte) 0x52, (byte) 0x29, (byte) 0x9d, (byte) 0x55,
+ (byte) 0xaa, (byte) 0xfb, (byte) 0x60, (byte) 0x86, (byte) 0xb1,
+ (byte) 0xbb, (byte) 0xcc, (byte) 0x3e, (byte) 0x5a, (byte) 0xcb,
+ (byte) 0x59, (byte) 0x5f, (byte) 0xb0, (byte) 0x9c, (byte) 0xa9,
+ (byte) 0xa0, (byte) 0x51, (byte) 0x0b, (byte) 0xf5, (byte) 0x16,
+ (byte) 0xeb, (byte) 0x7a, (byte) 0x75, (byte) 0x2c, (byte) 0xd7,
+ (byte) 0x4f, (byte) 0xae, (byte) 0xd5, (byte) 0xe9, (byte) 0xe6,
+ (byte) 0xe7, (byte) 0xad, (byte) 0xe8, (byte) 0x74, (byte) 0xd6,
+ (byte) 0xf4, (byte) 0xea, (byte) 0xa8, (byte) 0x50, (byte) 0x58,
+ (byte) 0xaf
+ };
+
+ @SuppressFBWarnings("MS_EXPOSE_REP")
+ private static byte[][] theGfMulTab; // multiply result table in GF 256 space
+
+ /**
+ * Initialize the GF multiply table for performance. Just compute once, and
+ * avoid repeatedly doing the multiply during encoding/decoding.
+ */
+ static {
+ theGfMulTab = new byte[256][256];
+ for (int i = 0; i < 256; i++) {
+ for (int j = 0; j < 256; j++) {
+ theGfMulTab[i][j] = gfMul((byte) i, (byte) j);
+ }
+ }
+ }
+
+ /**
+ * Get the big GF multiply table so utilize it efficiently.
+ * @return the big GF multiply table
+ */
+ public static byte[][] gfMulTab() {
+ return theGfMulTab;
+ }
+
+ public static byte gfMul(byte a, byte b) {
+ if ((a == 0) || (b == 0)) {
+ return 0;
+ }
+
+ int tmp = (GF_LOG_BASE[a & 0xff] & 0xff) +
+ (GF_LOG_BASE[b & 0xff] & 0xff);
+ if (tmp > 254) {
+ tmp -= 255;
+ }
+
+ return GF_BASE[tmp];
+ }
+
+ public static byte gfInv(byte a) {
+ if (a == 0) {
+ return 0;
+ }
+
+ return GF_BASE[255 - GF_LOG_BASE[a & 0xff] & 0xff];
+ }
+
+ /**
+ * Invert a matrix assuming it's invertible.
+ *
+ * Ported from Intel ISA-L library.
+ */
+ public static void gfInvertMatrix(byte[] inMatrix, byte[] outMatrix, int n) {
+ byte temp;
+
+ // Set outMatrix[] to the identity matrix
+ for (int i = 0; i < n * n; i++) {
+ // memset(outMatrix, 0, n*n)
+ outMatrix[i] = 0;
+ }
+
+ for (int i = 0; i < n; i++) {
+ outMatrix[i * n + i] = 1;
+ }
+
+ // Inverse
+ for (int j, i = 0; i < n; i++) {
+ // Check for 0 in pivot element
+ if (inMatrix[i * n + i] == 0) {
+ // Find a row with non-zero in current column and swap
+ for (j = i + 1; j < n; j++) {
+ if (inMatrix[j * n + i] != 0) {
+ break;
+ }
+ }
+ if (j == n) {
+ // Couldn't find means it's singular
+ throw new RuntimeException("Not invertible");
+ }
+
+ for (int k = 0; k < n; k++) {
+ // Swap rows i,j
+ temp = inMatrix[i * n + k];
+ inMatrix[i * n + k] = inMatrix[j * n + k];
+ inMatrix[j * n + k] = temp;
+
+ temp = outMatrix[i * n + k];
+ outMatrix[i * n + k] = outMatrix[j * n + k];
+ outMatrix[j * n + k] = temp;
+ }
+ }
+
+ temp = gfInv(inMatrix[i * n + i]); // 1/pivot
+ for (j = 0; j < n; j++) {
+ // Scale row i by 1/pivot
+ inMatrix[i * n + j] = gfMul(inMatrix[i * n + j], temp);
+ outMatrix[i * n + j] = gfMul(outMatrix[i * n + j], temp);
+ }
+
+ for (j = 0; j < n; j++) {
+ if (j == i) {
+ continue;
+ }
+
+ temp = inMatrix[j * n + i];
+ for (int k = 0; k < n; k++) {
+ outMatrix[j * n + k] ^= gfMul(temp, outMatrix[i * n + k]);
+ inMatrix[j * n + k] ^= gfMul(temp, inMatrix[i * n + k]);
+ }
+ }
+ }
+ }
+
+ /**
+ * Ported from Intel ISA-L library.
+ *
+ * Calculates const table gftbl in GF(2^8) from single input A
+ * gftbl(A) = {A{00}, A{01}, A{02}, ... , A{0f} }, {A{00}, A{10}, A{20},
+ * ... , A{f0} } -- from ISA-L implementation
+ */
+ public static void gfVectMulInit(byte c, byte[] tbl, int offset) {
+ byte c2 = (byte) ((c << 1) ^ ((c & 0x80) != 0 ? 0x1d : 0));
+ byte c4 = (byte) ((c2 << 1) ^ ((c2 & 0x80) != 0 ? 0x1d : 0));
+ byte c8 = (byte) ((c4 << 1) ^ ((c4 & 0x80) != 0 ? 0x1d : 0));
+
+ byte c3, c5, c6, c7, c9, c10, c11, c12, c13, c14, c15;
+ byte c17, c18, c19, c20, c21, c22, c23, c24, c25, c26,
+ c27, c28, c29, c30, c31;
+
+ c3 = (byte) (c2 ^ c);
+ c5 = (byte) (c4 ^ c);
+ c6 = (byte) (c4 ^ c2);
+ c7 = (byte) (c4 ^ c3);
+
+ c9 = (byte) (c8 ^ c);
+ c10 = (byte) (c8 ^ c2);
+ c11 = (byte) (c8 ^ c3);
+ c12 = (byte) (c8 ^ c4);
+ c13 = (byte) (c8 ^ c5);
+ c14 = (byte) (c8 ^ c6);
+ c15 = (byte) (c8 ^ c7);
+
+ tbl[offset + 0] = 0;
+ tbl[offset + 1] = c;
+ tbl[offset + 2] = c2;
+ tbl[offset + 3] = c3;
+ tbl[offset + 4] = c4;
+ tbl[offset + 5] = c5;
+ tbl[offset + 6] = c6;
+ tbl[offset + 7] = c7;
+ tbl[offset + 8] = c8;
+ tbl[offset + 9] = c9;
+ tbl[offset + 10] = c10;
+ tbl[offset + 11] = c11;
+ tbl[offset + 12] = c12;
+ tbl[offset + 13] = c13;
+ tbl[offset + 14] = c14;
+ tbl[offset + 15] = c15;
+
+ c17 = (byte) ((c8 << 1) ^ ((c8 & 0x80) != 0 ? 0x1d : 0));
+ c18 = (byte) ((c17 << 1) ^ ((c17 & 0x80) != 0 ? 0x1d : 0));
+ c19 = (byte) (c18 ^ c17);
+ c20 = (byte) ((c18 << 1) ^ ((c18 & 0x80) != 0 ? 0x1d : 0));
+ c21 = (byte) (c20 ^ c17);
+ c22 = (byte) (c20 ^ c18);
+ c23 = (byte) (c20 ^ c19);
+ c24 = (byte) ((c20 << 1) ^ ((c20 & 0x80) != 0 ? 0x1d : 0));
+ c25 = (byte) (c24 ^ c17);
+ c26 = (byte) (c24 ^ c18);
+ c27 = (byte) (c24 ^ c19);
+ c28 = (byte) (c24 ^ c20);
+ c29 = (byte) (c24 ^ c21);
+ c30 = (byte) (c24 ^ c22);
+ c31 = (byte) (c24 ^ c23);
+
+ tbl[offset + 16] = 0;
+ tbl[offset + 17] = c17;
+ tbl[offset + 18] = c18;
+ tbl[offset + 19] = c19;
+ tbl[offset + 20] = c20;
+ tbl[offset + 21] = c21;
+ tbl[offset + 22] = c22;
+ tbl[offset + 23] = c23;
+ tbl[offset + 24] = c24;
+ tbl[offset + 25] = c25;
+ tbl[offset + 26] = c26;
+ tbl[offset + 27] = c27;
+ tbl[offset + 28] = c28;
+ tbl[offset + 29] = c29;
+ tbl[offset + 30] = c30;
+ tbl[offset + 31] = c31;
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GaloisField.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GaloisField.java
new file mode 100644
index 0000000..7fb06ce
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/GaloisField.java
@@ -0,0 +1,565 @@
+/**
+ * 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.ozone.erasurecode.rawcoder.util;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Implementation of Galois field arithmetic with 2^p elements. The input must
+ * be unsigned integers. It's ported from HDFS-RAID, slightly adapted.
+ */
+@InterfaceAudience.Private
+public final class GaloisField {
+
+ // Field size 256 is good for byte based system
+ private static final int DEFAULT_FIELD_SIZE = 256;
+ // primitive polynomial 1 + X^2 + X^3 + X^4 + X^8 (substitute 2)
+ private static final int DEFAULT_PRIMITIVE_POLYNOMIAL = 285;
+ private static final Map<Integer, GaloisField> INSTANCES =
+ new HashMap<>();
+ private final int[] logTable;
+ private final int[] powTable;
+ private final int[][] mulTable;
+ private final int[][] divTable;
+ private final int fieldSize;
+ private final int primitivePeriod;
+ private final int primitivePolynomial;
+
+ private GaloisField(int fieldSize, int primitivePolynomial) {
+ assert fieldSize > 0;
+ assert primitivePolynomial > 0;
+
+ this.fieldSize = fieldSize;
+ this.primitivePeriod = fieldSize - 1;
+ this.primitivePolynomial = primitivePolynomial;
+ logTable = new int[fieldSize];
+ powTable = new int[fieldSize];
+ mulTable = new int[fieldSize][fieldSize];
+ divTable = new int[fieldSize][fieldSize];
+ int value = 1;
+ for (int pow = 0; pow < fieldSize - 1; pow++) {
+ powTable[pow] = value;
+ logTable[value] = pow;
+ value = value * 2;
+ if (value >= fieldSize) {
+ value = value ^ primitivePolynomial;
+ }
+ }
+ // building multiplication table
+ for (int i = 0; i < fieldSize; i++) {
+ for (int j = 0; j < fieldSize; j++) {
+ if (i == 0 || j == 0) {
+ mulTable[i][j] = 0;
+ continue;
+ }
+ int z = logTable[i] + logTable[j];
+ z = z >= primitivePeriod ? z - primitivePeriod : z;
+ z = powTable[z];
+ mulTable[i][j] = z;
+ }
+ }
+ // building division table
+ for (int i = 0; i < fieldSize; i++) {
+ for (int j = 1; j < fieldSize; j++) {
+ if (i == 0) {
+ divTable[i][j] = 0;
+ continue;
+ }
+ int z = logTable[i] - logTable[j];
+ z = z < 0 ? z + primitivePeriod : z;
+ z = powTable[z];
+ divTable[i][j] = z;
+ }
+ }
+ }
+
+ /**
+ * Get the object performs Galois field arithmetics.
+ *
+ * @param fieldSize size of the field
+ * @param primitivePolynomial a primitive polynomial corresponds to the size
+ */
+ public static GaloisField getInstance(int fieldSize,
+ int primitivePolynomial) {
+ int key = ((fieldSize << 16) & 0xFFFF0000)
+ + (primitivePolynomial & 0x0000FFFF);
+ GaloisField gf;
+ synchronized (INSTANCES) {
+ gf = INSTANCES.get(key);
+ if (gf == null) {
+ gf = new GaloisField(fieldSize, primitivePolynomial);
+ INSTANCES.put(key, gf);
+ }
+ }
+ return gf;
+ }
+
+ /**
+ * Get the object performs Galois field arithmetic with default setting.
+ */
+ public static GaloisField getInstance() {
+ return getInstance(DEFAULT_FIELD_SIZE, DEFAULT_PRIMITIVE_POLYNOMIAL);
+ }
+
+ /**
+ * Return number of elements in the field.
+ *
+ * @return number of elements in the field
+ */
+ public int getFieldSize() {
+ return fieldSize;
+ }
+
+ /**
+ * Return the primitive polynomial in GF(2).
+ *
+ * @return primitive polynomial as a integer
+ */
+ public int getPrimitivePolynomial() {
+ return primitivePolynomial;
+ }
+
+ /**
+ * Compute the sum of two fields.
+ *
+ * @param x input field
+ * @param y input field
+ * @return result of addition
+ */
+ public int add(int x, int y) {
+ assert (x >= 0 && x < getFieldSize() && y >= 0 && y < getFieldSize());
+ return x ^ y;
+ }
+
+ /**
+ * Compute the multiplication of two fields.
+ *
+ * @param x input field
+ * @param y input field
+ * @return result of multiplication
+ */
+ public int multiply(int x, int y) {
+ assert (x >= 0 && x < getFieldSize() && y >= 0 && y < getFieldSize());
+ return mulTable[x][y];
+ }
+
+ /**
+ * Compute the division of two fields.
+ *
+ * @param x input field
+ * @param y input field
+ * @return x/y
+ */
+ public int divide(int x, int y) {
+ assert (x >= 0 && x < getFieldSize() && y > 0 && y < getFieldSize());
+ return divTable[x][y];
+ }
+
+ /**
+ * Compute power n of a field.
+ *
+ * @param x input field
+ * @param n power
+ * @return x^n
+ */
+ public int power(int x, int n) {
+ assert (x >= 0 && x < getFieldSize());
+ if (n == 0) {
+ return 1;
+ }
+ if (x == 0) {
+ return 0;
+ }
+ x = logTable[x] * n;
+ if (x < primitivePeriod) {
+ return powTable[x];
+ }
+ x = x % primitivePeriod;
+ return powTable[x];
+ }
+
+ /**
+ * Given a Vandermonde matrix V[i][j]=x[j]^i and vector y, solve for z such
+ * that Vz=y. The output z will be placed in y.
+ *
+ * @param x the vector which describe the Vandermonde matrix
+ * @param y right-hand side of the Vandermonde system equation. will be
+ * replaced the output in this vector
+ */
+ public void solveVandermondeSystem(int[] x, int[] y) {
+ solveVandermondeSystem(x, y, x.length);
+ }
+
+ /**
+ * Given a Vandermonde matrix V[i][j]=x[j]^i and vector y, solve for z such
+ * that Vz=y. The output z will be placed in y.
+ *
+ * @param x the vector which describe the Vandermonde matrix
+ * @param y right-hand side of the Vandermonde system equation. will be
+ * replaced the output in this vector
+ * @param len consider x and y only from 0...len-1
+ */
+ public void solveVandermondeSystem(int[] x, int[] y, int len) {
+ assert (x.length <= len && y.length <= len);
+ for (int i = 0; i < len - 1; i++) {
+ for (int j = len - 1; j > i; j--) {
+ y[j] = y[j] ^ mulTable[x[i]][y[j - 1]];
+ }
+ }
+ for (int i = len - 1; i >= 0; i--) {
+ for (int j = i + 1; j < len; j++) {
+ y[j] = divTable[y[j]][x[j] ^ x[j - i - 1]];
+ }
+ for (int j = i; j < len - 1; j++) {
+ y[j] = y[j] ^ y[j + 1];
+ }
+ }
+ }
+
+ /**
+ * A "bulk" version to the solving of Vandermonde System.
+ */
+ public void solveVandermondeSystem(int[] x, byte[][] y, int[] outputOffsets,
+ int len, int dataLen) {
+ int idx1, idx2;
+ for (int i = 0; i < len - 1; i++) {
+ for (int j = len - 1; j > i; j--) {
+ for (idx2 = outputOffsets[j - 1], idx1 =
+ outputOffsets[j]; idx1 < outputOffsets[j] + dataLen;
+ idx1++, idx2++) {
+ y[j][idx1] = (byte) (y[j][idx1] ^ mulTable[x[i]][y[j - 1][idx2] &
+ 0x000000FF]);
+ }
+ }
+ }
+ for (int i = len - 1; i >= 0; i--) {
+ for (int j = i + 1; j < len; j++) {
+ for (idx1 = outputOffsets[j];
+ idx1 < outputOffsets[j] + dataLen; idx1++) {
+ y[j][idx1] = (byte) (divTable[y[j][idx1] & 0x000000FF][x[j] ^
+ x[j - i - 1]]);
+ }
+ }
+ for (int j = i; j < len - 1; j++) {
+ for (idx2 = outputOffsets[j + 1], idx1 = outputOffsets[j];
+ idx1 < outputOffsets[j] + dataLen; idx1++, idx2++) {
+ y[j][idx1] = (byte) (y[j][idx1] ^ y[j + 1][idx2]);
+ }
+ }
+ }
+ }
+
+ /**
+ * A "bulk" version of the solveVandermondeSystem, using ByteBuffer.
+ */
+ public void solveVandermondeSystem(int[] x, ByteBuffer[] y, int len) {
+ ByteBuffer p;
+ int idx1, idx2;
+ for (int i = 0; i < len - 1; i++) {
+ for (int j = len - 1; j > i; j--) {
+ p = y[j];
+ for (idx1 = p.position(), idx2 = y[j - 1].position(); idx1 < p
+ .limit(); idx1++, idx2++) {
+ p.put(idx1, (byte) (p.get(idx1) ^ mulTable[x[i]][y[j - 1].get(idx2) &
+ 0x000000FF]));
+ }
+ }
+ }
+
+ for (int i = len - 1; i >= 0; i--) {
+ for (int j = i + 1; j < len; j++) {
+ p = y[j];
+ for (idx1 = p.position(); idx1 < p.limit(); idx1++) {
+ p.put(idx1, (byte) (divTable[p.get(idx1) &
+ 0x000000FF][x[j] ^ x[j - i - 1]]));
+ }
+ }
+
+ for (int j = i; j < len - 1; j++) {
+ p = y[j];
+ for (idx1 = p.position(), idx2 = y[j + 1].position(); idx1 < p
+ .limit(); idx1++, idx2++) {
+ p.put(idx1, (byte) (p.get(idx1) ^ y[j + 1].get(idx2)));
+ }
+ }
+ }
+ }
+
+ /**
+ * Compute the multiplication of two polynomials. The index in the array
+ * corresponds to the power of the entry. For example p[0] is the constant
+ * term of the polynomial p.
+ *
+ * @param p input polynomial
+ * @param q input polynomial
+ * @return polynomial represents p*q
+ */
+ public int[] multiply(int[] p, int[] q) {
+ int len = p.length + q.length - 1;
+ int[] result = new int[len];
+ for (int i = 0; i < len; i++) {
+ result[i] = 0;
+ }
+ for (int i = 0; i < p.length; i++) {
+
+ for (int j = 0; j < q.length; j++) {
+ result[i + j] = add(result[i + j], multiply(p[i], q[j]));
+ }
+ }
+ return result;
+ }
+
+ /**
+ * Compute the remainder of a dividend and divisor pair. The index in the
+ * array corresponds to the power of the entry. For example p[0] is the
+ * constant term of the polynomial p.
+ *
+ * @param dividend dividend polynomial, the remainder will be placed
+ * here when return
+ * @param divisor divisor polynomial
+ */
+ public void remainder(int[] dividend, int[] divisor) {
+ for (int i = dividend.length - divisor.length; i >= 0; i--) {
+ int ratio = divTable[dividend[i +
+ divisor.length - 1]][divisor[divisor.length - 1]];
+ for (int j = 0; j < divisor.length; j++) {
+ int k = j + i;
+ dividend[k] = dividend[k] ^ mulTable[ratio][divisor[j]];
+ }
+ }
+ }
+
+ /**
+ * Compute the sum of two polynomials. The index in the array corresponds to
+ * the power of the entry. For example p[0] is the constant term of the
+ * polynomial p.
+ *
+ * @param p input polynomial
+ * @param q input polynomial
+ * @return polynomial represents p+q
+ */
+ public int[] add(int[] p, int[] q) {
+ int len = Math.max(p.length, q.length);
+ int[] result = new int[len];
+ for (int i = 0; i < len; i++) {
+ if (i < p.length && i < q.length) {
+ result[i] = add(p[i], q[i]);
+ } else if (i < p.length) {
+ result[i] = p[i];
+ } else {
+ result[i] = q[i];
+ }
+ }
+ return result;
+ }
+
+ /**
+ * Substitute x into polynomial p(x).
+ *
+ * @param p input polynomial
+ * @param x input field
+ * @return p(x)
+ */
+ public int substitute(int[] p, int x) {
+ int result = 0;
+ int y = 1;
+ for (int i = 0; i < p.length; i++) {
+ result = result ^ mulTable[p[i]][y];
+ y = mulTable[x][y];
+ }
+ return result;
+ }
+
+ /**
+ * A "bulk" version of the substitute.
+ * Tends to be 2X faster than the "int" substitute in a loop.
+ *
+ * @param p input polynomial
+ * @param q store the return result
+ * @param x input field
+ */
+ public void substitute(byte[][] p, byte[] q, int x) {
+ int y = 1;
+ for (int i = 0; i < p.length; i++) {
+ byte[] pi = p[i];
+ for (int j = 0; j < pi.length; j++) {
+ int pij = pi[j] & 0x000000FF;
+ q[j] = (byte) (q[j] ^ mulTable[pij][y]);
+ }
+ y = mulTable[x][y];
+ }
+ }
+
+ /**
+ * A "bulk" version of the substitute.
+ * Tends to be 2X faster than the "int" substitute in a loop.
+ *
+ * @param p input polynomial
+ * @param offsets
+ * @param len
+ * @param q store the return result
+ * @param offset
+ * @param x input field
+ */
+ public void substitute(byte[][] p, int[] offsets,
+ int len, byte[] q, int offset, int x) {
+ int y = 1, iIdx, oIdx;
+ for (int i = 0; i < p.length; i++) {
+ byte[] pi = p[i];
+ for (iIdx = offsets[i], oIdx = offset;
+ iIdx < offsets[i] + len; iIdx++, oIdx++) {
+ int pij = pi != null ? pi[iIdx] & 0x000000FF : 0;
+ q[oIdx] = (byte) (q[oIdx] ^ mulTable[pij][y]);
+ }
+ y = mulTable[x][y];
+ }
+ }
+
+ /**
+ * A "bulk" version of the substitute, using ByteBuffer.
+ * Tends to be 2X faster than the "int" substitute in a loop.
+ *
+ * @param p input polynomial
+ * @param q store the return result
+ * @param x input field
+ */
+ public void substitute(ByteBuffer[] p, int len, ByteBuffer q, int x) {
+ int y = 1, iIdx, oIdx;
+ for (int i = 0; i < p.length; i++) {
+ ByteBuffer pi = p[i];
+ int pos = pi != null ? pi.position() : 0;
+ int limit = pi != null ? pi.limit() : len;
+ for (oIdx = q.position(), iIdx = pos;
+ iIdx < limit; iIdx++, oIdx++) {
+ int pij = pi != null ? pi.get(iIdx) & 0x000000FF : 0;
+ q.put(oIdx, (byte) (q.get(oIdx) ^ mulTable[pij][y]));
+ }
+ y = mulTable[x][y];
+ }
+ }
+
+ /**
+ * The "bulk" version of the remainder.
+ * Warning: This function will modify the "dividend" inputs.
+ */
+ public void remainder(byte[][] dividend, int[] divisor) {
+ for (int i = dividend.length - divisor.length; i >= 0; i--) {
+ for (int j = 0; j < divisor.length; j++) {
+ for (int k = 0; k < dividend[i].length; k++) {
+ int ratio = divTable[dividend[i + divisor.length - 1][k] &
+ 0x00FF][divisor[divisor.length - 1]];
+ dividend[j + i][k] = (byte) ((dividend[j + i][k] & 0x00FF) ^
+ mulTable[ratio][divisor[j]]);
+ }
+ }
+ }
+ }
+
+ /**
+ * The "bulk" version of the remainder.
+ * Warning: This function will modify the "dividend" inputs.
+ */
+ public void remainder(byte[][] dividend, int[] offsets,
+ int len, int[] divisor) {
+ int idx1, idx2;
+ for (int i = dividend.length - divisor.length; i >= 0; i--) {
+ for (int j = 0; j < divisor.length; j++) {
+ for (idx2 = offsets[j + i], idx1 = offsets[i + divisor.length - 1];
+ idx1 < offsets[i + divisor.length - 1] + len;
+ idx1++, idx2++) {
+ int ratio = divTable[dividend[i + divisor.length - 1][idx1] &
+ 0x00FF][divisor[divisor.length - 1]];
+ dividend[j + i][idx2] = (byte) ((dividend[j + i][idx2] & 0x00FF) ^
+ mulTable[ratio][divisor[j]]);
+ }
+ }
+ }
+ }
+
+ /**
+ * The "bulk" version of the remainder, using ByteBuffer.
+ * Warning: This function will modify the "dividend" inputs.
+ */
+ public void remainder(ByteBuffer[] dividend, int[] divisor) {
+ int idx1, idx2;
+ ByteBuffer b1, b2;
+ for (int i = dividend.length - divisor.length; i >= 0; i--) {
+ for (int j = 0; j < divisor.length; j++) {
+ b1 = dividend[i + divisor.length - 1];
+ b2 = dividend[j + i];
+ for (idx1 = b1.position(), idx2 = b2.position();
+ idx1 < b1.limit(); idx1++, idx2++) {
+ int ratio = divTable[b1.get(idx1) &
+ 0x00FF][divisor[divisor.length - 1]];
+ b2.put(idx2, (byte) ((b2.get(idx2) & 0x00FF) ^
+ mulTable[ratio][divisor[j]]));
+ }
+ }
+ }
+ }
+
+ /**
+ * Perform Gaussian elimination on the given matrix. This matrix has to be a
+ * fat matrix (number of rows > number of columns).
+ */
+ public void gaussianElimination(int[][] matrix) {
+ assert (matrix != null && matrix.length > 0 && matrix[0].length > 0
+ && matrix.length < matrix[0].length);
+ int height = matrix.length;
+ int width = matrix[0].length;
+ for (int i = 0; i < height; i++) {
+ boolean pivotFound = false;
+ // scan the column for a nonzero pivot and swap it to the diagonal
+ for (int j = i; j < height; j++) {
+ if (matrix[i][j] != 0) {
+ int[] tmp = matrix[i];
+ matrix[i] = matrix[j];
+ matrix[j] = tmp;
+ pivotFound = true;
+ break;
+ }
+ }
+ if (!pivotFound) {
+ continue;
+ }
+ int pivot = matrix[i][i];
+ for (int j = i; j < width; j++) {
+ matrix[i][j] = divide(matrix[i][j], pivot);
+ }
+ for (int j = i + 1; j < height; j++) {
+ int lead = matrix[j][i];
+ for (int k = i; k < width; k++) {
+ matrix[j][k] = add(matrix[j][k], multiply(lead, matrix[i][k]));
+ }
+ }
+ }
+ for (int i = height - 1; i >= 0; i--) {
+ for (int j = 0; j < i; j++) {
+ int lead = matrix[j][i];
+ for (int k = i; k < width; k++) {
+ matrix[j][k] = add(matrix[j][k], multiply(lead, matrix[i][k]));
+ }
+ }
+ }
+ }
+
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/RSUtil.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/RSUtil.java
new file mode 100644
index 0000000..0ec88d0
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/RSUtil.java
@@ -0,0 +1,188 @@
+/**
+ * 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.ozone.erasurecode.rawcoder.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Utilities for implementing Reed-Solomon code, used by RS coder. Some of the
+ * codes are borrowed from ISA-L implementation (C or ASM codes).
+ */
+@InterfaceAudience.Private
+public final class RSUtil {
+
+ private RSUtil() {
+ }
+
+ // We always use the byte system (with symbol size 8, field size 256,
+ // primitive polynomial 285, and primitive root 2).
+ public static final GaloisField GF = GaloisField.getInstance();
+ public static final int PRIMITIVE_ROOT = 2;
+
+ public static int[] getPrimitivePower(int numDataUnits, int numParityUnits) {
+ int[] primitivePower = new int[numDataUnits + numParityUnits];
+ // compute powers of the primitive root
+ for (int i = 0; i < numDataUnits + numParityUnits; i++) {
+ primitivePower[i] = GF.power(PRIMITIVE_ROOT, i);
+ }
+ return primitivePower;
+ }
+
+ public static void initTables(int k, int rows, byte[] codingMatrix,
+ int matrixOffset, byte[] gfTables) {
+ int i, j;
+
+ int offset = 0, idx = matrixOffset;
+ for (i = 0; i < rows; i++) {
+ for (j = 0; j < k; j++) {
+ GF256.gfVectMulInit(codingMatrix[idx++], gfTables, offset);
+ offset += 32;
+ }
+ }
+ }
+
+ /**
+ * Ported from Intel ISA-L library.
+ */
+ public static void genCauchyMatrix(byte[] a, int m, int k) {
+ // Identity matrix in high position
+ for (int i = 0; i < k; i++) {
+ a[k * i + i] = 1;
+ }
+
+ // For the rest choose 1/(i + j) | i != j
+ int pos = k * k;
+ for (int i = k; i < m; i++) {
+ for (int j = 0; j < k; j++) {
+ a[pos++] = GF256.gfInv((byte) (i ^ j));
+ }
+ }
+ }
+
+ /**
+ * Encode a group of inputs data and generate the outputs. It's also used for
+ * decoding because, in this implementation, encoding and decoding are
+ * unified.
+ *
+ * The algorithm is ported from Intel ISA-L library for compatible. It
+ * leverages Java auto-vectorization support for performance.
+ */
+ public static void encodeData(byte[] gfTables, int dataLen, byte[][] inputs,
+ int[] inputOffsets, byte[][] outputs,
+ int[] outputOffsets) {
+ int numInputs = inputs.length;
+ int numOutputs = outputs.length;
+ int l, i, j, iPos, oPos;
+ byte[] input, output;
+ byte s;
+ final int times = dataLen / 8;
+ final int extra = dataLen - dataLen % 8;
+ byte[] tableLine;
+
+ for (l = 0; l < numOutputs; l++) {
+ output = outputs[l];
+
+ for (j = 0; j < numInputs; j++) {
+ input = inputs[j];
+ iPos = inputOffsets[j];
+ oPos = outputOffsets[l];
+
+ s = gfTables[j * 32 + l * numInputs * 32 + 1];
+ tableLine = GF256.gfMulTab()[s & 0xff];
+
+ /**
+ * Purely for performance, assuming we can use 8 bytes in the SIMD
+ * instruction. Subject to be improved.
+ */
+ for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
+ output[oPos + 0] ^= tableLine[0xff & input[iPos + 0]];
+ output[oPos + 1] ^= tableLine[0xff & input[iPos + 1]];
+ output[oPos + 2] ^= tableLine[0xff & input[iPos + 2]];
+ output[oPos + 3] ^= tableLine[0xff & input[iPos + 3]];
+ output[oPos + 4] ^= tableLine[0xff & input[iPos + 4]];
+ output[oPos + 5] ^= tableLine[0xff & input[iPos + 5]];
+ output[oPos + 6] ^= tableLine[0xff & input[iPos + 6]];
+ output[oPos + 7] ^= tableLine[0xff & input[iPos + 7]];
+ }
+
+ /**
+ * For the left bytes, do it one by one.
+ */
+ for (i = extra; i < dataLen; i++, iPos++, oPos++) {
+ output[oPos] ^= tableLine[0xff & input[iPos]];
+ }
+ }
+ }
+ }
+
+ /**
+ * See above. Try to use the byte[] version when possible.
+ */
+ public static void encodeData(byte[] gfTables, ByteBuffer[] inputs,
+ ByteBuffer[] outputs) {
+ int numInputs = inputs.length;
+ int numOutputs = outputs.length;
+ int dataLen = inputs[0].remaining();
+ int l, i, j, iPos, oPos;
+ ByteBuffer input, output;
+ byte s;
+ final int times = dataLen / 8;
+ final int extra = dataLen - dataLen % 8;
+ byte[] tableLine;
+
+ for (l = 0; l < numOutputs; l++) {
+ output = outputs[l];
+
+ for (j = 0; j < numInputs; j++) {
+ input = inputs[j];
+ iPos = input.position();
+ oPos = output.position();
+
+ s = gfTables[j * 32 + l * numInputs * 32 + 1];
+ tableLine = GF256.gfMulTab()[s & 0xff];
+
+ for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
+ output.put(oPos + 0, (byte) (output.get(oPos + 0) ^
+ tableLine[0xff & input.get(iPos + 0)]));
+ output.put(oPos + 1, (byte) (output.get(oPos + 1) ^
+ tableLine[0xff & input.get(iPos + 1)]));
+ output.put(oPos + 2, (byte) (output.get(oPos + 2) ^
+ tableLine[0xff & input.get(iPos + 2)]));
+ output.put(oPos + 3, (byte) (output.get(oPos + 3) ^
+ tableLine[0xff & input.get(iPos + 3)]));
+ output.put(oPos + 4, (byte) (output.get(oPos + 4) ^
+ tableLine[0xff & input.get(iPos + 4)]));
+ output.put(oPos + 5, (byte) (output.get(oPos + 5) ^
+ tableLine[0xff & input.get(iPos + 5)]));
+ output.put(oPos + 6, (byte) (output.get(oPos + 6) ^
+ tableLine[0xff & input.get(iPos + 6)]));
+ output.put(oPos + 7, (byte) (output.get(oPos + 7) ^
+ tableLine[0xff & input.get(iPos + 7)]));
+ }
+
+ for (i = extra; i < dataLen; i++, iPos++, oPos++) {
+ output.put(oPos, (byte) (output.get(oPos) ^
+ tableLine[0xff & input.get(iPos)]));
+ }
+ }
+ }
+ }
+
+}
diff --git a/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/package-info.java b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/package-info.java
new file mode 100644
index 0000000..5e6fac4
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/java/org/apache/ozone/erasurecode/rawcoder/util/package-info.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.
+ */
+
+/**
+ * General helpers for implementing raw erasure coders.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.ozone.erasurecode.rawcoder.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/main/resources/META-INF/services/org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory b/hadoop-hdds/erasurecode/src/main/resources/META-INF/services/org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory
new file mode 100644
index 0000000..24625b5
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/main/resources/META-INF/services/org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory
@@ -0,0 +1,17 @@
+#
+# Licensed 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.ozone.erasurecode.rawcoder.RSRawErasureCoderFactory
+org.apache.ozone.erasurecode.rawcoder.XORRawErasureCoderFactory
+org.apache.ozone.erasurecode.rawcoder.NativeRSRawErasureCoderFactory
+org.apache.ozone.erasurecode.rawcoder.NativeXORRawErasureCoderFactory
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/DumpUtil.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/DumpUtil.java
new file mode 100644
index 0000000..fbbf4f6
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/DumpUtil.java
@@ -0,0 +1,96 @@
+/**
+ * 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.ozone.erasurecode;
+
+
+/**
+ * A dump utility class for debugging data erasure coding/decoding issues.
+ * Don't suggest they are used in runtime production codes.
+ */
+public final class DumpUtil {
+ private static final String HEX_CHARS_STR = "0123456789ABCDEF";
+ private static final char[] HEX_CHARS = HEX_CHARS_STR.toCharArray();
+
+ private DumpUtil() {
+ // No called
+ }
+
+ /**
+ * Convert bytes into format like 0x02 02 00 80.
+ * If limit is negative or too large, then all bytes will be converted.
+ */
+ public static String bytesToHex(byte[] bytes, int limit) {
+ if (limit <= 0 || limit > bytes.length) {
+ limit = bytes.length;
+ }
+ int len = limit * 2;
+ len += limit; // for ' ' appended for each char
+ len += 2; // for '0x' prefix
+ char[] hexChars = new char[len];
+ hexChars[0] = '0';
+ hexChars[1] = 'x';
+ for (int j = 0; j < limit; j++) {
+ int v = bytes[j] & 0xFF;
+ hexChars[j * 3 + 2] = HEX_CHARS[v >>> 4];
+ hexChars[j * 3 + 3] = HEX_CHARS[v & 0x0F];
+ hexChars[j * 3 + 4] = ' ';
+ }
+
+ return new String(hexChars);
+ }
+
+ public static void dumpMatrix(byte[] matrix,
+ int numDataUnits, int numAllUnits) {
+ for (int i = 0; i < numDataUnits; i++) {
+ for (int j = 0; j < numAllUnits; j++) {
+ System.out.print(" ");
+ System.out.print(0xff & matrix[i * numAllUnits + j]);
+ }
+ System.out.println();
+ }
+ }
+
+ /**
+ * Print data in hex format in an array of chunks.
+ * @param header
+ * @param chunks
+ */
+ public static void dumpChunks(String header, ECChunk[] chunks) {
+ System.out.println();
+ System.out.println(header);
+ for (int i = 0; i < chunks.length; i++) {
+ dumpChunk(chunks[i]);
+ }
+ System.out.println();
+ }
+
+ /**
+ * Print data in hex format in a chunk.
+ * @param chunk
+ */
+ public static void dumpChunk(ECChunk chunk) {
+ String str;
+ if (chunk == null) {
+ str = "<EMPTY>";
+ } else {
+ byte[] bytes = chunk.toBytesArray();
+ str = DumpUtil.bytesToHex(bytes, 16);
+ }
+ System.out.println(str);
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCodecRegistry.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCodecRegistry.java
new file mode 100644
index 0000000..efc3a9f
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCodecRegistry.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.NativeXORRawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.RSRawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.ozone.erasurecode.rawcoder.XORRawErasureCoderFactory;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test CodecRegistry.
+ */
+public class TestCodecRegistry {
+
+ @Test
+ public void testGetCodecs() {
+ Set<String> codecs = CodecRegistry.getInstance().getCodecNames();
+ assertEquals(2, codecs.size());
+ assertTrue(
+ codecs.contains(ECReplicationConfig.EcCodec.RS.name().toLowerCase()));
+ assertTrue(
+ codecs.contains(ECReplicationConfig.EcCodec.XOR.name().toLowerCase()));
+ }
+
+ @Test
+ public void testGetCoders() {
+ List<RawErasureCoderFactory> coders = CodecRegistry.getInstance().
+ getCoders(ECReplicationConfig.EcCodec.RS.name().toLowerCase());
+ assertEquals(2, coders.size());
+ assertTrue(coders.get(0) instanceof NativeRSRawErasureCoderFactory);
+ assertTrue(coders.get(1) instanceof RSRawErasureCoderFactory);
+
+ coders = CodecRegistry.getInstance().
+ getCoders(ECReplicationConfig.EcCodec.XOR.name().toLowerCase());
+ assertEquals(2, coders.size());
+ assertTrue(coders.get(0) instanceof NativeXORRawErasureCoderFactory);
+ assertTrue(coders.get(1) instanceof XORRawErasureCoderFactory);
+ }
+
+ @Test
+ public void testGetCodersWrong() {
+ List<RawErasureCoderFactory> coders =
+ CodecRegistry.getInstance().getCoders("WRONG_CODEC");
+ assertNull(coders);
+ }
+
+ @Test
+ public void testGetCoderByNameWrong() {
+ RawErasureCoderFactory coder = CodecRegistry.getInstance().
+ getCoderByName(ECReplicationConfig.EcCodec.RS.name().toLowerCase(),
+ "WRONG_RS");
+ assertNull(coder);
+ }
+
+ @Test
+ public void testUpdateCoders() {
+ class RSUserDefinedIncorrectFactory implements RawErasureCoderFactory {
+ public RawErasureEncoder createEncoder(ECReplicationConfig coderOptions) {
+ return null;
+ }
+
+ public RawErasureDecoder createDecoder(ECReplicationConfig coderOptions) {
+ return null;
+ }
+
+ public String getCoderName() {
+ return "rs_java";
+ }
+
+ public String getCodecName() {
+ return ECReplicationConfig.EcCodec.RS.name().toLowerCase();
+ }
+ }
+
+ List<RawErasureCoderFactory> userDefinedFactories = new ArrayList<>();
+ userDefinedFactories.add(new RSUserDefinedIncorrectFactory());
+ CodecRegistry.getInstance().updateCoders(userDefinedFactories);
+
+ // check RS coders
+ List<RawErasureCoderFactory> rsCoders = CodecRegistry.getInstance().
+ getCoders(ECReplicationConfig.EcCodec.RS.name().toLowerCase());
+ assertEquals(2, rsCoders.size());
+ assertTrue(rsCoders.get(0) instanceof NativeRSRawErasureCoderFactory);
+ assertTrue(rsCoders.get(1) instanceof RSRawErasureCoderFactory);
+
+ // check RS coder names
+ String[] rsCoderNames = CodecRegistry.getInstance().
+ getCoderNames(ECReplicationConfig.EcCodec.RS.name().toLowerCase());
+ assertEquals(2, rsCoderNames.length);
+ assertEquals(NativeRSRawErasureCoderFactory.CODER_NAME, rsCoderNames[0]);
+ assertEquals(RSRawErasureCoderFactory.CODER_NAME, rsCoderNames[1]);
+ }
+
+ @Test
+ public void testGetCoderNames() {
+ String[] coderNames = CodecRegistry.getInstance().
+ getCoderNames(ECReplicationConfig.EcCodec.RS.name().toLowerCase());
+ assertEquals(2, coderNames.length);
+ assertEquals(NativeRSRawErasureCoderFactory.CODER_NAME, coderNames[0]);
+ assertEquals(RSRawErasureCoderFactory.CODER_NAME, coderNames[1]);
+
+ coderNames = CodecRegistry.getInstance().
+ getCoderNames(ECReplicationConfig.EcCodec.XOR.name().toLowerCase());
+ assertEquals(2, coderNames.length);
+ assertEquals(NativeXORRawErasureCoderFactory.CODER_NAME, coderNames[0]);
+ assertEquals(XORRawErasureCoderFactory.CODER_NAME, coderNames[1]);
+ }
+
+ @Test
+ public void testGetCoderByName() {
+ RawErasureCoderFactory coder = CodecRegistry.getInstance().
+ getCoderByName(ECReplicationConfig.EcCodec.RS.name().toLowerCase(),
+ RSRawErasureCoderFactory.CODER_NAME);
+ assertTrue(coder instanceof RSRawErasureCoderFactory);
+
+ coder = CodecRegistry.getInstance()
+ .getCoderByName(ECReplicationConfig.EcCodec.RS.name().toLowerCase(),
+ NativeRSRawErasureCoderFactory.CODER_NAME);
+ assertTrue(coder instanceof NativeRSRawErasureCoderFactory);
+
+ coder = CodecRegistry.getInstance()
+ .getCoderByName(ECReplicationConfig.EcCodec.XOR.name().toLowerCase(),
+ XORRawErasureCoderFactory.CODER_NAME);
+ assertTrue(coder instanceof XORRawErasureCoderFactory);
+
+ coder = CodecRegistry.getInstance()
+ .getCoderByName(ECReplicationConfig.EcCodec.XOR.name().toLowerCase(),
+ NativeXORRawErasureCoderFactory.CODER_NAME);
+ assertTrue(coder instanceof NativeXORRawErasureCoderFactory);
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCoderBase.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCoderBase.java
new file mode 100644
index 0000000..e4b02cd
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/TestCoderBase.java
@@ -0,0 +1,524 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test base of common utilities for tests not only raw coders but also block
+ * coders.
+ */
+@SuppressWarnings({"checkstyle:VisibilityModifier", "checkstyle:HiddenField"})
+public abstract class TestCoderBase {
+ protected static final Random RAND = new Random();
+ private static int fixedDataGenerator = 0;
+ protected boolean allowDump = true;
+ protected int numDataUnits;
+ protected int numParityUnits;
+ protected int baseChunkSize = 1024;
+ // Indexes of erased data units.
+ protected int[] erasedDataIndexes = new int[]{0};
+ // Indexes of erased parity units.
+ protected int[] erasedParityIndexes = new int[]{0};
+ // Data buffers are either direct or on-heap, for performance the two cases
+ // may go to different coding implementations.
+ protected boolean usingDirectBuffer = true;
+ protected boolean usingFixedData = true;
+ protected byte[][] fixedData;
+ private int chunkSize = baseChunkSize;
+ private BufferAllocator allocator;
+ private byte[] zeroChunkBytes;
+ private boolean startBufferWithZero = true;
+ // Using this the generated data can be repeatable across multiple calls to
+ // encode(), in order for troubleshooting.
+ private ConfigurationSource conf;
+
+ protected int getChunkSize() {
+ return chunkSize;
+ }
+
+ protected void setChunkSize(int chunkSize) {
+ this.chunkSize = chunkSize;
+ this.zeroChunkBytes = new byte[chunkSize]; // With ZERO by default
+ }
+
+ protected byte[] getZeroChunkBytes() {
+ return zeroChunkBytes;
+ }
+
+ protected void prepareBufferAllocator(boolean usingSlicedBuffer) {
+ if (usingSlicedBuffer) {
+ int roughEstimationSpace =
+ chunkSize * (numDataUnits + numParityUnits) * 10;
+ allocator = new BufferAllocator.SlicedBufferAllocator(usingDirectBuffer,
+ roughEstimationSpace);
+ } else {
+ allocator = new BufferAllocator.SimpleBufferAllocator(usingDirectBuffer);
+ }
+ }
+
+ protected boolean isAllowDump() {
+ return allowDump;
+ }
+
+ /**
+ * Prepare before running the case.
+ *
+ * @param usingFixedData Using fixed or pre-generated data to test instead of
+ * generating data
+ */
+ protected void prepare(ConfigurationSource conf,
+ int numDataUnits,
+ int numParityUnits,
+ int[] erasedDataIndexes,
+ int[] erasedParityIndexes,
+ boolean usingFixedData) {
+ this.conf = conf != null ? conf : new OzoneConfiguration();
+ this.numDataUnits = numDataUnits;
+ this.numParityUnits = numParityUnits;
+ this.erasedDataIndexes = erasedDataIndexes != null ?
+ erasedDataIndexes : new int[]{0};
+ this.erasedParityIndexes = erasedParityIndexes != null ?
+ erasedParityIndexes : new int[]{0};
+ this.usingFixedData = usingFixedData;
+ if (usingFixedData) {
+ prepareFixedData();
+ }
+ }
+
+ /**
+ * Prepare before running the case.
+ *
+ * @param conf
+ * @param numDataUnits
+ * @param numParityUnits
+ * @param erasedDataIndexes
+ * @param erasedParityIndexes
+ */
+ protected void prepare(ConfigurationSource conf, int numDataUnits,
+ int numParityUnits, int[] erasedDataIndexes,
+ int[] erasedParityIndexes) {
+ prepare(conf, numDataUnits, numParityUnits, erasedDataIndexes,
+ erasedParityIndexes, false);
+ }
+
+ /**
+ * Prepare before running the case.
+ *
+ * @param numDataUnits
+ * @param numParityUnits
+ * @param erasedDataIndexes
+ * @param erasedParityIndexes
+ */
+ protected void prepare(
+ int numDataUnits,
+ int numParityUnits,
+ int[] erasedDataIndexes,
+ int[] erasedParityIndexes) {
+ prepare(null, numDataUnits, numParityUnits, erasedDataIndexes,
+ erasedParityIndexes, false);
+ }
+
+ /**
+ * Get the conf the test.
+ *
+ * @return configuration
+ */
+ protected ConfigurationSource getConf() {
+ return this.conf;
+ }
+
+ /**
+ * Compare and verify if erased chunks are equal to recovered chunks.
+ *
+ * @param erasedChunks
+ * @param recoveredChunks
+ */
+ protected void compareAndVerify(ECChunk[] erasedChunks,
+ ECChunk[] recoveredChunks) {
+ byte[][] erased = toArrays(erasedChunks);
+ byte[][] recovered = toArrays(recoveredChunks);
+ boolean result = Arrays.deepEquals(erased, recovered);
+ if (!result) {
+ assertTrue("Decoding and comparing failed.", result);
+ }
+ }
+
+ /**
+ * Adjust and return erased indexes altogether, including erased data indexes
+ * and parity indexes.
+ * @return erased indexes altogether
+ */
+ protected int[] getErasedIndexesForDecoding() {
+ int[] erasedIndexesForDecoding =
+ new int[erasedDataIndexes.length + erasedParityIndexes.length];
+
+ int idx = 0;
+
+ for (int i = 0; i < erasedDataIndexes.length; i++) {
+ erasedIndexesForDecoding[idx++] = erasedDataIndexes[i];
+ }
+
+ for (int i = 0; i < erasedParityIndexes.length; i++) {
+ erasedIndexesForDecoding[idx++] = erasedParityIndexes[i] + numDataUnits;
+ }
+
+ return erasedIndexesForDecoding;
+ }
+
+ /**
+ * Return input chunks for decoding, which is dataChunks + parityChunks.
+ *
+ * @param dataChunks
+ * @param parityChunks
+ * @return
+ */
+ protected ECChunk[] prepareInputChunksForDecoding(ECChunk[] dataChunks,
+ ECChunk[] parityChunks) {
+ ECChunk[] inputChunks = new ECChunk[numDataUnits + numParityUnits];
+
+ int idx = 0;
+
+ for (int i = 0; i < numDataUnits; i++) {
+ inputChunks[idx++] = dataChunks[i];
+ }
+
+ for (int i = 0; i < numParityUnits; i++) {
+ inputChunks[idx++] = parityChunks[i];
+ }
+
+ return inputChunks;
+ }
+
+ /**
+ * Erase some data chunks to test the recovering of them. As they're erased,
+ * we don't need to read them and will not have the buffers at all, so just
+ * set them as null.
+ *
+ * @param dataChunks
+ * @param parityChunks
+ * @return clone of erased chunks
+ */
+ protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks,
+ ECChunk[] parityChunks) {
+ ECChunk[] toEraseChunks = new ECChunk[erasedDataIndexes.length +
+ erasedParityIndexes.length];
+
+ int idx = 0;
+
+ for (int i = 0; i < erasedDataIndexes.length; i++) {
+ toEraseChunks[idx++] = dataChunks[erasedDataIndexes[i]];
+ dataChunks[erasedDataIndexes[i]] = null;
+ }
+
+ for (int i = 0; i < erasedParityIndexes.length; i++) {
+ toEraseChunks[idx++] = parityChunks[erasedParityIndexes[i]];
+ parityChunks[erasedParityIndexes[i]] = null;
+ }
+
+ return toEraseChunks;
+ }
+
+ /**
+ * Erase data from the specified chunks, just setting them as null.
+ * @param chunks
+ */
+ protected void eraseDataFromChunks(ECChunk[] chunks) {
+ for (int i = 0; i < chunks.length; i++) {
+ chunks[i] = null;
+ }
+ }
+
+ protected void markChunks(ECChunk[] chunks) {
+ for (int i = 0; i < chunks.length; i++) {
+ if (chunks[i] != null) {
+ chunks[i].getBuffer().mark();
+ }
+ }
+ }
+
+ protected void restoreChunksFromMark(ECChunk[] chunks) {
+ for (int i = 0; i < chunks.length; i++) {
+ if (chunks[i] != null) {
+ chunks[i].getBuffer().reset();
+ }
+ }
+ }
+
+ /**
+ * Clone chunks along with copying the associated data. It respects how the
+ * chunk buffer is allocated, direct or non-direct. It avoids affecting the
+ * original chunk buffers.
+ * @param chunks
+ * @return
+ */
+ protected ECChunk[] cloneChunksWithData(ECChunk[] chunks) {
+ ECChunk[] results = new ECChunk[chunks.length];
+ for (int i = 0; i < chunks.length; i++) {
+ results[i] = cloneChunkWithData(chunks[i]);
+ }
+
+ return results;
+ }
+
+ /**
+ * Clone chunk along with copying the associated data. It respects how the
+ * chunk buffer is allocated, direct or non-direct. It avoids affecting the
+ * original chunk.
+ * @param chunk
+ * @return a new chunk
+ */
+ protected ECChunk cloneChunkWithData(ECChunk chunk) {
+ if (chunk == null) {
+ return null;
+ }
+
+ ByteBuffer srcBuffer = chunk.getBuffer();
+
+ byte[] bytesArr = new byte[srcBuffer.remaining()];
+ srcBuffer.mark();
+ srcBuffer.get(bytesArr, 0, bytesArr.length);
+ srcBuffer.reset();
+
+ ByteBuffer destBuffer = allocateOutputBuffer(bytesArr.length);
+ int pos = destBuffer.position();
+ destBuffer.put(bytesArr);
+ destBuffer.flip();
+ destBuffer.position(pos);
+
+ return new ECChunk(destBuffer);
+ }
+
+ /**
+ * Allocate a chunk for output or writing.
+ * @return
+ */
+ protected ECChunk allocateOutputChunk() {
+ ByteBuffer buffer = allocateOutputBuffer(chunkSize);
+
+ return new ECChunk(buffer);
+ }
+
+ /**
+ * Allocate a buffer for output or writing. It can prepare for two kinds of
+ * data buffers: one with position as 0, the other with position > 0
+ * @return a buffer ready to write chunkSize bytes from current position
+ */
+ protected ByteBuffer allocateOutputBuffer(int bufferLen) {
+ /**
+ * When startBufferWithZero, will prepare a buffer as:---------------
+ * otherwise, the buffer will be like: ___TO--BE--WRITTEN___,
+ * and in the beginning, dummy data are prefixed, to simulate a buffer of
+ * position > 0.
+ */
+ int startOffset = startBufferWithZero ? 0 : 11; // 11 is arbitrary
+ int allocLen = startOffset + bufferLen + startOffset;
+ ByteBuffer buffer = allocator.allocate(allocLen);
+ buffer.limit(startOffset + bufferLen);
+ fillDummyData(buffer, startOffset);
+ startBufferWithZero = !startBufferWithZero;
+
+ return buffer;
+ }
+
+ /**
+ * Prepare data chunks for each data unit, by generating random data.
+ * @return
+ */
+ protected ECChunk[] prepareDataChunksForEncoding() {
+ if (usingFixedData) {
+ ECChunk[] chunks = new ECChunk[numDataUnits];
+ for (int i = 0; i < chunks.length; i++) {
+ chunks[i] = makeChunkUsingData(fixedData[i]);
+ }
+ return chunks;
+ }
+
+ return generateDataChunks();
+ }
+
+ private ECChunk makeChunkUsingData(byte[] data) {
+ ECChunk chunk = allocateOutputChunk();
+ ByteBuffer buffer = chunk.getBuffer();
+ int pos = buffer.position();
+ buffer.put(data, 0, chunkSize);
+ buffer.flip();
+ buffer.position(pos);
+
+ return chunk;
+ }
+
+ private ECChunk[] generateDataChunks() {
+ ECChunk[] chunks = new ECChunk[numDataUnits];
+ for (int i = 0; i < chunks.length; i++) {
+ chunks[i] = generateDataChunk();
+ }
+
+ return chunks;
+ }
+
+ private void prepareFixedData() {
+ // We may load test data from a resource, or just generate randomly.
+ // The generated data will be used across subsequent encode/decode calls.
+ this.fixedData = new byte[numDataUnits][];
+ for (int i = 0; i < numDataUnits; i++) {
+ fixedData[i] = generateFixedData(baseChunkSize * 2);
+ }
+ }
+
+ /**
+ * Generate data chunk by making random data.
+ * @return
+ */
+ protected ECChunk generateDataChunk() {
+ ByteBuffer buffer = allocateOutputBuffer(chunkSize);
+ int pos = buffer.position();
+ buffer.put(generateData(chunkSize));
+ buffer.flip();
+ buffer.position(pos);
+
+ return new ECChunk(buffer);
+ }
+
+ /**
+ * Fill len of dummy data in the buffer at the current position.
+ * @param buffer
+ * @param len
+ */
+ protected void fillDummyData(ByteBuffer buffer, int len) {
+ byte[] dummy = new byte[len];
+ RAND.nextBytes(dummy);
+ buffer.put(dummy);
+ }
+
+ protected byte[] generateData(int len) {
+ byte[] buffer = new byte[len];
+ for (int i = 0; i < buffer.length; i++) {
+ buffer[i] = (byte) RAND.nextInt(256);
+ }
+ return buffer;
+ }
+
+ protected byte[] generateFixedData(int len) {
+ byte[] buffer = new byte[len];
+ for (int i = 0; i < buffer.length; i++) {
+ buffer[i] = (byte) fixedDataGenerator++;
+ if (fixedDataGenerator == 256) {
+ fixedDataGenerator = 0;
+ }
+ }
+ return buffer;
+ }
+
+ /**
+ * Prepare parity chunks for encoding, each chunk for each parity unit.
+ * @return
+ */
+ protected ECChunk[] prepareParityChunksForEncoding() {
+ ECChunk[] chunks = new ECChunk[numParityUnits];
+ for (int i = 0; i < chunks.length; i++) {
+ chunks[i] = allocateOutputChunk();
+ }
+
+ return chunks;
+ }
+
+ /**
+ * Prepare output chunks for decoding, each output chunk for each erased
+ * chunk.
+ * @return
+ */
+ protected ECChunk[] prepareOutputChunksForDecoding() {
+ ECChunk[] chunks = new ECChunk[erasedDataIndexes.length +
+ erasedParityIndexes.length];
+
+ for (int i = 0; i < chunks.length; i++) {
+ chunks[i] = allocateOutputChunk();
+ }
+
+ return chunks;
+ }
+
+ /**
+ * Convert an array of this chunks to an array of byte array.
+ * Note the chunk buffers are not affected.
+ * @param chunks
+ * @return an array of byte array
+ */
+ protected byte[][] toArrays(ECChunk[] chunks) {
+ byte[][] bytesArr = new byte[chunks.length][];
+
+ for (int i = 0; i < chunks.length; i++) {
+ if (chunks[i] != null) {
+ bytesArr[i] = chunks[i].toBytesArray();
+ }
+ }
+
+ return bytesArr;
+ }
+
+ /**
+ * Dump all the settings used in the test case if isAllowingVerboseDump
+ * is enabled.
+ */
+ protected void dumpSetting() {
+ if (allowDump) {
+ StringBuilder sb = new StringBuilder("Erasure coder test settings:\n");
+ sb.append(" numDataUnits=").append(numDataUnits);
+ sb.append(" numParityUnits=").append(numParityUnits);
+ sb.append(" chunkSize=").append(chunkSize).append("\n");
+
+ sb.append(" erasedDataIndexes=").
+ append(Arrays.toString(erasedDataIndexes));
+ sb.append(" erasedParityIndexes=").
+ append(Arrays.toString(erasedParityIndexes));
+ sb.append(" usingDirectBuffer=").append(usingDirectBuffer);
+ sb.append(" allowVerboseDump=").append(allowDump);
+ sb.append("\n");
+
+ System.out.println(sb.toString());
+ }
+ }
+
+ /**
+ * Dump chunks prefixed with a header if isAllowingVerboseDump is enabled.
+ */
+ protected void dumpChunks(String header, ECChunk[] chunks) {
+ if (allowDump) {
+ DumpUtil.dumpChunks(header, chunks);
+ }
+ }
+
+ /**
+ * Make some chunk messy or not correct any more.
+ */
+ protected void corruptSomeChunk(ECChunk[] chunks) {
+ int idx = new Random().nextInt(chunks.length);
+ ByteBuffer buffer = chunks[idx].getBuffer();
+ if (buffer.hasRemaining()) {
+ buffer.position(buffer.position() + 1);
+ }
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/package-info.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/package-info.java
new file mode 100644
index 0000000..c034c0b
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.
+ */
+
+/**
+ * Unit tests for EC related classes.
+ */
+package org.apache.ozone.erasurecode;
+
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderBenchmark.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderBenchmark.java
new file mode 100644
index 0000000..326950d
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/RawErasureCoderBenchmark.java
@@ -0,0 +1,412 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.util.StopWatch;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A benchmark tool to test the performance of different erasure coders.
+ * The tool launches multiple threads to encode/decode certain amount of data,
+ * and measures the total throughput. It only focuses on performance and doesn't
+ * validate correctness of the encoded/decoded results.
+ * User can specify the data size each thread processes, as well as the chunk
+ * size to use for the coder.
+ * Different coders are supported. User can specify the coder by a coder index.
+ * The coder is shared among all the threads.
+ */
+public final class RawErasureCoderBenchmark {
+
+ private RawErasureCoderBenchmark() {
+ // prevent instantiation
+ }
+
+ // target size of input data buffer
+ private static final int TARGET_BUFFER_SIZE_MB = 126;
+
+ private static final int MAX_CHUNK_SIZE =
+ TARGET_BUFFER_SIZE_MB / BenchData.NUM_DATA_UNITS * 1024;
+
+ private static final List<RawErasureCoderFactory> CODER_MAKERS =
+ Collections.unmodifiableList(
+ Arrays.asList(new DummyRawErasureCoderFactory(),
+ new RSRawErasureCoderFactory()));
+
+ enum CODER {
+ DUMMY_CODER("Dummy coder"),
+ RS_CODER("Reed-Solomon Java coder");
+
+ private final String name;
+
+ CODER(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+ }
+
+ static {
+ Preconditions.checkArgument(CODER_MAKERS.size() == CODER.values().length);
+ }
+
+ private static void printAvailableCoders() {
+ StringBuilder sb = new StringBuilder(
+ "Available coders with coderIndex:\n");
+ for (CODER coder : CODER.values()) {
+ sb.append(coder.ordinal()).append(":").append(coder).append("\n");
+ }
+ System.out.println(sb);
+ }
+
+ private static void usage(String message) {
+ if (message != null) {
+ System.out.println(message);
+ }
+ System.out.println(
+ "Usage: RawErasureCoderBenchmark <encode/decode> <coderIndex> " +
+ "[numThreads] [dataSize-in-MB] [chunkSize-in-KB]");
+ printAvailableCoders();
+ System.exit(1);
+ }
+
+ public static void main(String[] args) throws Exception {
+ String opType = null;
+ int coderIndex = 0;
+ // default values
+ int dataSizeMB = 10240;
+ int chunkSizeKB = 1024;
+ int numThreads = 1;
+
+ if (args.length > 1) {
+ opType = args[0];
+ if (!"encode".equals(opType) && !"decode".equals(opType)) {
+ usage("Invalid type: should be either 'encode' or 'decode'");
+ }
+
+ try {
+ coderIndex = Integer.parseInt(args[1]);
+ if (coderIndex < 0 || coderIndex >= CODER.values().length) {
+ usage("Invalid coder index, should be [0-" +
+ (CODER.values().length - 1) + "]");
+ }
+ } catch (NumberFormatException e) {
+ usage("Malformed coder index, " + e.getMessage());
+ }
+ } else {
+ usage(null);
+ }
+
+ if (args.length > 2) {
+ try {
+ numThreads = Integer.parseInt(args[2]);
+ if (numThreads <= 0) {
+ usage("Invalid number of threads.");
+ }
+ } catch (NumberFormatException e) {
+ usage("Malformed number of threads, " + e.getMessage());
+ }
+ }
+
+ if (args.length > 3) {
+ try {
+ dataSizeMB = Integer.parseInt(args[3]);
+ if (dataSizeMB <= 0) {
+ usage("Invalid data size.");
+ }
+ } catch (NumberFormatException e) {
+ usage("Malformed data size, " + e.getMessage());
+ }
+ }
+
+ if (args.length > 4) {
+ try {
+ chunkSizeKB = Integer.parseInt(args[4]);
+ if (chunkSizeKB <= 0) {
+ usage("Chunk size should be positive.");
+ }
+ if (chunkSizeKB > MAX_CHUNK_SIZE) {
+ usage("Chunk size should be no larger than " + MAX_CHUNK_SIZE);
+ }
+ } catch (NumberFormatException e) {
+ usage("Malformed chunk size, " + e.getMessage());
+ }
+ }
+
+ performBench(opType, CODER.values()[coderIndex],
+ numThreads, dataSizeMB, chunkSizeKB);
+ }
+
+ /**
+ * Performs benchmark.
+ *
+ * @param opType The operation to perform. Can be encode or decode
+ * @param coder The coder to use
+ * @param numThreads Number of threads to launch concurrently
+ * @param dataSizeMB Total test data size in MB
+ * @param chunkSizeKB Chunk size in KB
+ */
+ public static void performBench(String opType, CODER coder,
+ int numThreads, int dataSizeMB, int chunkSizeKB) throws Exception {
+ BenchData.configure(dataSizeMB, chunkSizeKB);
+
+ RawErasureEncoder encoder = null;
+ RawErasureDecoder decoder = null;
+ ByteBuffer testData;
+ boolean isEncode = opType.equals("encode");
+
+ if (isEncode) {
+ encoder = getRawEncoder(coder.ordinal());
+ testData = genTestData(encoder.preferDirectBuffer(),
+ BenchData.bufferSizeKB);
+ } else {
+ decoder = getRawDecoder(coder.ordinal());
+ testData = genTestData(decoder.preferDirectBuffer(),
+ BenchData.bufferSizeKB);
+ }
+
+ ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+ List<Future<Long>> futures = new ArrayList<>(numThreads);
+ StopWatch sw = new StopWatch().start();
+ for (int i = 0; i < numThreads; i++) {
+ futures.add(executor.submit(new BenchmarkCallable(isEncode,
+ encoder, decoder, testData.duplicate())));
+ }
+ List<Long> durations = new ArrayList<>(numThreads);
+ try {
+ for (Future<Long> future : futures) {
+ durations.add(future.get());
+ }
+ long duration = sw.now(TimeUnit.MILLISECONDS);
+ double totalDataSize = BenchData.totalDataSizeKB * numThreads / 1024.0;
+ DecimalFormat df = new DecimalFormat("#.##");
+ System.out.println(coder + " " + opType + " " +
+ df.format(totalDataSize) + "MB data, with chunk size " +
+ BenchData.chunkSize / 1024 + "KB");
+ System.out.println("Total time: " + df.format(duration / 1000.0) + " s.");
+ System.out.println("Total throughput: " + df.format(
+ totalDataSize / duration * 1000.0) + " MB/s");
+ printThreadStatistics(durations, df);
+ } catch (Exception e) {
+ System.out.println("Error waiting for thread to finish.");
+ e.printStackTrace();
+ throw e;
+ } finally {
+ executor.shutdown();
+ if (encoder != null) {
+ encoder.release();
+ }
+ if (decoder != null) {
+ decoder.release();
+ }
+ }
+ }
+
+ private static RawErasureEncoder getRawEncoder(int index) throws IOException {
+ RawErasureEncoder encoder =
+ CODER_MAKERS.get(index).createEncoder(BenchData.OPTIONS);
+ final boolean isDirect = encoder.preferDirectBuffer();
+ encoder.encode(
+ getBufferForInit(BenchData.NUM_DATA_UNITS, 1, isDirect),
+ getBufferForInit(BenchData.NUM_PARITY_UNITS, 1, isDirect));
+ return encoder;
+ }
+
+ private static RawErasureDecoder getRawDecoder(int index) throws IOException {
+ RawErasureDecoder decoder =
+ CODER_MAKERS.get(index).createDecoder(BenchData.OPTIONS);
+ final boolean isDirect = decoder.preferDirectBuffer();
+ ByteBuffer[] inputs = getBufferForInit(
+ BenchData.NUM_ALL_UNITS, 1, isDirect);
+ for (int erasedIndex : BenchData.ERASED_INDEXES) {
+ inputs[erasedIndex] = null;
+ }
+ decoder.decode(inputs, BenchData.ERASED_INDEXES,
+ getBufferForInit(BenchData.ERASED_INDEXES.length, 1, isDirect));
+ return decoder;
+ }
+
+ private static ByteBuffer[] getBufferForInit(int numBuf,
+ int bufCap, boolean isDirect) {
+ ByteBuffer[] buffers = new ByteBuffer[numBuf];
+ for (int i = 0; i < buffers.length; i++) {
+ buffers[i] = isDirect ? ByteBuffer.allocateDirect(bufCap) :
+ ByteBuffer.allocate(bufCap);
+ }
+ return buffers;
+ }
+
+ private static void printThreadStatistics(
+ List<Long> durations, DecimalFormat df) {
+ Collections.sort(durations);
+ System.out.println("Threads statistics: ");
+ Double min = durations.get(0) / 1000.0;
+ Double max = durations.get(durations.size() - 1) / 1000.0;
+ Long sum = 0L;
+ for (Long duration : durations) {
+ sum += duration;
+ }
+ Double avg = sum.doubleValue() / durations.size() / 1000.0;
+ Double percentile = durations.get(
+ (int) Math.ceil(durations.size() * 0.9) - 1) / 1000.0;
+ System.out.println(durations.size() + " threads in total.");
+ System.out.println("Min: " + df.format(min) + " s, Max: " +
+ df.format(max) + " s, Avg: " + df.format(avg) +
+ " s, 90th Percentile: " + df.format(percentile) + " s.");
+ }
+
+ private static ByteBuffer genTestData(boolean useDirectBuffer, int sizeKB) {
+ Random random = new Random();
+ int bufferSize = sizeKB * 1024;
+ byte[] tmp = new byte[bufferSize];
+ random.nextBytes(tmp);
+ ByteBuffer data = useDirectBuffer ?
+ ByteBuffer.allocateDirect(bufferSize) :
+ ByteBuffer.allocate(bufferSize);
+ data.put(tmp);
+ data.flip();
+ return data;
+ }
+
+ private static class BenchData {
+ public static final ECReplicationConfig OPTIONS =
+ new ECReplicationConfig(6, 3);
+ public static final int NUM_DATA_UNITS = OPTIONS.getData();
+ public static final int NUM_PARITY_UNITS = OPTIONS.getParity();
+ public static final int NUM_ALL_UNITS = OPTIONS.getData() +
+ OPTIONS.getParity();
+ private static int chunkSize;
+ private static long totalDataSizeKB;
+ private static int bufferSizeKB;
+
+ private static final int[] ERASED_INDEXES = new int[]{6, 7, 8};
+ private final ByteBuffer[] inputs = new ByteBuffer[NUM_DATA_UNITS];
+ private ByteBuffer[] outputs = new ByteBuffer[NUM_PARITY_UNITS];
+ private ByteBuffer[] decodeInputs = new ByteBuffer[NUM_ALL_UNITS];
+
+ public static void configure(int dataSizeMB, int chunkSizeKB) {
+ chunkSize = chunkSizeKB * 1024;
+ // buffer size needs to be a multiple of (numDataUnits * chunkSize)
+ int round = (int) Math.round(
+ TARGET_BUFFER_SIZE_MB * 1024.0 / NUM_DATA_UNITS / chunkSizeKB);
+ Preconditions.checkArgument(round > 0);
+ bufferSizeKB = NUM_DATA_UNITS * chunkSizeKB * round;
+ System.out.println("Using " + bufferSizeKB / 1024 + "MB buffer.");
+
+ round = (int) Math.round(
+ (dataSizeMB * 1024.0) / bufferSizeKB);
+ if (round == 0) {
+ round = 1;
+ }
+ totalDataSizeKB = round * bufferSizeKB;
+ }
+
+ BenchData(boolean useDirectBuffer) {
+ for (int i = 0; i < outputs.length; i++) {
+ outputs[i] = useDirectBuffer ? ByteBuffer.allocateDirect(chunkSize) :
+ ByteBuffer.allocate(chunkSize);
+ }
+ }
+
+ public void prepareDecInput() {
+ System.arraycopy(inputs, 0, decodeInputs, 0, NUM_DATA_UNITS);
+ }
+
+ public void encode(RawErasureEncoder encoder) throws IOException {
+ encoder.encode(inputs, outputs);
+ }
+
+ public void decode(RawErasureDecoder decoder) throws IOException {
+ decoder.decode(decodeInputs, ERASED_INDEXES, outputs);
+ }
+ }
+
+ private static class BenchmarkCallable implements Callable<Long> {
+ private final boolean isEncode;
+ private final RawErasureEncoder encoder;
+ private final RawErasureDecoder decoder;
+ private final BenchData benchData;
+ private final ByteBuffer testData;
+
+ BenchmarkCallable(boolean isEncode, RawErasureEncoder encoder,
+ RawErasureDecoder decoder, ByteBuffer testData) {
+ if (isEncode) {
+ Preconditions.checkArgument(encoder != null);
+ this.encoder = encoder;
+ this.decoder = null;
+ benchData = new BenchData(encoder.preferDirectBuffer());
+ } else {
+ Preconditions.checkArgument(decoder != null);
+ this.decoder = decoder;
+ this.encoder = null;
+ benchData = new BenchData(decoder.preferDirectBuffer());
+ }
+ this.isEncode = isEncode;
+ this.testData = testData;
+ }
+
+ @Override
+ public Long call() throws Exception {
+ long rounds = BenchData.totalDataSizeKB / BenchData.bufferSizeKB;
+
+ StopWatch sw = new StopWatch().start();
+ for (long i = 0; i < rounds; i++) {
+ while (testData.remaining() > 0) {
+ for (ByteBuffer output : benchData.outputs) {
+ output.clear();
+ }
+
+ for (int j = 0; j < benchData.inputs.length; j++) {
+ benchData.inputs[j] = testData.duplicate();
+ benchData.inputs[j].limit(
+ testData.position() + BenchData.chunkSize);
+ benchData.inputs[j] = benchData.inputs[j].slice();
+ testData.position(testData.position() + BenchData.chunkSize);
+ }
+
+ if (isEncode) {
+ benchData.encode(encoder);
+ } else {
+ benchData.prepareDecInput();
+ benchData.decode(decoder);
+ }
+ }
+ testData.clear();
+ }
+ return sw.now(TimeUnit.MILLISECONDS);
+ }
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestCodecRawCoderMapping.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestCodecRawCoderMapping.java
new file mode 100644
index 0000000..5d749ee
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestCodecRawCoderMapping.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.rawcoder.util.CodecUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test the codec to raw coder mapping.
+ */
+public class TestCodecRawCoderMapping {
+
+ private static final String RS_CODEC_NAME = "rs";
+ private static final String XOR_CODEC_NAME = "xor";
+
+ @Test
+ public void testRSDefaultRawCoder() {
+ ECReplicationConfig coderOptions =
+ new ECReplicationConfig(RS_CODEC_NAME + "-6-3-1024K");
+ // should return default raw coder of rs codec
+ RawErasureEncoder encoder =
+ CodecUtil.createRawEncoderWithFallback(coderOptions);
+ RawErasureDecoder decoder =
+ CodecUtil.createRawDecoderWithFallback(coderOptions);
+ if (ErasureCodeNative.isNativeCodeLoaded()) {
+ Assert.assertTrue(encoder instanceof NativeRSRawEncoder);
+ Assert.assertTrue(decoder instanceof NativeRSRawDecoder);
+ } else {
+ Assert.assertTrue(encoder instanceof RSRawEncoder);
+ Assert.assertTrue(decoder instanceof RSRawDecoder);
+ }
+ }
+
+ @Test
+ public void testXORRawCoder() {
+ ECReplicationConfig coderOptions =
+ new ECReplicationConfig(XOR_CODEC_NAME + "-6-3-1024K");
+ // should return default raw coder of rs codec
+ RawErasureEncoder encoder =
+ CodecUtil.createRawEncoderWithFallback(coderOptions);
+ RawErasureDecoder decoder =
+ CodecUtil.createRawDecoderWithFallback(coderOptions);
+ if (ErasureCodeNative.isNativeCodeLoaded()) {
+ Assert.assertTrue(encoder instanceof NativeXORRawEncoder);
+ Assert.assertTrue(decoder instanceof NativeXORRawDecoder);
+ } else {
+ Assert.assertTrue(encoder instanceof XORRawEncoder);
+ Assert.assertTrue(decoder instanceof XORRawDecoder);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestDummyRawCoder.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestDummyRawCoder.java
new file mode 100644
index 0000000..10c72b2
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestDummyRawCoder.java
@@ -0,0 +1,96 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.ozone.erasurecode.ECChunk;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Test dummy raw coder.
+ */
+public class TestDummyRawCoder extends TestRawCoderBase {
+
+ public TestDummyRawCoder() {
+ super(DummyRawErasureCoderFactory.class, DummyRawErasureCoderFactory.class);
+ }
+
+ @Before
+ public void setup() {
+ setAllowDump(false);
+ setChunkSize(baseChunkSize);
+ }
+
+ @Test
+ public void testCoding6x3ErasingD0Dd2() {
+ prepare(null, 6, 3, new int[]{0, 2}, new int[0], false);
+ testCodingDoMixed();
+ }
+
+ @Test
+ public void testCoding6x3ErasingD0P0() {
+ prepare(null, 6, 3, new int[]{0}, new int[]{0}, false);
+ testCodingDoMixed();
+ }
+
+ @Override
+ protected void testCoding(boolean usingDirectBuffer) {
+ this.usingDirectBuffer = usingDirectBuffer;
+ prepareCoders(true);
+
+ prepareBufferAllocator(true);
+
+ // Generate data and encode
+ ECChunk[] dataChunks = prepareDataChunksForEncoding();
+ markChunks(dataChunks);
+ ECChunk[] parityChunks = prepareParityChunksForEncoding();
+ try {
+ encode(dataChunks, parityChunks);
+ } catch (IOException e) {
+ Assert.fail("Unexpected IOException: " + e.getMessage());
+ }
+ compareAndVerify(parityChunks, getEmptyChunks(parityChunks.length));
+
+ // Decode
+ restoreChunksFromMark(dataChunks);
+ backupAndEraseChunks(dataChunks, parityChunks);
+ ECChunk[] inputChunks = prepareInputChunksForDecoding(
+ dataChunks, parityChunks);
+ ensureOnlyLeastRequiredChunks(inputChunks);
+ ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
+ try {
+ decode(inputChunks, getErasedIndexesForDecoding(),
+ recoveredChunks);
+ } catch (IOException e) {
+ Assert.fail("Unexpected IOException: " + e.getMessage());
+ }
+ compareAndVerify(recoveredChunks, getEmptyChunks(recoveredChunks.length));
+ }
+
+ private ECChunk[] getEmptyChunks(int num) {
+ ECChunk[] chunks = new ECChunk[num];
+ for (int i = 0; i < chunks.length; i++) {
+ chunks[i] = new ECChunk(ByteBuffer.wrap(getZeroChunkBytes()));
+ }
+ return chunks;
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeRSRawCoder.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeRSRawCoder.java
new file mode 100644
index 0000000..2ab038b
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeRSRawCoder.java
@@ -0,0 +1,129 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test native raw Reed-solomon encoding and decoding.
+ */
+public class TestNativeRSRawCoder extends TestRSRawCoderBase {
+
+ public TestNativeRSRawCoder() {
+ super(NativeRSRawErasureCoderFactory.class,
+ NativeRSRawErasureCoderFactory.class);
+ }
+
+ @Before
+ public void setup() {
+ Assume.assumeTrue(ErasureCodeNative.isNativeCodeLoaded());
+ setAllowDump(true);
+ }
+
+ @Test
+ public void testCoding6x3ErasingAllD() {
+ prepare(null, 6, 3, new int[]{0, 1, 2}, new int[0], true);
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6x3ErasingD0D2() {
+ prepare(null, 6, 3, new int[] {0, 2}, new int[]{});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6x3ErasingD0() {
+ prepare(null, 6, 3, new int[]{0}, new int[0]);
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6x3ErasingD2() {
+ prepare(null, 6, 3, new int[]{2}, new int[]{});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6x3ErasingD0P0() {
+ prepare(null, 6, 3, new int[]{0}, new int[]{0});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6x3ErasingAllP() {
+ prepare(null, 6, 3, new int[0], new int[]{0, 1, 2});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6x3ErasingP0() {
+ prepare(null, 6, 3, new int[0], new int[]{0});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6x3ErasingP2() {
+ prepare(null, 6, 3, new int[0], new int[]{2});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6x3ErasureP0P2() {
+ prepare(null, 6, 3, new int[0], new int[]{0, 2});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6x3ErasingD0P0P1() {
+ prepare(null, 6, 3, new int[]{0}, new int[]{0, 1});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6x3ErasingD0D2P2() {
+ prepare(null, 6, 3, new int[]{0, 2}, new int[]{2});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCodingNegative6x3ErasingD2D4() {
+ prepare(null, 6, 3, new int[]{2, 4}, new int[0]);
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCodingNegative6x3ErasingTooMany() {
+ prepare(null, 6, 3, new int[]{2, 4}, new int[]{0, 1});
+ testCodingWithErasingTooMany();
+ }
+
+ @Test
+ public void testCoding10x4ErasingD0P0() {
+ prepare(null, 10, 4, new int[] {0}, new int[] {0});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testAfterRelease63() throws Exception {
+ prepare(6, 3, null, null);
+ testAfterRelease();
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeXORRawCoder.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeXORRawCoder.java
new file mode 100644
index 0000000..9312c25
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestNativeXORRawCoder.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.ozone.erasurecode.rawcoder;
+
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test NativeXOR encoding and decoding.
+ */
+public class TestNativeXORRawCoder extends TestXORRawCoderBase {
+
+ public TestNativeXORRawCoder() {
+ super(NativeXORRawErasureCoderFactory.class,
+ NativeXORRawErasureCoderFactory.class);
+ }
+
+ @Before
+ public void setup() {
+ Assume.assumeTrue(ErasureCodeNative.isNativeCodeLoaded());
+ setAllowDump(true);
+ }
+
+ @Test
+ public void testAfterRelease63() throws Exception {
+ prepare(6, 3, null, null);
+ testAfterRelease();
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoder.java
new file mode 100644
index 0000000..93496b3
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoder.java
@@ -0,0 +1,35 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.junit.Before;
+
+/**
+ * Test the new raw Reed-solomon coder implemented in Java.
+ */
+public class TestRSRawCoder extends TestRSRawCoderBase {
+
+ public TestRSRawCoder() {
+ super(RSRawErasureCoderFactory.class, RSRawErasureCoderFactory.class);
+ }
+
+ @Before
+ public void setup() {
+ setAllowDump(false);
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoderBase.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoderBase.java
new file mode 100644
index 0000000..e6c795a
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRSRawCoderBase.java
@@ -0,0 +1,123 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.junit.Test;
+
+/**
+ * Test base for raw Reed-solomon coders.
+ */
+public abstract class TestRSRawCoderBase extends TestRawCoderBase {
+
+ public TestRSRawCoderBase(
+ Class<? extends RawErasureCoderFactory> encoderFactoryClass,
+ Class<? extends RawErasureCoderFactory> decoderFactoryClass) {
+ super(encoderFactoryClass, decoderFactoryClass);
+ }
+
+ @Test
+ public void testCoding6xaErasingAllD() {
+ prepare(null, 6, 3, new int[]{0, 1, 2}, new int[0], true);
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6xaErasingD0D2() {
+ prepare(null, 6, 3, new int[]{0, 2}, new int[]{});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6xaErasingD0() {
+ prepare(null, 6, 3, new int[]{0}, new int[0]);
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6xaErasingD2() {
+ prepare(null, 6, 3, new int[]{2}, new int[]{});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6xaErasingD0P0() {
+ prepare(null, 6, 3, new int[]{0}, new int[]{0});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6xaErasingAllP() {
+ prepare(null, 6, 3, new int[0], new int[]{0, 1, 2});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6xaErasingP0() {
+ prepare(null, 6, 3, new int[0], new int[]{0});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6xaErasingP2() {
+ prepare(null, 6, 3, new int[0], new int[]{2});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6xaErasureP0P2() {
+ prepare(null, 6, 3, new int[0], new int[]{0, 2});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6xaErasingD0P0P1() {
+ prepare(null, 6, 3, new int[]{0}, new int[]{0, 1});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding6xaErasingD0D2P2() {
+ prepare(null, 6, 3, new int[]{0, 2}, new int[]{2});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCodingNegative6xaErasingD2D4() {
+ prepare(null, 6, 3, new int[]{2, 4}, new int[0]);
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCodingNegative6xaErasingTooMany() {
+ prepare(null, 6, 3, new int[]{2, 4}, new int[]{0, 1});
+ testCodingWithErasingTooMany();
+ }
+
+ @Test
+ public void testCoding10x4ErasingD0P0() {
+ prepare(null, 10, 4, new int[] {0}, new int[] {0});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCodingInputBufferPosition() {
+ prepare(null, 6, 3, new int[]{0}, new int[]{0});
+ testInputPosition(false);
+ testInputPosition(true);
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawCoderBase.java
new file mode 100644
index 0000000..044b60f
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawCoderBase.java
@@ -0,0 +1,353 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.ozone.erasurecode.ECChunk;
+import org.apache.ozone.erasurecode.TestCoderBase;
+import org.apache.ozone.test.LambdaTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Raw coder test base with utilities.
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public abstract class TestRawCoderBase extends TestCoderBase {
+ private final Class<? extends RawErasureCoderFactory> encoderFactoryClass;
+ private final Class<? extends RawErasureCoderFactory> decoderFactoryClass;
+ private RawErasureEncoder encoder;
+ private RawErasureDecoder decoder;
+
+ public TestRawCoderBase(
+ Class<? extends RawErasureCoderFactory> encoderFactoryClass,
+ Class<? extends RawErasureCoderFactory> decoderFactoryClass) {
+ this.encoderFactoryClass = encoderFactoryClass;
+ this.decoderFactoryClass = decoderFactoryClass;
+ }
+
+ /**
+ * Doing twice to test if the coders can be repeatedly reused. This matters
+ * as the underlying coding buffers are shared, which may have bugs.
+ */
+ protected void testCodingDoMixAndTwice() {
+ testCodingDoMixed();
+ testCodingDoMixed();
+ }
+
+ /**
+ * Doing in mixed buffer usage model to test if the coders can be repeatedly
+ * reused with different buffer usage model. This matters as the underlying
+ * coding buffers are shared, which may have bugs.
+ */
+ protected void testCodingDoMixed() {
+ testCoding(true);
+ testCoding(false);
+ }
+
+ /**
+ * Generating source data, encoding, recovering and then verifying.
+ * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
+ * it supports two kinds of ByteBuffers, one is array backed, the other is
+ * direct ByteBuffer. Use usingDirectBuffer indicate which case to test.
+ *
+ * @param usingDirectBuffer
+ */
+ protected void testCoding(boolean usingDirectBuffer) {
+ this.usingDirectBuffer = usingDirectBuffer;
+ prepareCoders(true);
+
+ /**
+ * The following runs will use 3 different chunkSize for inputs and outputs,
+ * to verify the same encoder/decoder can process variable width of data.
+ */
+ performTestCoding(baseChunkSize, true, false, false);
+ performTestCoding(baseChunkSize - 17, false, false, false);
+ performTestCoding(baseChunkSize + 16, true, false, false);
+ }
+
+ /**
+ * Similar to above, but perform negative cases using bad input for encoding.
+ * @param usingDirectBuffer
+ */
+ protected void testCodingWithBadInput(boolean usingDirectBuffer) {
+ this.usingDirectBuffer = usingDirectBuffer;
+ prepareCoders(true);
+
+ try {
+ performTestCoding(baseChunkSize, false, true, false);
+ Assert.fail("Encoding test with bad input should fail");
+ } catch (Exception e) {
+ // Expected
+ }
+ }
+
+ /**
+ * Similar to above, but perform negative cases using bad output for decoding.
+ * @param usingDirectBuffer
+ */
+ protected void testCodingWithBadOutput(boolean usingDirectBuffer) {
+ this.usingDirectBuffer = usingDirectBuffer;
+ prepareCoders(true);
+
+ try {
+ performTestCoding(baseChunkSize, false, false, true);
+ Assert.fail("Decoding test with bad output should fail");
+ } catch (Exception e) {
+ // Expected
+ }
+ }
+
+ /**
+ * Test encode / decode after release(). It should raise IOException.
+ *
+ * @throws Exception
+ */
+ void testAfterRelease() throws Exception {
+ prepareCoders(true);
+ prepareBufferAllocator(true);
+
+ encoder.release();
+ final ECChunk[] data = prepareDataChunksForEncoding();
+ final ECChunk[] parity = prepareParityChunksForEncoding();
+ LambdaTestUtils.intercept(IOException.class, "closed",
+ () -> encoder.encode(data, parity));
+
+ decoder.release();
+ final ECChunk[] in = prepareInputChunksForDecoding(data, parity);
+ final ECChunk[] out = prepareOutputChunksForDecoding();
+ LambdaTestUtils.intercept(IOException.class, "closed",
+ () -> decoder.decode(in, getErasedIndexesForDecoding(), out));
+ }
+
+ @Test
+ public void testCodingWithErasingTooMany() {
+ try {
+ testCoding(true);
+ Assert.fail("Decoding test erasing too many should fail");
+ } catch (Exception e) {
+ // Expected
+ }
+
+ try {
+ testCoding(false);
+ Assert.fail("Decoding test erasing too many should fail");
+ } catch (Exception e) {
+ // Expected
+ }
+ }
+
+ @Test
+ public void testIdempotentReleases() {
+ prepareCoders(true);
+
+ for (int i = 0; i < 3; i++) {
+ encoder.release();
+ decoder.release();
+ }
+ }
+
+ private void performTestCoding(int chunkSize, boolean usingSlicedBuffer,
+ boolean useBadInput, boolean useBadOutput) {
+ setChunkSize(chunkSize);
+ prepareBufferAllocator(usingSlicedBuffer);
+
+ dumpSetting();
+
+ // Generate data and encode
+ ECChunk[] dataChunks = prepareDataChunksForEncoding();
+ if (useBadInput) {
+ corruptSomeChunk(dataChunks);
+ }
+ dumpChunks("Testing data chunks", dataChunks);
+
+ ECChunk[] parityChunks = prepareParityChunksForEncoding();
+
+ // Backup all the source chunks for later recovering because some coders
+ // may affect the source data.
+ ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks);
+ markChunks(dataChunks);
+
+ try {
+ encoder.encode(dataChunks, parityChunks);
+ } catch (IOException e) {
+ Assert.fail("Should not get IOException: " + e.getMessage());
+ }
+ dumpChunks("Encoded parity chunks", parityChunks);
+
+ //TODOif (!allowChangeInputs) {
+ restoreChunksFromMark(dataChunks);
+ compareAndVerify(clonedDataChunks, dataChunks);
+ //}
+
+ // Backup and erase some chunks
+ ECChunk[] backupChunks =
+ backupAndEraseChunks(clonedDataChunks, parityChunks);
+
+ // Decode
+ ECChunk[] inputChunks = prepareInputChunksForDecoding(
+ clonedDataChunks, parityChunks);
+
+ // Remove unnecessary chunks,
+ // allowing only least required chunks to be read.
+ ensureOnlyLeastRequiredChunks(inputChunks);
+
+ ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
+ if (useBadOutput) {
+ corruptSomeChunk(recoveredChunks);
+ }
+
+ ECChunk[] clonedInputChunks = null;
+ //TODOif (!allowChangeInputs) {
+ markChunks(inputChunks);
+ clonedInputChunks = cloneChunksWithData(inputChunks);
+ //}
+
+ dumpChunks("Decoding input chunks", inputChunks);
+ try {
+ decoder.decode(inputChunks, getErasedIndexesForDecoding(),
+ recoveredChunks);
+ } catch (IOException e) {
+ Assert.fail("Should not get IOException: " + e.getMessage());
+ }
+ dumpChunks("Decoded/recovered chunks", recoveredChunks);
+
+ //TODOif (!allowChangeInputs) {
+ restoreChunksFromMark(inputChunks);
+ compareAndVerify(clonedInputChunks, inputChunks);
+ //}
+
+ // Compare
+ compareAndVerify(backupChunks, recoveredChunks);
+ }
+
+ /**
+ * Set true during setup if want to dump test settings and coding data,
+ * useful in debugging.
+ * @param allowDump
+ */
+ protected void setAllowDump(boolean allowDump) {
+ this.allowDump = allowDump;
+ }
+
+ protected void prepareCoders(boolean recreate) {
+ if (encoder == null || recreate) {
+ encoder = createEncoder();
+ }
+
+ if (decoder == null || recreate) {
+ decoder = createDecoder();
+ }
+ }
+
+ protected void ensureOnlyLeastRequiredChunks(ECChunk[] inputChunks) {
+ int leastRequiredNum = numDataUnits;
+ int erasedNum = erasedDataIndexes.length + erasedParityIndexes.length;
+ int goodNum = inputChunks.length - erasedNum;
+ int redundantNum = goodNum - leastRequiredNum;
+
+ for (int i = 0; i < inputChunks.length && redundantNum > 0; i++) {
+ if (inputChunks[i] != null) {
+ inputChunks[i] = null; // Setting it null, not needing it actually
+ redundantNum--;
+ }
+ }
+ }
+
+ /**
+ * Create the raw erasure encoder to test.
+ */
+ protected RawErasureEncoder createEncoder() {
+ ECReplicationConfig replicationConfig =
+ new ECReplicationConfig(numDataUnits, numParityUnits);
+ try {
+ RawErasureCoderFactory factory = encoderFactoryClass.newInstance();
+ return factory.createEncoder(replicationConfig);
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to create encoder", e);
+ }
+ }
+
+ /**
+ * Create the raw erasure decoder to test.
+ */
+ protected RawErasureDecoder createDecoder() {
+ ECReplicationConfig replicationConfig =
+ new ECReplicationConfig(numDataUnits, numParityUnits);
+ try {
+ RawErasureCoderFactory factory = encoderFactoryClass.newInstance();
+ return factory.createDecoder(replicationConfig);
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to create decoder", e);
+ }
+ }
+
+ /**
+ * Tests that the input buffer's position is moved to the end after
+ * encode/decode.
+ */
+ protected void testInputPosition(boolean usingDirectBuffer) {
+ this.usingDirectBuffer = usingDirectBuffer;
+ prepareCoders(true);
+ prepareBufferAllocator(false);
+
+ // verify encode
+ ECChunk[] dataChunks = prepareDataChunksForEncoding();
+ ECChunk[] parityChunks = prepareParityChunksForEncoding();
+ ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks);
+ try {
+ encoder.encode(dataChunks, parityChunks);
+ } catch (IOException e) {
+ Assert.fail("Should not get IOException: " + e.getMessage());
+ }
+ verifyBufferPositionAtEnd(dataChunks);
+
+ // verify decode
+ backupAndEraseChunks(clonedDataChunks, parityChunks);
+ ECChunk[] inputChunks = prepareInputChunksForDecoding(
+ clonedDataChunks, parityChunks);
+ ensureOnlyLeastRequiredChunks(inputChunks);
+ ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
+ try {
+ decoder.decode(inputChunks, getErasedIndexesForDecoding(),
+ recoveredChunks);
+ } catch (IOException e) {
+ Assert.fail("Should not get IOException: " + e.getMessage());
+ }
+ verifyBufferPositionAtEnd(inputChunks);
+ }
+
+ private void verifyBufferPositionAtEnd(ECChunk[] inputChunks) {
+ for (ECChunk chunk : inputChunks) {
+ if (chunk != null) {
+ Assert.assertEquals(0, chunk.getBuffer().remaining());
+ }
+ }
+ }
+
+ public void encode(ECChunk[] inputs, ECChunk[] outputs) throws IOException {
+ this.encoder.encode(inputs, outputs);
+ }
+
+ public void decode(ECChunk[] inputs, int[] erasedIndexes,
+ ECChunk[] outputs) throws IOException {
+ this.decoder.decode(inputs, erasedIndexes, outputs);
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawErasureCoderBenchmark.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawErasureCoderBenchmark.java
new file mode 100644
index 0000000..cb700c4
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestRawErasureCoderBenchmark.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.junit.Test;
+
+/**
+ * Tests for the raw erasure coder benchmark tool.
+ */
+public class TestRawErasureCoderBenchmark {
+
+ @Test
+ public void testDummyCoder() throws Exception {
+ // Dummy coder
+ RawErasureCoderBenchmark.performBench("encode",
+ RawErasureCoderBenchmark.CODER.DUMMY_CODER, 2, 100, 1024);
+ RawErasureCoderBenchmark.performBench("decode",
+ RawErasureCoderBenchmark.CODER.DUMMY_CODER, 5, 150, 100);
+ }
+
+ @Test
+ public void testRSCoder() throws Exception {
+ // RS Java coder
+ RawErasureCoderBenchmark.performBench("encode",
+ RawErasureCoderBenchmark.CODER.RS_CODER, 3, 200, 200);
+ RawErasureCoderBenchmark.performBench("decode",
+ RawErasureCoderBenchmark.CODER.RS_CODER, 4, 135, 20);
+ }
+
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoder.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoder.java
new file mode 100644
index 0000000..dcc8099
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoder.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.ozone.erasurecode.rawcoder;
+
+/**
+ * Test pure Java XOR encoding and decoding.
+ */
+public class TestXORRawCoder extends TestXORRawCoderBase {
+
+ public TestXORRawCoder() {
+ super(XORRawErasureCoderFactory.class, XORRawErasureCoderFactory.class);
+ }
+
+
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoderBase.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoderBase.java
new file mode 100644
index 0000000..77ed8c7
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/TestXORRawCoderBase.java
@@ -0,0 +1,65 @@
+/**
+ * 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.ozone.erasurecode.rawcoder;
+
+import org.junit.Test;
+
+/**
+ * Test base for raw XOR coders.
+ */
+public abstract class TestXORRawCoderBase extends TestRawCoderBase {
+
+ public TestXORRawCoderBase(
+ Class<? extends RawErasureCoderFactory> encoderFactoryClass,
+ Class<? extends RawErasureCoderFactory> decoderFactoryClass) {
+ super(encoderFactoryClass, decoderFactoryClass);
+ }
+
+ @Test
+ public void testCoding10x1ErasingD0() {
+ prepare(null, 10, 1, new int[]{0}, new int[0]);
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding10x1ErasingP0() {
+ prepare(null, 10, 1, new int[0], new int[]{0});
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCoding10x1ErasingD5() {
+ prepare(null, 10, 1, new int[]{5}, new int[0]);
+ testCodingDoMixAndTwice();
+ }
+
+ @Test
+ public void testCodingNegative10x1ErasingTooMany() {
+ prepare(null, 10, 1, new int[]{2}, new int[]{0});
+ testCodingWithErasingTooMany();
+ }
+
+ @Test
+ public void testCodingNegative10x1ErasingD5() {
+ prepare(null, 10, 1, new int[]{5}, new int[0]);
+ testCodingWithBadInput(true);
+ testCodingWithBadOutput(false);
+ testCodingWithBadInput(true);
+ testCodingWithBadOutput(false);
+ }
+}
diff --git a/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/package-info.java b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/package-info.java
new file mode 100644
index 0000000..b2422af
--- /dev/null
+++ b/hadoop-hdds/erasurecode/src/test/java/org/apache/ozone/erasurecode/rawcoder/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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.
+ */
+
+/**
+ * Unit tests for rawencoders.
+ */
+package org.apache.ozone.erasurecode.rawcoder;
+
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
index f3c1da8..eb19e46 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
@@ -24,6 +24,7 @@
import org.apache.hadoop.hdds.annotation.InterfaceAudience;
import org.apache.hadoop.hdds.client.ContainerBlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
@@ -170,6 +171,13 @@
requestBuilder.setFactor(
((RatisReplicationConfig) replicationConfig).getReplicationFactor());
break;
+ case EC:
+ // We do not check for server support here, as this call is used only
+ // from OM which has the same software version as SCM.
+ // TODO: Rolling upgrade support needs to change this.
+ requestBuilder.setEcReplicationConfig(
+ ((ECReplicationConfig)replicationConfig).toProto());
+ break;
default:
throw new IllegalArgumentException(
"Unsupported replication type " + replicationConfig
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
index beb3b75..568985e 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
@@ -21,6 +21,9 @@
import com.google.protobuf.ServiceException;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicatedReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.GetScmInfoResponseProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.UpgradeFinalizationStatus;
@@ -113,6 +116,8 @@
import java.util.Optional;
import java.util.function.Consumer;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
+
/**
* This class is the client-side translator to translate the requests made on
* the {@link StorageContainerLocationProtocol} interface to the RPC server
@@ -357,18 +362,20 @@
@Override
public List<ContainerInfo> listContainer(long startContainerID, int count)
throws IOException {
- return listContainer(startContainerID, count, null, null);
+ return listContainer(startContainerID, count, null, null, null);
}
@Override
public List<ContainerInfo> listContainer(long startContainerID, int count,
HddsProtos.LifeCycleState state) throws IOException {
- return listContainer(startContainerID, count, state, null);
+ return listContainer(startContainerID, count, state, null, null);
}
@Override
public List<ContainerInfo> listContainer(long startContainerID, int count,
- HddsProtos.LifeCycleState state, HddsProtos.ReplicationFactor factor)
+ HddsProtos.LifeCycleState state,
+ HddsProtos.ReplicationType replicationType,
+ ReplicationConfig replicationConfig)
throws IOException {
Preconditions.checkState(startContainerID >= 0,
"Container ID cannot be negative.");
@@ -382,8 +389,18 @@
if (state != null) {
builder.setState(state);
}
- if (factor != null) {
- builder.setFactor(factor);
+ if (replicationConfig != null) {
+ if (replicationConfig.getReplicationType() == EC) {
+ builder.setType(EC);
+ builder.setEcReplicationConfig(
+ ((ECReplicationConfig)replicationConfig).toProto());
+ } else {
+ builder.setType(replicationConfig.getReplicationType());
+ builder.setFactor(((ReplicatedReplicationConfig)replicationConfig)
+ .getReplicationFactor());
+ }
+ } else if (replicationType != null) {
+ builder.setType(replicationType);
}
SCMListContainerRequestProto request = builder.build();
@@ -400,6 +417,15 @@
return containerList;
}
+ @Deprecated
+ @Override
+ public List<ContainerInfo> listContainer(long startContainerID, int count,
+ HddsProtos.LifeCycleState state, HddsProtos.ReplicationFactor factor)
+ throws IOException {
+ throw new UnsupportedOperationException("Should no longer be called from " +
+ "the client side");
+ }
+
/**
* Ask SCM to delete a container by name. SCM will remove
* the container mapping in its database.
diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
index 97e3dae..882f6c6 100644
--- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
+++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
@@ -265,6 +265,8 @@
optional string traceID = 3;
optional LifeCycleState state = 4;
optional ReplicationFactor factor = 5;
+ optional ReplicationType type = 6;
+ optional ECReplicationConfig ecReplicationConfig = 7;
}
message SCMListContainerResponseProto {
diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto
index 01c9a1e..540ac7b 100644
--- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto
+++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto
@@ -155,6 +155,8 @@
required int64 containerID = 1;
required int64 localID = 2;
optional uint64 blockCommitSequenceId = 3 [default = 0];
+ optional int32 replicaIndex = 4;
+
}
message KeyValue {
@@ -266,6 +268,7 @@
message CreateContainerRequestProto {
repeated KeyValue metadata = 2;
optional ContainerType containerType = 3 [default = KeyValueContainer];
+ optional int32 replicaIndex = 4;
}
message CreateContainerResponseProto {
diff --git a/hadoop-hdds/interface-client/src/main/proto/hdds.proto b/hadoop-hdds/interface-client/src/main/proto/hdds.proto
index caac440..c99274d 100644
--- a/hadoop-hdds/interface-client/src/main/proto/hdds.proto
+++ b/hadoop-hdds/interface-client/src/main/proto/hdds.proto
@@ -125,6 +125,8 @@
repeated uint32 memberOrders = 7;
optional uint64 creationTimeStamp = 8;
optional UUID suggestedLeaderID = 9;
+ repeated uint32 memberReplicaIndexes = 10;
+ optional ECReplicationConfig ecReplicationConfig = 11;
// TODO(runzhiwang): when leaderID is gone, specify 6 as the index of leaderID128
optional UUID leaderID128 = 100;
}
@@ -230,8 +232,9 @@
required string owner = 7;
optional int64 deleteTransactionId = 8;
optional int64 sequenceId = 9;
- required ReplicationFactor replicationFactor = 10;
+ optional ReplicationFactor replicationFactor = 10;
required ReplicationType replicationType = 11;
+ optional ECReplicationConfig ecReplicationConfig = 12;
}
message ContainerWithPipeline {
@@ -264,11 +267,27 @@
RATIS = 1;
STAND_ALONE = 2;
CHAINED = 3;
+ EC = 4;
+ NONE = -1; // Invalid Type
}
enum ReplicationFactor {
ONE = 1;
THREE = 3;
+ ZERO = 0; // Invalid Factor
+}
+
+message ECReplicationConfig {
+ required int32 data = 1;
+ required int32 parity = 2;
+ required string codec = 3;
+ required int32 ecChunkSize = 4;
+}
+
+message DefaultReplicationConfig {
+ required ReplicationType type = 1;
+ optional ReplicationFactor factor = 2;
+ optional ECReplicationConfig ecReplicationConfig = 3;
}
enum ScmOps {
@@ -394,6 +413,7 @@
required int64 sequenceID = 5;
required int64 keyCount = 6;
required int64 bytesUsed = 7;
+ optional int64 replicaIndex = 8;
}
message KeyContainerIDList {
diff --git a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
index ffed754..7c07154 100644
--- a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
+++ b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
@@ -228,6 +228,7 @@
optional int64 deleteTransactionId = 11;
optional uint64 blockCommitSequenceId = 12;
optional string originNodeId = 13;
+ optional int32 replicaIndex = 14;
}
message CommandStatusReportsProto {
diff --git a/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto b/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto
index facdf58..90ce7f7 100644
--- a/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto
+++ b/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto
@@ -139,9 +139,13 @@
required uint64 size = 1;
required uint32 numBlocks = 2;
required ReplicationType type = 3;
- required hadoop.hdds.ReplicationFactor factor = 4;
+ optional hadoop.hdds.ReplicationFactor factor = 4;
required string owner = 5;
optional ExcludeListProto excludeList = 7;
+
+ //used for EC replicaiton instead of the replication factor
+ optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 8;
+
}
/**
diff --git a/hadoop-hdds/pom.xml b/hadoop-hdds/pom.xml
index f7480a2..35248b7 100644
--- a/hadoop-hdds/pom.xml
+++ b/hadoop-hdds/pom.xml
@@ -45,6 +45,7 @@
<module>docs</module>
<module>config</module>
<module>test-utils</module>
+ <module>erasurecode</module>
</modules>
<repositories>
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java
index 9b94e20..dd9f71a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/AbstractContainerReportHandler.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.scm.container;
import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
@@ -132,7 +133,6 @@
final ContainerInfo containerInfo,
final ContainerReplicaProto replicaProto)
throws ContainerNotFoundException {
- final ContainerID containerId = containerInfo.containerID();
if (isHealthy(replicaProto::getState)) {
if (containerInfo.getSequenceId() <
@@ -140,30 +140,77 @@
containerInfo.updateSequenceId(
replicaProto.getBlockCommitSequenceId());
}
- List<ContainerReplica> otherReplicas =
- getOtherReplicas(containerId, datanodeDetails);
- long usedBytes = replicaProto.getUsed();
- long keyCount = replicaProto.getKeyCount();
- for (ContainerReplica r : otherReplicas) {
- // Open containers are generally growing in key count and size, the
- // overall size should be the min of all reported replicas.
- if (containerInfo.getState().equals(HddsProtos.LifeCycleState.OPEN)) {
- usedBytes = Math.min(usedBytes, r.getBytesUsed());
- keyCount = Math.min(keyCount, r.getKeyCount());
- } else {
- // Containers which are not open can only shrink in size, so use the
- // largest values reported.
- usedBytes = Math.max(usedBytes, r.getBytesUsed());
- keyCount = Math.max(keyCount, r.getKeyCount());
- }
+ if (containerInfo.getReplicationConfig().getReplicationType()
+ == HddsProtos.ReplicationType.EC) {
+ updateECContainerStats(containerInfo, replicaProto, datanodeDetails);
+ } else {
+ updateRatisContainerStats(containerInfo, replicaProto, datanodeDetails);
}
+ }
+ }
- if (containerInfo.getUsedBytes() != usedBytes) {
- containerInfo.setUsedBytes(usedBytes);
+ private void updateRatisContainerStats(ContainerInfo containerInfo,
+ ContainerReplicaProto newReplica, DatanodeDetails newSource)
+ throws ContainerNotFoundException {
+ List<ContainerReplica> otherReplicas =
+ getOtherReplicas(containerInfo.containerID(), newSource);
+ long usedBytes = newReplica.getUsed();
+ long keyCount = newReplica.getKeyCount();
+ for (ContainerReplica r : otherReplicas) {
+ usedBytes = calculateUsage(containerInfo, usedBytes, r.getBytesUsed());
+ keyCount = calculateUsage(containerInfo, keyCount, r.getKeyCount());
+ }
+ updateContainerUsedAndKeys(containerInfo, usedBytes, keyCount);
+ }
+
+ private void updateECContainerStats(ContainerInfo containerInfo,
+ ContainerReplicaProto newReplica, DatanodeDetails newSource)
+ throws ContainerNotFoundException {
+ int dataNum =
+ ((ECReplicationConfig)containerInfo.getReplicationConfig()).getData();
+ // The first EC index and the parity indexes must all be the same size
+ // while the other data indexes may be smaller due to partial stripes.
+ // When calculating the stats, we only use the first data and parity and
+ // ignore the others. We only need to run the check if we are processing
+ // the first data or parity replicas.
+ if (newReplica.getReplicaIndex() == 1
+ || newReplica.getReplicaIndex() > dataNum) {
+ List<ContainerReplica> otherReplicas =
+ getOtherReplicas(containerInfo.containerID(), newSource);
+ long usedBytes = newReplica.getUsed();
+ long keyCount = newReplica.getKeyCount();
+ for (ContainerReplica r : otherReplicas) {
+ if (r.getReplicaIndex() > 1 && r.getReplicaIndex() <= dataNum) {
+ // Ignore all data replicas except the first for stats
+ continue;
+ }
+ usedBytes = calculateUsage(containerInfo, usedBytes, r.getBytesUsed());
+ keyCount = calculateUsage(containerInfo, keyCount, r.getKeyCount());
}
- if (containerInfo.getNumberOfKeys() != keyCount) {
- containerInfo.setNumberOfKeys(keyCount);
- }
+ updateContainerUsedAndKeys(containerInfo, usedBytes, keyCount);
+ }
+ }
+
+ private long calculateUsage(ContainerInfo containerInfo, long lastValue,
+ long thisValue) {
+ if (containerInfo.getState().equals(HddsProtos.LifeCycleState.OPEN)) {
+ // Open containers are generally growing in key count and size, the
+ // overall size should be the min of all reported replicas.
+ return Math.min(lastValue, thisValue);
+ } else {
+ // Containers which are not open can only shrink in size, so use the
+ // largest values reported.
+ return Math.max(lastValue, thisValue);
+ }
+ }
+
+ private void updateContainerUsedAndKeys(ContainerInfo containerInfo,
+ long usedBytes, long keyCount) {
+ if (containerInfo.getUsedBytes() != usedBytes) {
+ containerInfo.setUsedBytes(usedBytes);
+ }
+ if (containerInfo.getNumberOfKeys() != keyCount) {
+ containerInfo.setNumberOfKeys(keyCount);
}
}
@@ -308,6 +355,7 @@
.setOriginNodeId(UUID.fromString(replicaProto.getOriginNodeId()))
.setSequenceId(replicaProto.getBlockCommitSequenceId())
.setKeyCount(replicaProto.getKeyCount())
+ .setReplicaIndex(replicaProto.getReplicaIndex())
.setBytesUsed(replicaProto.getUsed())
.build();
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
index 1c39efd..e4076d9 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java
@@ -35,6 +35,7 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto;
@@ -261,7 +262,8 @@
"Cannot allocate container, negative container id" +
" generated. %s.", uniqueId);
final ContainerID containerID = ContainerID.valueOf(uniqueId);
- final ContainerInfoProto containerInfo = ContainerInfoProto.newBuilder()
+ final ContainerInfoProto.Builder containerInfoBuilder = ContainerInfoProto
+ .newBuilder()
.setState(LifeCycleState.OPEN)
.setPipelineID(pipeline.getId().getProtobuf())
.setUsedBytes(0)
@@ -270,11 +272,17 @@
.setOwner(owner)
.setContainerID(containerID.getId())
.setDeleteTransactionId(0)
- .setReplicationFactor(
- ReplicationConfig.getLegacyFactor(pipeline.getReplicationConfig()))
- .setReplicationType(pipeline.getType())
- .build();
- containerStateManager.addContainer(containerInfo);
+ .setReplicationType(pipeline.getType());
+
+ if (pipeline.getReplicationConfig() instanceof ECReplicationConfig) {
+ containerInfoBuilder.setEcReplicationConfig(
+ ((ECReplicationConfig) pipeline.getReplicationConfig()).toProto());
+ } else {
+ containerInfoBuilder.setReplicationFactor(
+ ReplicationConfig.getLegacyFactor(pipeline.getReplicationConfig()));
+ }
+
+ containerStateManager.addContainer(containerInfoBuilder.build());
scmContainerManagerMetrics.incNumSuccessfulCreateContainers();
return containerStateManager.getContainer(containerID);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplica.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplica.java
index 0cb1bdd..727fdf8 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplica.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplica.java
@@ -37,21 +37,28 @@
private final ContainerReplicaProto.State state;
private final DatanodeDetails datanodeDetails;
private final UUID placeOfBirth;
+ private final int replicaIndex;
private Long sequenceId;
private final long keyCount;
private final long bytesUsed;
- private ContainerReplica(final ContainerID containerID,
- final ContainerReplicaProto.State state, final DatanodeDetails datanode,
- final UUID originNodeId, long keyNum, long dataSize) {
+ private ContainerReplica(
+ final ContainerID containerID,
+ final ContainerReplicaProto.State state,
+ final int replicaIndex,
+ final DatanodeDetails datanode,
+ final UUID originNodeId,
+ long keyNum,
+ long dataSize) {
this.containerID = containerID;
this.state = state;
this.datanodeDetails = datanode;
this.placeOfBirth = originNodeId;
this.keyCount = keyNum;
this.bytesUsed = dataSize;
+ this.replicaIndex = replicaIndex;
}
private void setSequenceId(Long seqId) {
@@ -181,6 +188,7 @@
private Long sequenceId;
private long bytesUsed;
private long keyCount;
+ private int replicaIndex;
/**
* Set Container Id.
@@ -212,6 +220,12 @@
return this;
}
+ public ContainerReplicaBuilder setReplicaIndex(
+ int index) {
+ this.replicaIndex = index;
+ return this;
+ }
+
/**
* Set replica origin node id.
*
@@ -257,11 +271,15 @@
Preconditions.checkNotNull(datanode,
"DatanodeDetails can't be null");
ContainerReplica replica = new ContainerReplica(
- containerID, state, datanode,
+ containerID, state, replicaIndex, datanode,
Optional.ofNullable(placeOfBirth).orElse(datanode.getUuid()),
keyCount, bytesUsed);
Optional.ofNullable(sequenceId).ifPresent(replica::setSequenceId);
return replica;
}
}
+
+ public int getReplicaIndex() {
+ return replicaIndex;
+ }
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
index 6e49db9..4207482 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
@@ -477,6 +477,13 @@
}
}
+ if (container.getReplicationType() == HddsProtos.ReplicationType.EC) {
+ // TODO We do not support replicating EC containers as yet, so at this
+ // point, after handing the closing etc states, we just return.
+ // EC Support will be added later.
+ return;
+ }
+
/*
* Before processing the container we have to reconcile the
* inflightReplication and inflightDeletion actions.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/ContainerPlacementPolicyFactory.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/ContainerPlacementPolicyFactory.java
index bf0ea7c..9c64566 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/ContainerPlacementPolicyFactory.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/ContainerPlacementPolicyFactory.java
@@ -30,7 +30,8 @@
/**
* A factory to create container placement instance based on configuration
- * property {@link ScmConfigKeys#OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY}.
+ * property {@link ScmConfigKeys#OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY} and
+ * property {@link ScmConfigKeys#OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY}.
*/
public final class ContainerPlacementPolicyFactory {
private static final Logger LOG =
@@ -39,6 +40,9 @@
private static final Class<? extends PlacementPolicy>
OZONE_SCM_CONTAINER_PLACEMENT_IMPL_DEFAULT =
SCMContainerPlacementRandom.class;
+ private static final Class<? extends PlacementPolicy>
+ OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_DEFAULT =
+ SCMContainerPlacementRackScatter.class;
private ContainerPlacementPolicyFactory() {
}
@@ -52,6 +56,28 @@
.getClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
OZONE_SCM_CONTAINER_PLACEMENT_IMPL_DEFAULT,
PlacementPolicy.class);
+ return getPolicyInternal(placementClass, conf, nodeManager, clusterMap,
+ fallback, metrics);
+ }
+
+ public static PlacementPolicy getECPolicy(
+ ConfigurationSource conf, final NodeManager nodeManager,
+ NetworkTopology clusterMap, final boolean fallback,
+ SCMContainerPlacementMetrics metrics) throws SCMException {
+ // TODO: Change default placement policy for EC
+ final Class<? extends PlacementPolicy> placementClass = conf
+ .getClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY,
+ OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_DEFAULT,
+ PlacementPolicy.class);
+ return getPolicyInternal(placementClass, conf, nodeManager, clusterMap,
+ fallback, metrics);
+ }
+
+ private static PlacementPolicy getPolicyInternal(
+ Class<? extends PlacementPolicy> placementClass,
+ ConfigurationSource conf, final NodeManager nodeManager,
+ NetworkTopology clusterMap, final boolean fallback,
+ SCMContainerPlacementMetrics metrics) throws SCMException {
Constructor<? extends PlacementPolicy> constructor;
try {
constructor = placementClass.getDeclaredConstructor(NodeManager.class,
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementMetrics.java
index 22bdf21..aaa7855 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementMetrics.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementMetrics.java
@@ -53,8 +53,18 @@
public SCMContainerPlacementMetrics() {
}
+ /**
+ * Return the existing instance of SCMContainerPlacementMetrics if it is
+ * registered in the metrics System, otherwise create a new instance, register
+ * it and return it.
+ * @return A new or existing SCMContainerPlacementMetrics object
+ */
public static SCMContainerPlacementMetrics create() {
MetricsSystem ms = DefaultMetricsSystem.instance();
+ MetricsSource existingSource = ms.getSource(SOURCE_NAME);
+ if (existingSource != null) {
+ return (SCMContainerPlacementMetrics)existingSource;
+ }
registry = new MetricsRegistry(RECORD_INFO);
return ms.register(SOURCE_NAME, "SCM Container Placement Metrics",
new SCMContainerPlacementMetrics());
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java
new file mode 100644
index 0000000..1902d51
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java
@@ -0,0 +1,329 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.container.placement.algorithms;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.SCMCommonPlacementPolicy;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.net.Node;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Container placement policy that scatter datanodes on different racks
+ * , together with the space to satisfy the size constraints.
+ * <p>
+ * This placement policy will try to distribute datanodes on as many racks as
+ * possible.
+ * <p>
+ * This implementation applies to network topology like "/rack/node". Don't
+ * recommend to use this if the network topology has more layers.
+ * <p>
+ */
+public final class SCMContainerPlacementRackScatter
+ extends SCMCommonPlacementPolicy {
+ @VisibleForTesting
+ public static final Logger LOG =
+ LoggerFactory.getLogger(SCMContainerPlacementRackScatter.class);
+ private final NetworkTopology networkTopology;
+ private static final int RACK_LEVEL = 1;
+ // OUTER_LOOP is to avoid infinity choose on all racks
+ private static final int OUTER_LOOP_MAX_RETRY = 3;
+ // INNER_LOOP is to choose node in each rack
+ private static final int INNER_LOOP_MAX_RETRY = 5;
+ private final SCMContainerPlacementMetrics metrics;
+
+ /**
+ * Constructs a Container Placement with rack awareness.
+ *
+ * @param nodeManager Node Manager
+ * @param conf Configuration
+ */
+ public SCMContainerPlacementRackScatter(final NodeManager nodeManager,
+ final ConfigurationSource conf, final NetworkTopology networkTopology,
+ boolean fallback, final SCMContainerPlacementMetrics metrics) {
+ super(nodeManager, conf);
+ this.networkTopology = networkTopology;
+ this.metrics = metrics;
+ }
+
+ /**
+ * Called by SCM to choose datanodes.
+ *
+ *
+ * @param excludedNodes - list of the datanodes to exclude.
+ * @param favoredNodes - list of nodes preferred. This is a hint to the
+ * allocator, whether the favored nodes will be used
+ * depends on whether the nodes meets the allocator's
+ * requirement.
+ * @param nodesRequired - number of datanodes required.
+ * @param dataSizeRequired - size required for the container.
+ * @param metadataSizeRequired - size required for Ratis metadata.
+ * @return List of datanodes.
+ * @throws SCMException SCMException
+ */
+ @Override
+ public List<DatanodeDetails> chooseDatanodes(
+ List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes,
+ int nodesRequired, long metadataSizeRequired, long dataSizeRequired)
+ throws SCMException {
+ Preconditions.checkArgument(nodesRequired > 0);
+ metrics.incrDatanodeRequestCount(nodesRequired);
+ int excludedNodesCount = excludedNodes == null ? 0 : excludedNodes.size();
+ List<Node> availableNodes = networkTopology.getNodes(
+ networkTopology.getMaxLevel());
+ int totalNodesCount = availableNodes.size();
+ if (excludedNodes != null) {
+ availableNodes.removeAll(excludedNodes);
+ }
+ if (availableNodes.size() < nodesRequired) {
+ throw new SCMException("No enough datanodes to choose. " +
+ "TotalNode = " + totalNodesCount +
+ " AvailableNode = " + availableNodes.size() +
+ " RequiredNode = " + nodesRequired +
+ " ExcludedNode = " + excludedNodesCount, null);
+ }
+
+ List<DatanodeDetails> mutableFavoredNodes = new ArrayList<>();
+ if (favoredNodes != null) {
+ // Generate mutableFavoredNodes, only stores valid favoredNodes
+ for (DatanodeDetails datanodeDetails : favoredNodes) {
+ if (isValidNode(datanodeDetails, metadataSizeRequired,
+ dataSizeRequired)) {
+ mutableFavoredNodes.add(datanodeDetails);
+ }
+ }
+ Collections.shuffle(mutableFavoredNodes);
+ }
+ if (excludedNodes != null) {
+ mutableFavoredNodes.removeAll(excludedNodes);
+ }
+
+ // For excluded nodes, we sort their racks at rear
+ List<Node> racks = getAllRacks();
+ if (excludedNodes != null && excludedNodes.size() > 0) {
+ racks = sortRackWithExcludedNodes(racks, excludedNodes);
+ }
+
+ List<Node> toChooseRacks = new LinkedList<>(racks);
+ List<DatanodeDetails> chosenNodes = new ArrayList<>();
+ List<Node> unavailableNodes = new ArrayList<>();
+ Set<Node> skippedRacks = new HashSet<>();
+ if (excludedNodes != null) {
+ unavailableNodes.addAll(excludedNodes);
+ }
+
+ // If the result doesn't change after retryCount, we return with exception
+ int retryCount = 0;
+ while (nodesRequired > 0) {
+ if (retryCount > OUTER_LOOP_MAX_RETRY) {
+ throw new SCMException("No satisfied datanode to meet the" +
+ " excludedNodes and affinityNode constrains.", null);
+ }
+ int chosenListSize = chosenNodes.size();
+
+ // Refill toChooseRacks, we put skippedRacks in front of toChooseRacks
+ // for a even distribution
+ toChooseRacks.addAll(racks);
+ if (!skippedRacks.isEmpty()) {
+ toChooseRacks.removeAll(skippedRacks);
+ toChooseRacks.addAll(0, skippedRacks);
+ skippedRacks.clear();
+ }
+
+ if (mutableFavoredNodes.size() > 0) {
+ List<Node> chosenFavoredNodesInForLoop = new ArrayList<>();
+ for (DatanodeDetails favoredNode : mutableFavoredNodes) {
+ Node curRack = getRackOfDatanodeDetails(favoredNode);
+ if (toChooseRacks.contains(curRack)) {
+ chosenNodes.add(favoredNode);
+ toChooseRacks.remove(curRack);
+ chosenFavoredNodesInForLoop.add(favoredNode);
+ unavailableNodes.add(favoredNode);
+ nodesRequired--;
+ if (nodesRequired == 0) {
+ break;
+ }
+ }
+ }
+ mutableFavoredNodes.removeAll(chosenFavoredNodesInForLoop);
+ }
+
+ // If satisfied by favored nodes, return then.
+ if (nodesRequired == 0) {
+ break;
+ }
+
+ for (Node rack : toChooseRacks) {
+ if (rack == null) {
+ // TODO: need to recheck why null coming here.
+ continue;
+ }
+ Node node = chooseNode(rack.getNetworkFullPath(), unavailableNodes,
+ metadataSizeRequired, dataSizeRequired);
+ if (node != null) {
+ chosenNodes.add((DatanodeDetails) node);
+ mutableFavoredNodes.remove(node);
+ unavailableNodes.add(node);
+ nodesRequired--;
+ if (nodesRequired == 0) {
+ break;
+ }
+ } else {
+ // Store the skipped racks to check them first in next outer loop
+ skippedRacks.add(rack);
+ }
+ }
+ // Clear toChooseRacks for this loop
+ toChooseRacks.clear();
+
+ // If chosenNodes not changed, increase the retryCount
+ if (chosenListSize == chosenNodes.size()) {
+ retryCount++;
+ } else {
+ // If chosenNodes changed, reset the retryCount
+ retryCount = 0;
+ }
+ }
+ ContainerPlacementStatus placementStatus =
+ validateContainerPlacement(chosenNodes, nodesRequired);
+ if (!placementStatus.isPolicySatisfied()) {
+ LOG.warn("ContainerPlacementPolicy not met, currentRacks is {}," +
+ " desired racks is {}.", placementStatus.actualPlacementCount(),
+ placementStatus.expectedPlacementCount());
+ }
+ return chosenNodes;
+ }
+
+ @Override
+ public DatanodeDetails chooseNode(List<DatanodeDetails> healthyNodes) {
+ return null;
+ }
+
+ /**
+ * Choose a datanode which meets the requirements. If there is no node which
+ * meets all the requirements, there is fallback chosen process depending on
+ * whether fallback is allowed when this class is instantiated.
+ *
+ *
+ * @param scope - the rack we are searching nodes under
+ * @param excludedNodes - list of the datanodes to excluded. Can be null.
+ * @param dataSizeRequired - size required for the container.
+ * @param metadataSizeRequired - size required for Ratis metadata.
+ * @return the chosen datanode.
+ */
+ private Node chooseNode(String scope, List<Node> excludedNodes,
+ long metadataSizeRequired, long dataSizeRequired) {
+ int maxRetry = INNER_LOOP_MAX_RETRY;
+ while (true) {
+ metrics.incrDatanodeChooseAttemptCount();
+ Node node = networkTopology.chooseRandom(scope, excludedNodes);
+ if (node != null) {
+ DatanodeDetails datanodeDetails = (DatanodeDetails) node;
+ if (isValidNode(datanodeDetails, metadataSizeRequired,
+ dataSizeRequired)) {
+ metrics.incrDatanodeChooseSuccessCount();
+ return node;
+ }
+ // exclude the unavailable node for the following retries.
+ excludedNodes.add(node);
+ } else {
+ LOG.debug("Failed to find the datanode for container. excludedNodes: " +
+ "{}, rack {}", excludedNodes, scope);
+ }
+ maxRetry--;
+ if (maxRetry == 0) {
+ // avoid the infinite loop
+ LOG.info("No satisfied datanode to meet the constraints. "
+ + "Metadatadata size required: {} Data size required: {}, scope "
+ + "{}, excluded nodes {}",
+ metadataSizeRequired, dataSizeRequired, scope, excludedNodes);
+ return null;
+ }
+ }
+ }
+
+ /**
+ * For EC placement policy, desired rack count would be equal to the num of
+ * Replicas.
+ * @param numReplicas - num of Replicas.
+ * @return required rack count.
+ */
+ @Override
+ protected int getRequiredRackCount(int numReplicas) {
+ if (networkTopology == null) {
+ return 1;
+ }
+ int maxLevel = networkTopology.getMaxLevel();
+ int numRacks = networkTopology.getNumOfNodes(maxLevel - 1);
+ // Return the num of Rack if numRack less than numReplicas
+ return Math.min(numRacks, numReplicas);
+ }
+
+ private Node getRackOfDatanodeDetails(DatanodeDetails datanodeDetails) {
+ String location = datanodeDetails.getNetworkLocation();
+ return networkTopology.getNode(location);
+ }
+
+ /**
+ * For the rack holding excludedNodes, we prefer not to choose from these
+ * racks, so we sort these racks at rear.
+ * @param racks
+ * @param excludedNodes
+ * @return
+ */
+ private List<Node> sortRackWithExcludedNodes(List<Node> racks,
+ List<DatanodeDetails> excludedNodes) {
+ Set<Node> lessPreferredRacks = excludedNodes.stream()
+ .map(node -> networkTopology.getAncestor(node, RACK_LEVEL))
+ // Dead Nodes have been removed from the topology and so have a
+ // null rack. We need to exclude those from the rack list.
+ .filter(node -> node != null)
+ .collect(Collectors.toSet());
+ List <Node> result = new ArrayList<>();
+ for (Node rack : racks) {
+ if (!lessPreferredRacks.contains(rack)) {
+ result.add(rack);
+ }
+ }
+ result.addAll(lessPreferredRacks);
+ return result;
+ }
+
+ private List<Node> getAllRacks() {
+ int rackLevel = networkTopology.getMaxLevel() - 1;
+ List<Node> racks = networkTopology.getNodes(rackLevel);
+ Collections.shuffle(racks);
+ return racks;
+ }
+
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java
index 75abd53..6fed1e2 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java
@@ -213,6 +213,9 @@
new ArrayList<>();
for (HddsProtos.ReplicationFactor factor : HddsProtos.ReplicationFactor
.values()) {
+ if (factor == ReplicationFactor.ZERO) {
+ continue; // Ignore it.
+ }
final ReplicationConfig replicationConfig =
ReplicationConfig.fromProtoTypeAndFactor(type, factor);
if (skipCreation(replicationConfig, autoCreateFactorOne)) {
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/ECPipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/ECPipelineProvider.java
new file mode 100644
index 0000000..11c23f1
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/ECPipelineProvider.java
@@ -0,0 +1,126 @@
+/**
+ * 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.pipeline;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Class to create pipelines for EC containers.
+ */
+public class ECPipelineProvider extends PipelineProvider<ECReplicationConfig> {
+
+ // TODO - EC Placement Policy. Standard Network Aware topology will not work
+ // for EC as it stands. We may want an "as many racks as possible"
+ // policy. HDDS-5326.
+
+ private final ConfigurationSource conf;
+ private final PlacementPolicy placementPolicy;
+ private final long containerSizeBytes;
+
+ public ECPipelineProvider(NodeManager nodeManager,
+ PipelineStateManager stateManager,
+ ConfigurationSource conf,
+ PlacementPolicy placementPolicy) {
+ super(nodeManager, stateManager);
+ this.conf = conf;
+ this.placementPolicy = placementPolicy;
+ this.containerSizeBytes = (long) this.conf
+ .getStorageSize(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+ ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
+ }
+
+ @Override
+ public synchronized Pipeline create(ECReplicationConfig replicationConfig)
+ throws IOException {
+ return create(replicationConfig, Collections.emptyList(),
+ Collections.emptyList());
+ }
+
+ @Override
+ protected Pipeline create(ECReplicationConfig replicationConfig,
+ List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+ throws IOException {
+ List<DatanodeDetails> dns = placementPolicy
+ .chooseDatanodes(excludedNodes, favoredNodes,
+ replicationConfig.getRequiredNodes(), 0, this.containerSizeBytes);
+ return create(replicationConfig, dns);
+ }
+
+ @Override
+ protected Pipeline create(ECReplicationConfig replicationConfig,
+ List<DatanodeDetails> nodes) {
+
+ Map<DatanodeDetails, Integer> dnIndexes = new HashMap<>();
+ int ecIndex = 1;
+ for (DatanodeDetails dn : nodes) {
+ dnIndexes.put(dn, ecIndex);
+ ecIndex++;
+ }
+
+ return createPipelineInternal(replicationConfig, nodes, dnIndexes);
+ }
+
+ @Override
+ public Pipeline createForRead(
+ ECReplicationConfig replicationConfig,
+ Set<ContainerReplica> replicas) {
+ Map<DatanodeDetails, Integer> map = new HashMap<>();
+ List<DatanodeDetails> dns = new ArrayList<>(replicas.size());
+
+ for (ContainerReplica r : replicas) {
+ map.put(r.getDatanodeDetails(), r.getReplicaIndex());
+ dns.add(r.getDatanodeDetails());
+ }
+ return createPipelineInternal(replicationConfig, dns, map);
+ }
+
+ private Pipeline createPipelineInternal(ECReplicationConfig repConfig,
+ List<DatanodeDetails> dns, Map<DatanodeDetails, Integer> indexes) {
+ return Pipeline.newBuilder()
+ .setId(PipelineID.randomId())
+ .setState(Pipeline.PipelineState.ALLOCATED)
+ .setReplicationConfig(repConfig)
+ .setNodes(dns)
+ .setReplicaIndexes(indexes)
+ .build();
+ }
+
+ @Override
+ protected void close(Pipeline pipeline) throws IOException {
+ }
+
+ @Override
+ protected void shutdown() {
+ }
+
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java
index b58716b..780a4ee 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java
@@ -23,6 +23,11 @@
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicyFactory;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementMetrics;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.ha.SCMContext;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.server.events.EventPublisher;
@@ -32,6 +37,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Set;
/**
* Creates pipeline based on replication type.
@@ -50,6 +56,18 @@
new RatisPipelineProvider(nodeManager,
stateManager, conf,
eventPublisher, scmContext));
+ PlacementPolicy ecPlacementPolicy;
+ try {
+ ecPlacementPolicy = ContainerPlacementPolicyFactory.getECPolicy(conf,
+ nodeManager, nodeManager.getClusterNetworkTopologyMap(), true,
+ SCMContainerPlacementMetrics.create());
+ } catch (SCMException e) {
+ throw new RuntimeException("Unable to get the container placement policy",
+ e);
+ }
+ providers.put(ReplicationType.EC,
+ new ECPipelineProvider(nodeManager, stateManager, conf,
+ ecPlacementPolicy));
}
protected PipelineFactory() {
@@ -64,12 +82,12 @@
}
public Pipeline create(
- ReplicationConfig replicationConfig
- )
+ ReplicationConfig replicationConfig, List<DatanodeDetails> excludedNodes,
+ List<DatanodeDetails> favoredNodes)
throws IOException {
return providers
.get(replicationConfig.getReplicationType())
- .create(replicationConfig);
+ .create(replicationConfig, excludedNodes, favoredNodes);
}
public Pipeline create(ReplicationConfig replicationConfig,
@@ -79,6 +97,12 @@
.create(replicationConfig, nodes);
}
+ public Pipeline createForRead(ReplicationConfig replicationConfig,
+ Set<ContainerReplica> replicas) {
+ return providers.get(replicationConfig.getReplicationType())
+ .createForRead(replicationConfig, replicas);
+ }
+
public void close(ReplicationType type, Pipeline pipeline)
throws IOException {
providers.get(type).close(pipeline);
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java
index b96d616..6a50876 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java
@@ -23,10 +23,12 @@
import java.util.Collection;
import java.util.List;
import java.util.NavigableSet;
+import java.util.Set;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.utils.db.Table;
/**
@@ -40,10 +42,18 @@
throws IOException;
Pipeline createPipeline(
+ ReplicationConfig replicationConfig, List<DatanodeDetails> excludedNodes,
+ List<DatanodeDetails> favoredNodes) throws IOException;
+
+
+ Pipeline createPipeline(
ReplicationConfig replicationConfig,
List<DatanodeDetails> nodes
);
+ Pipeline createPipelineForRead(
+ ReplicationConfig replicationConfig, Set<ContainerReplica> replicas);
+
Pipeline getPipeline(PipelineID pipelineID) throws PipelineNotFoundException;
boolean containsPipeline(PipelineID pipelineID);
@@ -65,6 +75,17 @@
Collection<PipelineID> excludePipelines
);
+ /**
+ * Returns the count of pipelines meeting the given ReplicationConfig and
+ * state.
+ * @param replicationConfig The ReplicationConfig of the pipelines to count
+ * @param state The current state of the pipelines to count
+ * @return The count of pipelines meeting the above criteria
+ */
+ int getPipelineCount(
+ ReplicationConfig replicationConfig, Pipeline.PipelineState state
+ );
+
void addContainerToPipeline(PipelineID pipelineID, ContainerID containerID)
throws IOException;
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java
index 2168138..19b7a51 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java
@@ -31,6 +31,7 @@
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.ha.SCMContext;
import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
@@ -51,6 +52,7 @@
import java.time.Duration;
import java.time.Instant;
import java.util.Collection;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -151,6 +153,14 @@
public Pipeline createPipeline(
ReplicationConfig replicationConfig
) throws IOException {
+ return createPipeline(replicationConfig, Collections.emptyList(),
+ Collections.emptyList());
+ }
+
+ @Override
+ public Pipeline createPipeline(ReplicationConfig replicationConfig,
+ List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+ throws IOException {
if (!isPipelineCreationAllowed() && !factorOne(replicationConfig)) {
LOG.debug("Pipeline creation is not allowed until safe mode prechecks " +
"complete");
@@ -167,7 +177,8 @@
acquireWriteLock();
try {
- Pipeline pipeline = pipelineFactory.create(replicationConfig);
+ Pipeline pipeline = pipelineFactory.create(replicationConfig,
+ excludedNodes, favoredNodes);
stateManager.addPipeline(pipeline.getProtobufMessage(
ClientVersion.CURRENT_VERSION));
recordMetricsForPipeline(pipeline);
@@ -207,6 +218,12 @@
}
@Override
+ public Pipeline createPipelineForRead(
+ ReplicationConfig replicationConfig, Set<ContainerReplica> replicas) {
+ return pipelineFactory.createForRead(replicationConfig, replicas);
+ }
+
+ @Override
public Pipeline getPipeline(PipelineID pipelineID)
throws PipelineNotFoundException {
return stateManager.getPipeline(pipelineID);
@@ -248,6 +265,19 @@
}
@Override
+ /**
+ * Returns the count of pipelines meeting the given ReplicationConfig and
+ * state.
+ * @param replicationConfig The ReplicationConfig of the pipelines to count
+ * @param state The current state of the pipelines to count
+ * @return The count of pipelines meeting the above criteria
+ */
+ public int getPipelineCount(ReplicationConfig config,
+ Pipeline.PipelineState state) {
+ return stateManager.getPipelineCount(config, state);
+ }
+
+ @Override
public void addContainerToPipeline(
PipelineID pipelineID, ContainerID containerID) throws IOException {
// should not lock here, since no ratis operation happens.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java
index 3c00906..063ca76 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java
@@ -27,6 +27,7 @@
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.SCMCommonPlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.node.NodeStatus;
@@ -65,11 +66,20 @@
protected abstract Pipeline create(REPLICATION_CONFIG replicationConfig)
throws IOException;
+ protected abstract Pipeline create(REPLICATION_CONFIG replicationConfig,
+ List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+ throws IOException;
+
protected abstract Pipeline create(
REPLICATION_CONFIG replicationConfig,
List<DatanodeDetails> nodes
);
+ protected abstract Pipeline createForRead(
+ REPLICATION_CONFIG replicationConfig,
+ Set<ContainerReplica> replicas
+ );
+
protected abstract void close(Pipeline pipeline) throws IOException;
protected abstract void shutdown();
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java
index 4af64b5..6180158 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java
@@ -95,6 +95,11 @@
Collection<PipelineID> excludePipelines
);
+ int getPipelineCount(
+ ReplicationConfig replicationConfig,
+ Pipeline.PipelineState state
+ );
+
NavigableSet<ContainerID> getContainers(PipelineID pipelineID)
throws IOException;
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManagerImpl.java
index 0b93f4d..ff00e3b 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManagerImpl.java
@@ -188,6 +188,26 @@
}
}
+
+ /**
+ * Returns the count of pipelines meeting the given ReplicationConfig and
+ * state.
+ * @param replicationConfig The ReplicationConfig of the pipelines to count
+ * @param state The current state of the pipelines to count
+ * @return The count of pipelines meeting the above criteria
+ */
+ @Override
+ public int getPipelineCount(
+ ReplicationConfig replicationConfig,
+ Pipeline.PipelineState state) {
+ lock.readLock().lock();
+ try {
+ return pipelineStateMap.getPipelineCount(replicationConfig, state);
+ } finally {
+ lock.readLock().unlock();
+ }
+ }
+
@Override
public NavigableSet<ContainerID> getContainers(PipelineID pipelineID)
throws IOException {
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java
index 8b9d913..23351e5 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateMap.java
@@ -220,6 +220,36 @@
}
/**
+ * Get a count of pipelines with the given replicationConfig and state.
+ * This method is most efficient when getting a count for OPEN pipeline
+ * as the result can be obtained directly from the cached open list.
+ *
+ * @param replicationConfig - ReplicationConfig
+ * @param state - Required PipelineState
+ * @return Count of pipelines with the specified replication config and state
+ */
+ int getPipelineCount(ReplicationConfig replicationConfig,
+ PipelineState state) {
+ Preconditions
+ .checkNotNull(replicationConfig, "ReplicationConfig cannot be null");
+ Preconditions.checkNotNull(state, "Pipeline state cannot be null");
+
+ if (state == PipelineState.OPEN) {
+ return query2OpenPipelines.getOrDefault(
+ replicationConfig, Collections.EMPTY_LIST).size();
+ }
+
+ int count = 0;
+ for (Pipeline pipeline : pipelineMap.values()) {
+ if (pipeline.getReplicationConfig().equals(replicationConfig)
+ && pipeline.getPipelineState() == state) {
+ count++;
+ }
+ }
+ return count;
+ }
+
+ /**
* Get list of pipeline corresponding to specified replication type,
* replication factor and pipeline state.
*
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
index 4c18f20..43b2e01 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
@@ -19,7 +19,10 @@
package org.apache.hadoop.hdds.scm.pipeline;
import java.io.IOException;
+import java.util.Collections;
import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
@@ -27,6 +30,7 @@
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.ha.SCMContext;
@@ -134,6 +138,14 @@
@Override
public synchronized Pipeline create(RatisReplicationConfig replicationConfig)
throws IOException {
+ return create(replicationConfig, Collections.emptyList(),
+ Collections.emptyList());
+ }
+
+ @Override
+ public synchronized Pipeline create(RatisReplicationConfig replicationConfig,
+ List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+ throws IOException {
if (exceedPipelineNumberLimit(replicationConfig)) {
throw new SCMException("Ratis pipeline number meets the limit: " +
pipelineNumberLimit + " replicationConfig : " +
@@ -151,8 +163,8 @@
containerSizeBytes);
break;
case THREE:
- dns = placementPolicy.chooseDatanodes(null,
- null, factor.getNumber(), minRatisVolumeSizeBytes,
+ dns = placementPolicy.chooseDatanodes(excludedNodes,
+ favoredNodes, factor.getNumber(), minRatisVolumeSizeBytes,
containerSizeBytes);
break;
default:
@@ -201,6 +213,16 @@
}
@Override
+ public Pipeline createForRead(
+ RatisReplicationConfig replicationConfig,
+ Set<ContainerReplica> replicas) {
+ return create(replicationConfig, replicas
+ .stream()
+ .map(ContainerReplica::getDatanodeDetails)
+ .collect(Collectors.toList()));
+ }
+
+ @Override
public void shutdown() {
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java
index 43af98b..a61ed27 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java
@@ -20,12 +20,15 @@
import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
/**
* Implements Api for creating stand alone pipelines.
@@ -41,6 +44,14 @@
@Override
public Pipeline create(StandaloneReplicationConfig replicationConfig)
throws IOException {
+ return create(replicationConfig, Collections.emptyList(),
+ Collections.emptyList());
+ }
+
+ @Override
+ public Pipeline create(StandaloneReplicationConfig replicationConfig,
+ List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+ throws IOException {
List<DatanodeDetails> dns = pickNodesNotUsed(replicationConfig);
if (dns.size() < replicationConfig.getRequiredNodes()) {
String e = String
@@ -71,6 +82,15 @@
}
@Override
+ public Pipeline createForRead(StandaloneReplicationConfig replicationConfig,
+ Set<ContainerReplica> replicas) {
+ return create(replicationConfig, replicas
+ .stream()
+ .map(ContainerReplica::getDatanodeDetails)
+ .collect(Collectors.toList()));
+ }
+
+ @Override
public void close(Pipeline pipeline) throws IOException {
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java
index f130eed..7d25ee9 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdds.scm.pipeline;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
@@ -33,12 +34,16 @@
private final WritableContainerProvider<ReplicationConfig> ratisProvider;
private final WritableContainerProvider<ReplicationConfig> standaloneProvider;
+ private final WritableContainerProvider<ECReplicationConfig> ecProvider;
public WritableContainerFactory(StorageContainerManager scm) {
this.ratisProvider = new WritableRatisContainerProvider(
scm.getConfiguration(), scm.getPipelineManager(),
scm.getContainerManager(), scm.getPipelineChoosePolicy());
this.standaloneProvider = ratisProvider;
+ this.ecProvider = new WritableECContainerProvider(scm.getConfiguration(),
+ scm.getPipelineManager(), scm.getContainerManager(),
+ scm.getPipelineChoosePolicy());
}
public ContainerInfo getContainer(final long size,
@@ -50,6 +55,9 @@
.getContainer(size, repConfig, owner, excludeList);
case RATIS:
return ratisProvider.getContainer(size, repConfig, owner, excludeList);
+ case EC:
+ return ecProvider.getContainer(size, (ECReplicationConfig)repConfig,
+ owner, excludeList);
default:
throw new IOException(repConfig.getReplicationType()
+ " is an invalid replication type");
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java
new file mode 100644
index 0000000..117d957
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java
@@ -0,0 +1,239 @@
+/**
+ * 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.pipeline;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+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;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.PipelineChoosePolicy;
+import org.apache.hadoop.hdds.scm.PipelineRequestInformation;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.NavigableSet;
+
+import static org.apache.hadoop.hdds.conf.StorageUnit.BYTES;
+
+/**
+ * Writable Container provider to obtain a writable container for EC pipelines.
+ */
+public class WritableECContainerProvider
+ implements WritableContainerProvider<ECReplicationConfig> {
+
+ private static final Logger LOG = LoggerFactory
+ .getLogger(WritableECContainerProvider.class);
+
+ private final ConfigurationSource conf;
+ private final PipelineManager pipelineManager;
+ private final PipelineChoosePolicy pipelineChoosePolicy;
+ private final ContainerManager containerManager;
+ private final long containerSize;
+ private final WritableECContainerProviderConfig providerConfig;
+
+ public WritableECContainerProvider(ConfigurationSource conf,
+ PipelineManager pipelineManager, ContainerManager containerManager,
+ PipelineChoosePolicy pipelineChoosePolicy) {
+ this.conf = conf;
+ this.providerConfig =
+ conf.getObject(WritableECContainerProviderConfig.class);
+ this.pipelineManager = pipelineManager;
+ this.containerManager = containerManager;
+ this.pipelineChoosePolicy = pipelineChoosePolicy;
+ this.containerSize = getConfiguredContainerSize();
+ }
+
+ /**
+ *
+ * @param size The max size of block in bytes which will be written. This
+ * comes from Ozone Manager and will be the block size configured
+ * for the cluster. The client cannot pass any arbitrary value
+ * from this setting.
+ * @param repConfig The replication Config indicating the EC data and partiy
+ * block counts.
+ * @param owner The owner of the container
+ * @param excludeList A set of datanodes, container and pipelines which should
+ * not be considered.
+ * @return A containerInfo representing a block group with with space for the
+ * write, or null if no container can be allocated.
+ * @throws IOException
+ */
+ @Override
+ public ContainerInfo getContainer(final long size,
+ ECReplicationConfig repConfig, String owner, ExcludeList excludeList)
+ throws IOException {
+ // Bound this at a minimum of 1 byte in case a request is made for a very
+ // small size, which when divided by EC DataNum is zero.
+ long requiredSpace = Math.max(1, size / repConfig.getData());
+ synchronized (this) {
+ int openPipelineCount = pipelineManager.getPipelineCount(repConfig,
+ Pipeline.PipelineState.OPEN);
+ if (openPipelineCount < providerConfig.getMinimumPipelines()) {
+ try {
+ return allocateContainer(
+ repConfig, requiredSpace, owner, excludeList);
+ } catch (IOException e) {
+ LOG.warn("Unable to allocate a container for {} with {} existing "
+ + "containers", repConfig, openPipelineCount, e);
+ }
+ }
+ }
+ List<Pipeline> existingPipelines = pipelineManager.getPipelines(
+ repConfig, Pipeline.PipelineState.OPEN,
+ excludeList.getDatanodes(), excludeList.getPipelineIds());
+
+ PipelineRequestInformation pri =
+ PipelineRequestInformation.Builder.getBuilder()
+ .setSize(requiredSpace)
+ .build();
+ while (existingPipelines.size() > 0) {
+ Pipeline pipeline =
+ pipelineChoosePolicy.choosePipeline(existingPipelines, pri);
+ if (pipeline == null) {
+ LOG.warn("Unable to select a pipeline from {} in the list",
+ existingPipelines.size());
+ break;
+ }
+ synchronized (pipeline.getId()) {
+ try {
+ ContainerInfo containerInfo = getContainerFromPipeline(pipeline);
+ if (containerInfo == null
+ || !containerHasSpace(containerInfo, requiredSpace)) {
+ // This is O(n), which isn't great if there are a lot of pipelines
+ // and we keep finding pipelines without enough space.
+ existingPipelines.remove(pipeline);
+ pipelineManager.closePipeline(pipeline, true);
+ } else {
+ if (containerIsExcluded(containerInfo, excludeList)) {
+ existingPipelines.remove(pipeline);
+ } else {
+ return containerInfo;
+ }
+ }
+ } catch (PipelineNotFoundException | ContainerNotFoundException e) {
+ LOG.warn("Pipeline or container not found when selecting a writable "
+ + "container", e);
+ existingPipelines.remove(pipeline);
+ pipelineManager.closePipeline(pipeline, true);
+ }
+ }
+ }
+ // If we get here, all the pipelines we tried were no good. So try to
+ // allocate a new one and usePipelineManagerV2Impl.java it.
+ try {
+ synchronized (this) {
+ return allocateContainer(repConfig, requiredSpace, owner, excludeList);
+ }
+ } catch (IOException e) {
+ LOG.error("Unable to allocate a container for {} after trying all "
+ + "existing containers", repConfig, e);
+ return null;
+ }
+ }
+
+ private ContainerInfo allocateContainer(ReplicationConfig repConfig,
+ long size, String owner, ExcludeList excludeList) throws IOException {
+
+ List<DatanodeDetails> excludedNodes = Collections.emptyList();
+ if (excludeList.getDatanodes().size() > 0) {
+ excludedNodes = new ArrayList<>(excludeList.getDatanodes());
+ }
+
+ Pipeline newPipeline = pipelineManager.createPipeline(repConfig,
+ excludedNodes, Collections.emptyList());
+ ContainerInfo container =
+ containerManager.getMatchingContainer(size, owner, newPipeline);
+ pipelineManager.openPipeline(newPipeline.getId());
+ return container;
+ }
+
+ private boolean containerIsExcluded(ContainerInfo container,
+ ExcludeList excludeList) {
+ return excludeList.getContainerIds().contains(container.containerID());
+ }
+
+ private ContainerInfo getContainerFromPipeline(Pipeline pipeline)
+ throws IOException {
+ // Assume the container is still open if the below method returns it. On
+ // container FINALIZE, ContainerManager will remove the container from the
+ // pipeline list in PipelineManager. Finalize can be triggered by a DN
+ // sending a message that the container is full, or on close pipeline or
+ // on a stale / dead node event (via close pipeline).
+ NavigableSet<ContainerID> containers =
+ pipelineManager.getContainersInPipeline(pipeline.getId());
+ // Assume 1 container per pipeline for EC
+ if (containers.size() == 0) {
+ return null;
+ }
+ ContainerID containerID = containers.first();
+ return containerManager.getContainer(containerID);
+ }
+
+ private boolean containerHasSpace(ContainerInfo container, long size) {
+ // The size passed from OM will be the cluster block size. Therefore we
+ // just check if the container has enough free space to accommodate another
+ // full block.
+ return container.getUsedBytes() + size <= containerSize;
+ }
+
+ private long getConfiguredContainerSize() {
+ return (long) conf.getStorageSize(
+ ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+ ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, BYTES);
+ }
+
+ /**
+ * Class to hold configuration for WriteableECContainerProvider.
+ */
+ @ConfigGroup(prefix = "ozone.scm.ec")
+ public static class WritableECContainerProviderConfig {
+
+ @Config(key = "pipeline.minimum",
+ defaultValue = "5",
+ type = ConfigType.INT,
+ description = "The minimum number of pipelines to have open for each " +
+ "Erasure Coding configuration",
+ tags = ConfigTag.STORAGE)
+ private int minimumPipelines = 5;
+
+ public int getMinimumPipelines() {
+ return minimumPipelines;
+ }
+
+ public void setMinimumPipelines(int minPipelines) {
+ this.minimumPipelines = minPipelines;
+ }
+
+ }
+
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java
index 7b0fc67..bdf248b 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java
@@ -47,6 +47,7 @@
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher;
+import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature;
import org.apache.hadoop.ozone.common.BlockGroup;
import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics;
@@ -127,6 +128,17 @@
try {
switch (request.getCmdType()) {
case AllocateScmBlock:
+ if (scm.getLayoutVersionManager().needsFinalization() &&
+ !scm.getLayoutVersionManager()
+ .isAllowed(HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)
+ ) {
+ if (request.getAllocateScmBlockRequest().hasEcReplicationConfig()) {
+ throw new SCMException("Cluster is not finalized yet, it is"
+ + " not enabled to create blocks with Erasure Coded"
+ + " replication type.",
+ SCMException.ResultCodes.INTERNAL_ERROR);
+ }
+ }
response.setAllocateScmBlockResponse(allocateScmBlock(
request.getAllocateScmBlockRequest(), request.getVersion()));
break;
@@ -179,9 +191,10 @@
List<AllocatedBlock> allocatedBlocks =
impl.allocateBlock(request.getSize(),
request.getNumBlocks(),
- ReplicationConfig.fromProtoTypeAndFactor(
+ ReplicationConfig.fromProto(
request.getType(),
- request.getFactor()),
+ request.getFactor(),
+ request.getEcReplicationConfig()),
request.getOwner(),
ExcludeList.getFromProtoBuf(request.getExcludeList()));
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 26b7c0b..c2ba0ff 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
@@ -22,6 +22,8 @@
import com.google.protobuf.ServiceException;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.UpgradeFinalizationStatus;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos;
@@ -99,13 +101,16 @@
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.ha.RatisUtil;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.protocolPB.OzonePBHelper;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher;
+import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature;
import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics;
+import org.apache.hadoop.ozone.ClientVersion;
import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -118,6 +123,13 @@
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.PipelineResponseProto.Error.errorPipelineAlreadyExists;
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.PipelineResponseProto.Error.success;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.GetContainer;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.GetContainerWithPipeline;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.GetContainerWithPipelineBatch;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.GetExistContainerWithPipelinesInBatch;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.GetPipeline;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.ListContainer;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type.ListPipelines;
import static org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol.ADMIN_COMMAND_TYPE;
/**
@@ -133,6 +145,16 @@
private static final Logger LOG =
LoggerFactory.getLogger(
StorageContainerLocationProtocolServerSideTranslatorPB.class);
+ private static final String ERROR_LIST_CONTAINS_EC_REPLICATION_CONFIG =
+ "The returned list of containers contains containers with Erasure Coded"
+ + " replication type, which the client won't be able to understand."
+ + " Please upgrade the client to a version that supports Erasure"
+ + " Coded data, and retry!";
+ private static final String ERROR_RESPONSE_CONTAINS_EC_REPLICATION_CONFIG =
+ "The returned container data contains Erasure Coded replication"
+ + " information, which the client won't be able to understand."
+ + " Please upgrade the client to a version that supports Erasure"
+ + " Coded data, and retry!";
private final StorageContainerLocationProtocol impl;
private final StorageContainerManager scm;
@@ -170,9 +192,196 @@
scm.getScmHAManager().getRatisServer().triggerNotLeaderException(),
scm.getClientRpcPort(), scm.getScmId());
}
- return dispatcher
+ // After the request interceptor (now validator) framework is extended to
+ // this server interface, this should be removed and solved via new
+ // annotated interceptors.
+ boolean checkResponseForECRepConfig = false;
+ if (request.getVersion() <
+ ClientVersion.ERASURE_CODING_SUPPORT.toProtoValue()) {
+ if (request.getCmdType() == GetContainer
+ || request.getCmdType() == ListContainer
+ || request.getCmdType() == GetContainerWithPipeline
+ || request.getCmdType() == GetContainerWithPipelineBatch
+ || request.getCmdType() == GetExistContainerWithPipelinesInBatch
+ || request.getCmdType() == ListPipelines
+ || request.getCmdType() == GetPipeline) {
+
+ checkResponseForECRepConfig = true;
+ }
+ }
+ ScmContainerLocationResponse response = dispatcher
.processRequest(request, this::processRequest, request.getCmdType(),
request.getTraceID());
+ if (checkResponseForECRepConfig) {
+ try {
+ switch (response.getCmdType()) {
+ case GetContainer:
+ disallowECReplicationConfigInGetContainerResponse(response);
+ break;
+ case ListContainer:
+ disallowECReplicationConfigInListContainerResponse(response);
+ break;
+ case GetContainerWithPipeline:
+ disallowECReplicationConfigInGetContainerWithPipelineResponse(
+ response);
+ break;
+ case GetContainerWithPipelineBatch:
+ disallowECReplicationConfigInGetContainerWithPipelineBatchResponse(
+ response);
+ break;
+ case GetExistContainerWithPipelinesInBatch:
+ disallowECReplicationConfigInGetExistContainerWithPipelineBatchResp(
+ response);
+ break;
+ case ListPipelines:
+ disallowECReplicationConfigInListPipelinesResponse(response);
+ break;
+ case GetPipeline:
+ disallowECReplicationConfigInGetPipelineResponse(response);
+ break;
+ default:
+ }
+ } catch (SCMException e) {
+ throw new ServiceException(e);
+ }
+ }
+ return response;
+ }
+
+ private void disallowECReplicationConfigInListContainerResponse(
+ ScmContainerLocationResponse response) throws SCMException {
+ if (!response.hasScmListContainerResponse()) {
+ return;
+ }
+ for (HddsProtos.ContainerInfoProto containerInfo :
+ response.getScmListContainerResponse().getContainersList()) {
+ if (containerInfo.hasEcReplicationConfig()) {
+ throw new SCMException(ERROR_LIST_CONTAINS_EC_REPLICATION_CONFIG,
+ SCMException.ResultCodes.INTERNAL_ERROR);
+ }
+ }
+ }
+
+ private void disallowECReplicationConfigInGetContainerResponse(
+ ScmContainerLocationResponse response) throws SCMException {
+ if (!response.hasGetContainerResponse()) {
+ return;
+ }
+ if (!response.getGetContainerResponse().hasContainerInfo()) {
+ return;
+ }
+ if (response.getGetContainerResponse().getContainerInfo()
+ .hasEcReplicationConfig()) {
+ throw new SCMException(ERROR_RESPONSE_CONTAINS_EC_REPLICATION_CONFIG,
+ SCMException.ResultCodes.INTERNAL_ERROR);
+ }
+ }
+
+ private void disallowECReplicationConfigInGetContainerWithPipelineResponse(
+ ScmContainerLocationResponse response) throws SCMException {
+ if (!response.hasGetContainerWithPipelineResponse()) {
+ return;
+ }
+ if (!response.getGetContainerWithPipelineResponse()
+ .hasContainerWithPipeline()) {
+ return;
+ }
+ if (response.getGetContainerWithPipelineResponse()
+ .getContainerWithPipeline().hasContainerInfo()) {
+ HddsProtos.ContainerInfoProto containerInfo =
+ response.getGetContainerWithPipelineResponse()
+ .getContainerWithPipeline().getContainerInfo();
+ if (containerInfo.hasEcReplicationConfig()) {
+ throw new SCMException(ERROR_RESPONSE_CONTAINS_EC_REPLICATION_CONFIG,
+ SCMException.ResultCodes.INTERNAL_ERROR);
+ }
+ }
+ if (response.getGetContainerWithPipelineResponse()
+ .getContainerWithPipeline().hasPipeline()) {
+ HddsProtos.Pipeline pipeline =
+ response.getGetContainerWithPipelineResponse()
+ .getContainerWithPipeline().getPipeline();
+ if (pipeline.hasEcReplicationConfig()) {
+ throw new SCMException(ERROR_RESPONSE_CONTAINS_EC_REPLICATION_CONFIG,
+ SCMException.ResultCodes.INTERNAL_ERROR);
+ }
+ }
+ }
+
+ private void
+ disallowECReplicationConfigInGetContainerWithPipelineBatchResponse(
+ ScmContainerLocationResponse response) throws SCMException {
+ if (!response.hasGetContainerWithPipelineBatchResponse()) {
+ return;
+ }
+ List<HddsProtos.ContainerWithPipeline> cwps =
+ response.getGetContainerWithPipelineBatchResponse()
+ .getContainerWithPipelinesList();
+ checkForECReplicationConfigIn(cwps);
+ }
+
+ private void
+ disallowECReplicationConfigInGetExistContainerWithPipelineBatchResp(
+ ScmContainerLocationResponse response) throws SCMException {
+ if (!response.hasGetExistContainerWithPipelinesInBatchResponse()) {
+ return;
+ }
+ List<HddsProtos.ContainerWithPipeline> cwps =
+ response.getGetExistContainerWithPipelinesInBatchResponse()
+ .getContainerWithPipelinesList();
+ checkForECReplicationConfigIn(cwps);
+ }
+
+ private void checkForECReplicationConfigIn(
+ List<HddsProtos.ContainerWithPipeline> cwps)
+ throws SCMException {
+ for (HddsProtos.ContainerWithPipeline cwp : cwps) {
+ if (cwp.hasContainerInfo()) {
+ if (cwp.getContainerInfo().hasEcReplicationConfig()) {
+ throw new SCMException(ERROR_LIST_CONTAINS_EC_REPLICATION_CONFIG,
+ SCMException.ResultCodes.INTERNAL_ERROR);
+ }
+ }
+ if (cwp.hasPipeline()) {
+ if (cwp.getPipeline().hasEcReplicationConfig()) {
+ throw new SCMException(ERROR_LIST_CONTAINS_EC_REPLICATION_CONFIG,
+ SCMException.ResultCodes.INTERNAL_ERROR);
+ }
+ }
+ }
+ }
+
+ private void disallowECReplicationConfigInListPipelinesResponse(
+ ScmContainerLocationResponse response) throws SCMException {
+ if (!response.hasListPipelineResponse()) {
+ return;
+ }
+ for (HddsProtos.Pipeline pipeline :
+ response.getListPipelineResponse().getPipelinesList()) {
+ if (pipeline.hasEcReplicationConfig()) {
+ throw new SCMException("The returned list of pipelines contains"
+ + " pipelines with Erasure Coded replication type, which the"
+ + " client won't be able to understand."
+ + " Please upgrade the client to a version that supports Erasure"
+ + " Coded data, and retry!",
+ SCMException.ResultCodes.INTERNAL_ERROR);
+ }
+ }
+ }
+
+ private void disallowECReplicationConfigInGetPipelineResponse(
+ ScmContainerLocationResponse response) throws SCMException {
+ if (!response.hasGetPipelineResponse()) {
+ return;
+ }
+ if (response.getPipelineResponse().getPipeline().hasEcReplicationConfig()) {
+ throw new SCMException("The returned pipeline data contains"
+ + " Erasure Coded replication information, which the client won't"
+ + " be able to understand."
+ + " Please upgrade the client to a version that supports Erasure"
+ + " Coded data, and retry!",
+ SCMException.ResultCodes.INTERNAL_ERROR);
+ }
}
@SuppressWarnings("checkstyle:methodlength")
@@ -249,6 +458,18 @@
request.getScmCloseContainerRequest()))
.build();
case AllocatePipeline:
+ if (scm.getLayoutVersionManager().needsFinalization() &&
+ !scm.getLayoutVersionManager().isAllowed(
+ HDDSLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)
+ ) {
+ if (request.getPipelineRequest().getReplicationType() ==
+ HddsProtos.ReplicationType.EC) {
+ throw new SCMException("Cluster is not finalized yet, it is"
+ + " not enabled to create pipelines with Erasure Coded"
+ + " replication type.",
+ SCMException.ResultCodes.INTERNAL_ERROR);
+ }
+ }
return ScmContainerLocationResponse.newBuilder()
.setCmdType(request.getCmdType())
.setStatus(Status.OK)
@@ -528,14 +749,40 @@
count = request.getCount();
HddsProtos.LifeCycleState state = null;
HddsProtos.ReplicationFactor factor = null;
+ HddsProtos.ReplicationType replicationType = null;
+ ReplicationConfig repConfig = null;
if (request.hasState()) {
state = request.getState();
}
- if (request.hasFactor()) {
+ if (request.hasType()) {
+ replicationType = request.getType();
+ }
+ if (replicationType != null) {
+ // This must come from an upgraded client as the older version never
+ // passed Type. Therefore, we must check for replicationConfig.
+ if (replicationType == HddsProtos.ReplicationType.EC) {
+ if (request.hasEcReplicationConfig()) {
+ repConfig = new ECReplicationConfig(request.getEcReplicationConfig());
+ }
+ } else {
+ if (request.hasFactor()) {
+ repConfig = ReplicationConfig
+ .fromProtoTypeAndFactor(request.getType(), request.getFactor());
+ }
+ }
+ } else if (request.hasFactor()) {
factor = request.getFactor();
}
- List<ContainerInfo> containerList =
- impl.listContainer(startContainerID, count, state, factor);
+ List<ContainerInfo> containerList;
+ if (factor != null) {
+ // Call from a legacy client
+ containerList =
+ impl.listContainer(startContainerID, count, state, factor);
+ } else {
+ containerList =
+ impl.listContainer(startContainerID, count, state, replicationType,
+ repConfig);
+ }
SCMListContainerResponseProto.Builder builder =
SCMListContainerResponseProto.newBuilder();
for (ContainerInfo container : containerList) {
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 a4d1ae4..3feca15 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
@@ -29,7 +29,6 @@
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hdds.client.ReplicationConfig;
-import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -93,6 +92,7 @@
import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
+import java.util.stream.Stream;
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StorageContainerLocationProtocolService.newReflectiveBlockingService;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_DEFAULT;
@@ -261,13 +261,9 @@
}
if (pipeline == null) {
- pipeline = scm.getPipelineManager().createPipeline(
- StandaloneReplicationConfig.getInstance(ReplicationConfig
- .getLegacyFactor(container.getReplicationConfig())),
- scm.getContainerManager()
- .getContainerReplicas(cid).stream()
- .map(ContainerReplica::getDatanodeDetails)
- .collect(Collectors.toList()));
+ pipeline = scm.getPipelineManager().createPipelineForRead(
+ container.getReplicationConfig(),
+ scm.getContainerManager().getContainerReplicas(cid));
}
return new ContainerWithPipeline(container, pipeline);
@@ -310,7 +306,8 @@
.setBytesUsed(r.getBytesUsed())
.setPlaceOfBirth(r.getOriginDatanodeId().toString())
.setKeyCount(r.getKeyCount())
- .setSequenceID(r.getSequenceId()).build()
+ .setSequenceID(r.getSequenceId())
+ .setReplicaIndex(r.getReplicaIndex()).build()
);
}
return results;
@@ -390,7 +387,7 @@
@Override
public List<ContainerInfo> listContainer(long startContainerID,
int count) throws IOException {
- return listContainer(startContainerID, count, null, null);
+ return listContainer(startContainerID, count, null, null, null);
}
/**
@@ -406,7 +403,7 @@
@Override
public List<ContainerInfo> listContainer(long startContainerID,
int count, HddsProtos.LifeCycleState state) throws IOException {
- return listContainer(startContainerID, count, state, null);
+ return listContainer(startContainerID, count, state, null, null);
}
/**
@@ -420,6 +417,7 @@
* @throws IOException
*/
@Override
+ @Deprecated
public List<ContainerInfo> listContainer(long startContainerID,
int count, HddsProtos.LifeCycleState state,
HddsProtos.ReplicationFactor factor) throws IOException {
@@ -439,6 +437,9 @@
if (factor != null) {
return scm.getContainerManager().getContainers(state).stream()
.filter(info -> info.containerID().getId() >= startContainerID)
+ //Filtering EC replication type as EC will not have factor.
+ .filter(info -> info
+ .getReplicationType() != HddsProtos.ReplicationType.EC)
.filter(info -> (info.getReplicationFactor() == factor))
.sorted().limit(count).collect(Collectors.toList());
} else {
@@ -450,6 +451,9 @@
if (factor != null) {
return scm.getContainerManager().getContainers().stream()
.filter(info -> info.containerID().getId() >= startContainerID)
+ //Filtering EC replication type as EC will not have factor.
+ .filter(info -> info
+ .getReplicationType() != HddsProtos.ReplicationType.EC)
.filter(info -> info.getReplicationFactor() == factor)
.sorted().limit(count).collect(Collectors.toList());
} else {
@@ -469,6 +473,76 @@
}
}
+ /**
+ * Lists a range of containers and get their info.
+ *
+ * @param startContainerID start containerID.
+ * @param count count must be {@literal >} 0.
+ * @param state Container with this state will be returned.
+ * @param repConfig Replication Config for the container.
+ * @return a list of pipeline.
+ * @throws IOException
+ */
+ @Override
+ public List<ContainerInfo> listContainer(long startContainerID,
+ int count, HddsProtos.LifeCycleState state,
+ HddsProtos.ReplicationType replicationType,
+ ReplicationConfig repConfig) throws IOException {
+ boolean auditSuccess = true;
+ Map<String, String> auditMap = Maps.newHashMap();
+ auditMap.put("startContainerID", String.valueOf(startContainerID));
+ auditMap.put("count", String.valueOf(count));
+ if (state != null) {
+ auditMap.put("state", state.name());
+ }
+ if (replicationType != null) {
+ auditMap.put("replicationType", replicationType.toString());
+ }
+ if (repConfig != null) {
+ auditMap.put("replicationConfig", repConfig.toString());
+ }
+ try {
+ final ContainerID containerId = ContainerID.valueOf(startContainerID);
+ if (state == null && replicationType == null && repConfig == null) {
+ // Not filters, so just return everything
+ return scm.getContainerManager().getContainers(containerId, count);
+ }
+
+ List<ContainerInfo> containerList;
+ if (state != null) {
+ containerList = scm.getContainerManager().getContainers(state);
+ } else {
+ containerList = scm.getContainerManager().getContainers();
+ }
+
+ Stream<ContainerInfo> containerStream = containerList.stream()
+ .filter(info -> info.containerID().getId() >= startContainerID);
+ // If we have repConfig filter by it, as it includes repType too.
+ // Otherwise, we may have a filter just for repType, eg all EC containers
+ // without filtering on their replication scheme
+ if (repConfig != null) {
+ containerStream = containerStream
+ .filter(info -> info.getReplicationConfig().equals(repConfig));
+ } else if (replicationType != null) {
+ containerStream = containerStream
+ .filter(info -> info.getReplicationType() == replicationType);
+ }
+ return containerStream.sorted()
+ .limit(count)
+ .collect(Collectors.toList());
+ } catch (Exception ex) {
+ auditSuccess = false;
+ AUDIT.logReadFailure(
+ buildAuditMessageForFailure(SCMAction.LIST_CONTAINER, auditMap, ex));
+ throw ex;
+ } finally {
+ if (auditSuccess) {
+ AUDIT.logReadSuccess(
+ buildAuditMessageForSuccess(SCMAction.LIST_CONTAINER, auditMap));
+ }
+ }
+ }
+
@Override
public void deleteContainer(long containerID) throws IOException {
boolean auditSuccess = true;
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
index 4a82224..c9e4014 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
@@ -19,6 +19,7 @@
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
@@ -651,6 +652,15 @@
.build();
}
+ public static ContainerInfo getECContainer(
+ final HddsProtos.LifeCycleState state, PipelineID pipelineID,
+ ECReplicationConfig replicationConfig) {
+ return getDefaultContainerInfoBuilder(state)
+ .setReplicationConfig(replicationConfig)
+ .setPipelineID(pipelineID)
+ .build();
+ }
+
public static Set<ContainerReplica> getReplicas(
final ContainerID containerId,
final ContainerReplicaProto.State state,
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java
index d1e97fe..ee0b637 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java
@@ -22,6 +22,7 @@
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -169,4 +170,13 @@
Assert.assertEquals(1, containerManager
.getContainerStateCount(HddsProtos.LifeCycleState.CLOSED));
}
+
+ @Test
+ public void testAllocateContainersWithECReplicationConfig() throws Exception {
+ final ContainerInfo admin = containerManager
+ .allocateContainer(new ECReplicationConfig(3, 2), "admin");
+ Assert.assertEquals(1, containerManager.getContainers().size());
+ Assert.assertNotNull(containerManager.getContainer(admin.containerID()));
+ }
+
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
index c653691..c0c8ed2 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
@@ -18,6 +18,7 @@
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -53,13 +54,16 @@
import java.io.File;
import java.io.IOException;
+import java.util.HashMap;
import java.util.Iterator;
+import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static junit.framework.TestCase.assertEquals;
+import static org.apache.hadoop.hdds.scm.HddsTestUtils.getECContainer;
import static org.apache.hadoop.hdds.scm.HddsTestUtils.getReplicas;
import static org.apache.hadoop.hdds.scm.HddsTestUtils.getContainer;
@@ -632,6 +636,158 @@
}
@Test
+ public void openECContainerKeyAndBytesUsedUpdatedToMinimumOfAllReplicas()
+ throws IOException {
+ final ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+ final ContainerReportHandler reportHandler = new ContainerReportHandler(
+ nodeManager, containerManager);
+
+ Pipeline pipeline = pipelineManager.createPipeline(repConfig);
+ Map<Integer, DatanodeDetails> dns = new HashMap<>();
+ final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
+ NodeStatus.inServiceHealthy()).iterator();
+ for (int i = 1; i <= repConfig.getRequiredNodes(); i++) {
+ dns.put(i, nodeIterator.next());
+ }
+ final ContainerReplicaProto.State replicaState
+ = ContainerReplicaProto.State.OPEN;
+ final ContainerInfo containerOne =
+ getECContainer(LifeCycleState.OPEN, pipeline.getId(), repConfig);
+
+ containerStateManager.addContainer(containerOne.getProtobuf());
+ // Container loaded, no replicas reported from DNs. Expect zeros for
+ // usage values.
+ assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+
+ // Report from data index 2 - should not update stats
+ reportHandler.onMessage(getContainerReportFromDatanode(
+ containerOne.containerID(), replicaState,
+ dns.get(2), 50L, 60L, 2), publisher);
+ assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+
+ // Report from replica 1, it should update
+ reportHandler.onMessage(getContainerReportFromDatanode(
+ containerOne.containerID(), replicaState,
+ dns.get(1), 50L, 60L, 1), publisher);
+ assertEquals(50L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(60L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+
+ // Parity 1 report a greater value, but as the container is own the stats
+ // should be the min value.
+ // Report from replica 1, it should update
+ reportHandler.onMessage(getContainerReportFromDatanode(
+ containerOne.containerID(), replicaState,
+ dns.get(4), 80L, 90L, 4), publisher);
+ assertEquals(50L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(60L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+
+ // Parity 2 reports a lesser value, so the stored values should update
+ reportHandler.onMessage(getContainerReportFromDatanode(
+ containerOne.containerID(), replicaState,
+ dns.get(5), 40, 30, 5), publisher);
+ assertEquals(40L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(30L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+
+ // Report from data index 3 - should not update stats even though it has
+ // lesser values
+ reportHandler.onMessage(getContainerReportFromDatanode(
+ containerOne.containerID(), replicaState,
+ dns.get(2), 10L, 10L, 2), publisher);
+ assertEquals(40L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(30L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+ }
+
+ @Test
+ public void closedECContainerKeyAndBytesUsedUpdatedToMinimumOfAllReplicas()
+ throws IOException {
+ final ECReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+ final ContainerReportHandler reportHandler = new ContainerReportHandler(
+ nodeManager, containerManager);
+
+ Pipeline pipeline = pipelineManager.createPipeline(repConfig);
+ Map<Integer, DatanodeDetails> dns = new HashMap<>();
+ final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
+ NodeStatus.inServiceHealthy()).iterator();
+ for (int i = 1; i <= repConfig.getRequiredNodes(); i++) {
+ dns.put(i, nodeIterator.next());
+ }
+ final ContainerReplicaProto.State replicaState
+ = ContainerReplicaProto.State.OPEN;
+ final ContainerInfo containerOne =
+ getECContainer(LifeCycleState.CLOSED, pipeline.getId(), repConfig);
+
+ containerStateManager.addContainer(containerOne.getProtobuf());
+ // Container loaded, no replicas reported from DNs. Expect zeros for
+ // usage values.
+ assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+
+ // Report from data index 2 - should not update stats
+ reportHandler.onMessage(getContainerReportFromDatanode(
+ containerOne.containerID(), replicaState,
+ dns.get(2), 50L, 60L, 2), publisher);
+ assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(0L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+
+ // Report from replica 1, it should update
+ reportHandler.onMessage(getContainerReportFromDatanode(
+ containerOne.containerID(), replicaState,
+ dns.get(1), 50L, 60L, 1), publisher);
+ assertEquals(50L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(60L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+
+ // Parity 1 report a greater value, as the container is closed the stats
+ // should be the max value.
+ // Report from replica 1, it should update
+ reportHandler.onMessage(getContainerReportFromDatanode(
+ containerOne.containerID(), replicaState,
+ dns.get(4), 80L, 90L, 4), publisher);
+ assertEquals(80L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(90L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+
+ // Parity 2 reports a lesser value, so the stored values should not update
+ reportHandler.onMessage(getContainerReportFromDatanode(
+ containerOne.containerID(), replicaState,
+ dns.get(5), 40, 30, 5), publisher);
+ assertEquals(80L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(90L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+
+ // Report from data index 3 - should not update stats even though it has
+ // greater values
+ reportHandler.onMessage(getContainerReportFromDatanode(
+ containerOne.containerID(), replicaState,
+ dns.get(2), 110L, 120L, 2), publisher);
+ assertEquals(80L, containerManager.getContainer(containerOne.containerID())
+ .getUsedBytes());
+ assertEquals(90L, containerManager.getContainer(containerOne.containerID())
+ .getNumberOfKeys());
+ }
+
+ @Test
public void testStaleReplicaOfDeletedContainer() throws NodeNotFoundException,
IOException {
@@ -653,7 +809,7 @@
// Expects the replica will be deleted.
final ContainerReportsProto containerReport = getContainerReportsProto(
containerOne.containerID(), ContainerReplicaProto.State.CLOSED,
- datanodeOne.getUuidString());
+ datanodeOne.getUuidString(), 0);
final ContainerReportFromDatanode containerReportFromDatanode =
new ContainerReportFromDatanode(datanodeOne, containerReport);
reportHandler.onMessage(containerReportFromDatanode, publisher);
@@ -668,8 +824,16 @@
private ContainerReportFromDatanode getContainerReportFromDatanode(
ContainerID containerId, ContainerReplicaProto.State state,
DatanodeDetails dn, long bytesUsed, long keyCount) {
+ return getContainerReportFromDatanode(containerId, state, dn, bytesUsed,
+ keyCount, 0);
+ }
+
+ private ContainerReportFromDatanode getContainerReportFromDatanode(
+ ContainerID containerId, ContainerReplicaProto.State state,
+ DatanodeDetails dn, long bytesUsed, long keyCount, int replicaIndex) {
ContainerReportsProto containerReport = getContainerReportsProto(
- containerId, state, dn.getUuidString(), bytesUsed, keyCount);
+ containerId, state, dn.getUuidString(), bytesUsed, keyCount,
+ replicaIndex);
return new ContainerReportFromDatanode(dn, containerReport);
}
@@ -678,12 +842,20 @@
final ContainerID containerId, final ContainerReplicaProto.State state,
final String originNodeId) {
return getContainerReportsProto(containerId, state, originNodeId,
- 2000000000L, 100000000L);
+ 2000000000L, 100000000L, 0);
}
protected static ContainerReportsProto getContainerReportsProto(
final ContainerID containerId, final ContainerReplicaProto.State state,
- final String originNodeId, final long usedBytes, final long keyCount) {
+ final String originNodeId, int replicaIndex) {
+ return getContainerReportsProto(containerId, state, originNodeId,
+ 2000000000L, 100000000L, replicaIndex);
+ }
+
+ protected static ContainerReportsProto getContainerReportsProto(
+ final ContainerID containerId, final ContainerReplicaProto.State state,
+ final String originNodeId, final long usedBytes, final long keyCount,
+ final int replicaIndex) {
final ContainerReportsProto.Builder crBuilder =
ContainerReportsProto.newBuilder();
final ContainerReplicaProto replicaProto =
@@ -701,6 +873,7 @@
.setWriteBytes(2000000000L)
.setBlockCommitSequenceId(10000L)
.setDeleteTransactionId(0)
+ .setReplicaIndex(replicaIndex)
.build();
return crBuilder.addReports(replicaProto).build();
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java
index 4ed2887..f85f66e 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java
@@ -165,6 +165,14 @@
Assert.assertSame(SCMContainerPlacementRandom.class, policy.getClass());
}
+ @Test
+ public void testECPolicy() throws IOException {
+ PlacementPolicy policy = ContainerPlacementPolicyFactory
+ .getECPolicy(conf, null, null, true, null);
+ Assert.assertSame(SCMContainerPlacementRackScatter.class,
+ policy.getClass());
+ }
+
/**
* A dummy container placement implementation for test.
*/
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java
index a830a71..12f63fe 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java
@@ -39,6 +39,7 @@
import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.junit.After;
import org.apache.hadoop.ozone.container.upgrade.UpgradeUtils;
import org.junit.Assert;
import org.junit.Before;
@@ -196,6 +197,11 @@
nodeManager, conf, cluster, false, metrics);
}
+ @After
+ public void teardown() {
+ metrics.unRegister();
+ }
+
@Test
public void chooseNodeWithNoExcludedNodes() throws SCMException {
// test choose new datanodes for new pipeline cases
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackScatter.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackScatter.java
new file mode 100644
index 0000000..1b45a75
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackScatter.java
@@ -0,0 +1,529 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.container.placement.algorithms;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.MetadataStorageReportProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.HddsTestUtils;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.net.NetConstants;
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.net.NetworkTopologyImpl;
+import org.apache.hadoop.hdds.scm.net.Node;
+import org.apache.hadoop.hdds.scm.net.NodeSchema;
+import org.apache.hadoop.hdds.scm.net.NodeSchemaManager;
+import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.ozone.container.upgrade.UpgradeUtils;
+import org.hamcrest.MatcherAssert;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.RACK_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test for the scm container rack aware placement.
+ */
+@RunWith(Parameterized.class)
+public class TestSCMContainerPlacementRackScatter {
+ private NetworkTopology cluster;
+ private OzoneConfiguration conf;
+ private NodeManager nodeManager;
+ private final Integer datanodeCount;
+ private final List<DatanodeDetails> datanodes = new ArrayList<>();
+ private final List<DatanodeInfo> dnInfos = new ArrayList<>();
+ // policy with fallback capability
+ private SCMContainerPlacementRackScatter policy;
+ // node storage capacity
+ private static final long STORAGE_CAPACITY = 100L;
+ private SCMContainerPlacementMetrics metrics;
+ private static final int NODE_PER_RACK = 5;
+
+ public TestSCMContainerPlacementRackScatter(Integer count) {
+ this.datanodeCount = count;
+ }
+
+ @Parameterized.Parameters
+ public static Collection<Object[]> setupDatanodes() {
+ return Arrays.asList(new Object[][] {{3}, {4}, {5}, {6}, {7}, {8}, {9},
+ {10}, {11}, {12}, {13}, {14}, {15}, {20}, {25}, {30}});
+ }
+
+ @Before
+ public void setup() {
+ //initialize network topology instance
+ conf = new OzoneConfiguration();
+ // We are using small units here
+ conf.setStorageSize(OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN,
+ 1, StorageUnit.BYTES);
+ NodeSchema[] schemas = new NodeSchema[]
+ {ROOT_SCHEMA, RACK_SCHEMA, LEAF_SCHEMA};
+ NodeSchemaManager.getInstance().init(schemas, true);
+ cluster = new NetworkTopologyImpl(NodeSchemaManager.getInstance());
+
+ // build datanodes, and network topology
+ String rack = "/rack";
+ String hostname = "node";
+ for (int i = 0; i < datanodeCount; i++) {
+ // Totally 6 racks, each has 5 datanodes
+ DatanodeDetails datanodeDetails =
+ MockDatanodeDetails.createDatanodeDetails(
+ hostname + i, rack + (i / NODE_PER_RACK));
+ DatanodeInfo datanodeInfo = new DatanodeInfo(
+ datanodeDetails, NodeStatus.inServiceHealthy(),
+ UpgradeUtils.defaultLayoutVersionProto());
+
+ StorageReportProto storage1 = HddsTestUtils.createStorageReport(
+ datanodeInfo.getUuid(), "/data1-" + datanodeInfo.getUuidString(),
+ STORAGE_CAPACITY, 0, 100L, null);
+ MetadataStorageReportProto metaStorage1 =
+ HddsTestUtils.createMetadataStorageReport(
+ "/metadata1-" + datanodeInfo.getUuidString(),
+ STORAGE_CAPACITY, 0, 100L, null);
+ datanodeInfo.updateStorageReports(
+ new ArrayList<>(Arrays.asList(storage1)));
+ datanodeInfo.updateMetaDataStorageReports(
+ new ArrayList<>(Arrays.asList(metaStorage1)));
+
+ datanodes.add(datanodeDetails);
+ cluster.add(datanodeDetails);
+ dnInfos.add(datanodeInfo);
+ }
+
+ if (datanodeCount > 4) {
+ StorageReportProto storage2 = HddsTestUtils.createStorageReport(
+ dnInfos.get(2).getUuid(),
+ "/data1-" + datanodes.get(2).getUuidString(),
+ STORAGE_CAPACITY, 90L, 10L, null);
+ dnInfos.get(2).updateStorageReports(
+ new ArrayList<>(Arrays.asList(storage2)));
+ StorageReportProto storage3 = HddsTestUtils.createStorageReport(
+ dnInfos.get(3).getUuid(),
+ "/data1-" + dnInfos.get(3).getUuidString(),
+ STORAGE_CAPACITY, 80L, 20L, null);
+ dnInfos.get(3).updateStorageReports(
+ new ArrayList<>(Arrays.asList(storage3)));
+ StorageReportProto storage4 = HddsTestUtils.createStorageReport(
+ dnInfos.get(4).getUuid(),
+ "/data1-" + dnInfos.get(4).getUuidString(),
+ STORAGE_CAPACITY, 70L, 30L, null);
+ dnInfos.get(4).updateStorageReports(
+ new ArrayList<>(Arrays.asList(storage4)));
+ } else if (datanodeCount > 3) {
+ StorageReportProto storage2 = HddsTestUtils.createStorageReport(
+ dnInfos.get(2).getUuid(),
+ "/data1-" + dnInfos.get(2).getUuidString(),
+ STORAGE_CAPACITY, 90L, 10L, null);
+ dnInfos.get(2).updateStorageReports(
+ new ArrayList<>(Arrays.asList(storage2)));
+ StorageReportProto storage3 = HddsTestUtils.createStorageReport(
+ dnInfos.get(3).getUuid(),
+ "/data1-" + dnInfos.get(3).getUuidString(),
+ STORAGE_CAPACITY, 80L, 20L, null);
+ dnInfos.get(3).updateStorageReports(
+ new ArrayList<>(Arrays.asList(storage3)));
+ } else if (datanodeCount > 2) {
+ StorageReportProto storage2 = HddsTestUtils.createStorageReport(
+ dnInfos.get(2).getUuid(),
+ "/data1-" + dnInfos.get(2).getUuidString(),
+ STORAGE_CAPACITY, 84L, 16L, null);
+ dnInfos.get(2).updateStorageReports(
+ new ArrayList<>(Arrays.asList(storage2)));
+ }
+
+ // create mock node manager
+ nodeManager = Mockito.mock(NodeManager.class);
+ when(nodeManager.getNodes(NodeStatus.inServiceHealthy()))
+ .thenReturn(new ArrayList<>(datanodes));
+ for (DatanodeInfo dn: dnInfos) {
+ when(nodeManager.getNodeByUuid(dn.getUuidString()))
+ .thenReturn(dn);
+ }
+ when(nodeManager.getClusterNetworkTopologyMap())
+ .thenReturn(cluster);
+
+ // create placement policy instances
+ metrics = SCMContainerPlacementMetrics.create();
+ policy = new SCMContainerPlacementRackScatter(
+ nodeManager, conf, cluster, true, metrics);
+ }
+
+ @After
+ public void teardown() {
+ metrics.unRegister();
+ }
+
+ @Test
+ public void chooseNodeWithNoExcludedNodes() throws SCMException {
+ int rackLevel = cluster.getMaxLevel() - 1;
+ int rackNum = cluster.getNumOfNodes(rackLevel);
+
+ // test choose new datanodes for new pipeline cases
+ // 1 replica
+ int nodeNum = 1;
+ List<DatanodeDetails> datanodeDetails =
+ policy.chooseDatanodes(null, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+
+ // 2 replicas
+ nodeNum = 2;
+ datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertTrue(!cluster.isSameParent(datanodeDetails.get(0),
+ datanodeDetails.get(1)) || (datanodeCount <= NODE_PER_RACK));
+
+ // 3 replicas
+ nodeNum = 3;
+ if (datanodeCount > nodeNum) {
+ datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(getRackSize(datanodeDetails),
+ Math.min(nodeNum, rackNum));
+ }
+
+ // 5 replicas
+ nodeNum = 5;
+ if (datanodeCount > nodeNum) {
+ assumeTrue(datanodeCount >= NODE_PER_RACK);
+ datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(getRackSize(datanodeDetails),
+ Math.min(nodeNum, rackNum));
+ }
+
+ // 10 replicas
+ nodeNum = 10;
+ if (datanodeCount > nodeNum) {
+ assumeTrue(datanodeCount > 2 * NODE_PER_RACK);
+ datanodeDetails = policy.chooseDatanodes(null, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(getRackSize(datanodeDetails),
+ Math.min(nodeNum, rackNum));
+ }
+ }
+
+ @Test
+ public void chooseNodeWithExcludedNodes() throws SCMException {
+ int rackLevel = cluster.getMaxLevel() - 1;
+ int rackNum = cluster.getNumOfNodes(rackLevel);
+ int totalNum;
+ // test choose new datanodes for under replicated pipeline
+ // 3 replicas, two existing datanodes on same rack
+ assumeTrue(datanodeCount > NODE_PER_RACK);
+ int nodeNum = 1;
+ List<DatanodeDetails> excludedNodes = new ArrayList<>();
+
+ excludedNodes.add(datanodes.get(0));
+ excludedNodes.add(datanodes.get(1));
+ List<DatanodeDetails> datanodeDetails = policy.chooseDatanodes(
+ excludedNodes, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertFalse(cluster.isSameParent(datanodeDetails.get(0),
+ excludedNodes.get(0)));
+ Assert.assertFalse(cluster.isSameParent(datanodeDetails.get(0),
+ excludedNodes.get(1)));
+
+ // 3 replicas, one existing datanode
+ nodeNum = 2;
+ totalNum = 3;
+ excludedNodes.clear();
+ excludedNodes.add(datanodes.get(0));
+ datanodeDetails = policy.chooseDatanodes(
+ excludedNodes, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(getRackSize(datanodeDetails, excludedNodes),
+ Math.min(totalNum, rackNum));
+
+ // 3 replicas, two existing datanodes on different rack
+ nodeNum = 1;
+ totalNum = 3;
+ excludedNodes.clear();
+ excludedNodes.add(datanodes.get(0));
+ excludedNodes.add(datanodes.get(5));
+ datanodeDetails = policy.chooseDatanodes(
+ excludedNodes, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(getRackSize(datanodeDetails, excludedNodes),
+ Math.min(totalNum, rackNum));
+
+ // 5 replicas, one existing datanode
+ nodeNum = 4;
+ totalNum = 5;
+ excludedNodes.clear();
+ excludedNodes.add(datanodes.get(0));
+ datanodeDetails = policy.chooseDatanodes(
+ excludedNodes, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(getRackSize(datanodeDetails, excludedNodes),
+ Math.min(totalNum, rackNum));
+
+ // 5 replicas, two existing datanodes on different rack
+ nodeNum = 3;
+ totalNum = 5;
+ excludedNodes.clear();
+ excludedNodes.add(datanodes.get(0));
+ excludedNodes.add(datanodes.get(5));
+ datanodeDetails = policy.chooseDatanodes(
+ excludedNodes, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(getRackSize(datanodeDetails, excludedNodes),
+ Math.min(totalNum, rackNum));
+ }
+
+ @Test
+ public void chooseNodeWithFavoredNodes() throws SCMException {
+ int nodeNum = 1;
+ List<DatanodeDetails> excludedNodes = new ArrayList<>();
+ List<DatanodeDetails> favoredNodes = new ArrayList<>();
+
+ // no excludedNodes, only favoredNodes
+ favoredNodes.add(datanodes.get(0));
+ List<DatanodeDetails> datanodeDetails = policy.chooseDatanodes(
+ excludedNodes, favoredNodes, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(datanodeDetails.get(0).getNetworkFullPath(),
+ favoredNodes.get(0).getNetworkFullPath());
+
+ // no overlap between excludedNodes and favoredNodes, favoredNodes can been
+ // chosen.
+ excludedNodes.clear();
+ favoredNodes.clear();
+ excludedNodes.add(datanodes.get(0));
+ favoredNodes.add(datanodes.get(1));
+ datanodeDetails = policy.chooseDatanodes(
+ excludedNodes, favoredNodes, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(datanodeDetails.get(0).getNetworkFullPath(),
+ favoredNodes.get(0).getNetworkFullPath());
+
+ // there is overlap between excludedNodes and favoredNodes, favoredNodes
+ // should not be chosen.
+ excludedNodes.clear();
+ favoredNodes.clear();
+ excludedNodes.add(datanodes.get(0));
+ favoredNodes.add(datanodes.get(0));
+ datanodeDetails = policy.chooseDatanodes(
+ excludedNodes, favoredNodes, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertFalse(datanodeDetails.get(0).getNetworkFullPath()
+ .equals(favoredNodes.get(0).getNetworkFullPath()));
+ }
+
+ @Test
+ public void testNoInfiniteLoop() throws SCMException {
+ int nodeNum = 1;
+
+ try {
+ // request storage space larger than node capability
+ policy.chooseDatanodes(null, null, nodeNum, STORAGE_CAPACITY + 0, 15);
+ fail("Storage requested exceeds capacity, this call should fail");
+ } catch (Exception e) {
+ assertTrue(e.getClass().getSimpleName().equals("SCMException"));
+ }
+
+ // get metrics
+ long totalRequest = metrics.getDatanodeRequestCount();
+ long successCount = metrics.getDatanodeChooseSuccessCount();
+ long tryCount = metrics.getDatanodeChooseAttemptCount();
+ long compromiseCount = metrics.getDatanodeChooseFallbackCount();
+
+ Assert.assertEquals(totalRequest, nodeNum);
+ Assert.assertEquals(successCount, 0);
+ MatcherAssert.assertThat("Not enough try", tryCount,
+ greaterThanOrEqualTo((long) nodeNum));
+ Assert.assertEquals(compromiseCount, 0);
+ }
+
+ @Test
+ public void testDatanodeWithDefaultNetworkLocation() throws SCMException {
+ String hostname = "node";
+ List<DatanodeInfo> dnInfoList = new ArrayList<>();
+ List<DatanodeDetails> dataList = new ArrayList<>();
+ NetworkTopology clusterMap =
+ new NetworkTopologyImpl(NodeSchemaManager.getInstance());
+ for (int i = 0; i < 30; i++) {
+ // Totally 6 racks, each has 5 datanodes
+ DatanodeDetails dn = MockDatanodeDetails.createDatanodeDetails(
+ hostname + i, null);
+ DatanodeInfo dnInfo = new DatanodeInfo(
+ dn, NodeStatus.inServiceHealthy(),
+ UpgradeUtils.defaultLayoutVersionProto());
+
+ StorageReportProto storage1 = HddsTestUtils.createStorageReport(
+ dnInfo.getUuid(), "/data1-" + dnInfo.getUuidString(),
+ STORAGE_CAPACITY, 0, 100L, null);
+ MetadataStorageReportProto metaStorage1 =
+ HddsTestUtils.createMetadataStorageReport(
+ "/metadata1-" + dnInfo.getUuidString(),
+ STORAGE_CAPACITY, 0, 100L, null);
+ dnInfo.updateStorageReports(
+ new ArrayList<>(Arrays.asList(storage1)));
+ dnInfo.updateMetaDataStorageReports(
+ new ArrayList<>(Arrays.asList(metaStorage1)));
+
+ dataList.add(dn);
+ clusterMap.add(dn);
+ dnInfoList.add(dnInfo);
+ }
+ Assert.assertEquals(dataList.size(), StringUtils.countMatches(
+ clusterMap.toString(), NetConstants.DEFAULT_RACK));
+ for (DatanodeInfo dn: dnInfoList) {
+ when(nodeManager.getNodeByUuid(dn.getUuidString()))
+ .thenReturn(dn);
+ }
+
+ // choose nodes to host 5 replica
+ int nodeNum = 5;
+ SCMContainerPlacementRackScatter newPolicy =
+ new SCMContainerPlacementRackScatter(nodeManager, conf, clusterMap,
+ true, metrics);
+ List<DatanodeDetails> datanodeDetails =
+ newPolicy.chooseDatanodes(null, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(1, getRackSize(datanodeDetails));
+ }
+
+ @Test
+ public void testvalidateContainerPlacement() {
+ // Only run this test for the full set of DNs. 5 DNs per rack on 6 racks.
+ assumeTrue(datanodeCount >= 15);
+ List<DatanodeDetails> dns = new ArrayList<>();
+ // First 5 node are on the same rack
+ dns.add(datanodes.get(0));
+ dns.add(datanodes.get(1));
+ dns.add(datanodes.get(2));
+ ContainerPlacementStatus stat = policy.validateContainerPlacement(dns, 3);
+ assertFalse(stat.isPolicySatisfied());
+ assertEquals(2, stat.misReplicationCount());
+
+ // Pick a new list which spans 2 racks
+ dns = new ArrayList<>();
+ dns.add(datanodes.get(0));
+ dns.add(datanodes.get(1));
+ dns.add(datanodes.get(5)); // This is on second rack
+ stat = policy.validateContainerPlacement(dns, 3);
+ assertFalse(stat.isPolicySatisfied());
+ assertEquals(1, stat.misReplicationCount());
+
+ // Pick single DN, expecting 3 replica. Policy is not met.
+ dns = new ArrayList<>();
+ dns.add(datanodes.get(0));
+ stat = policy.validateContainerPlacement(dns, 3);
+ assertFalse(stat.isPolicySatisfied());
+ assertEquals(2, stat.misReplicationCount());
+
+ // Pick single DN, expecting 1 replica. Policy is met.
+ dns = new ArrayList<>();
+ dns.add(datanodes.get(0));
+ stat = policy.validateContainerPlacement(dns, 1);
+ assertTrue(stat.isPolicySatisfied());
+ assertEquals(0, stat.misReplicationCount());
+ }
+
+ @Test
+ public void testvalidateContainerPlacementSingleRackCluster() {
+ assumeTrue(datanodeCount == 5);
+
+ // All nodes are on the same rack in this test, and the cluster only has
+ // one rack.
+ List<DatanodeDetails> dns = new ArrayList<>();
+ dns.add(datanodes.get(0));
+ dns.add(datanodes.get(1));
+ dns.add(datanodes.get(2));
+ ContainerPlacementStatus stat = policy.validateContainerPlacement(dns, 3);
+ assertTrue(stat.isPolicySatisfied());
+ assertEquals(0, stat.misReplicationCount());
+
+ // Single DN - policy met as cluster only has one rack.
+ dns = new ArrayList<>();
+ dns.add(datanodes.get(0));
+ stat = policy.validateContainerPlacement(dns, 3);
+ assertTrue(stat.isPolicySatisfied());
+ assertEquals(0, stat.misReplicationCount());
+
+ // Single DN - only 1 replica expected
+ dns = new ArrayList<>();
+ dns.add(datanodes.get(0));
+ stat = policy.validateContainerPlacement(dns, 1);
+ assertTrue(stat.isPolicySatisfied());
+ assertEquals(0, stat.misReplicationCount());
+ }
+
+ @Test
+ public void testExcludedNodesOverlapsOutOfServiceNodes() throws SCMException {
+ assumeTrue(datanodeCount == 6);
+
+ // DN 5 is out of service
+ dnInfos.get(5).setNodeStatus(new NodeStatus(DECOMMISSIONED, HEALTHY));
+
+ // SCM should have detected that DN 5 is dead
+ cluster.remove(datanodes.get(5));
+
+ // Here we still have 5 DNs, so pick 5 should be possible
+ int nodeNum = 5;
+ List<DatanodeDetails> excludedNodes = new ArrayList<>();
+ // The DN 5 is out of service,
+ // but the client already has it in the excludeList.
+ // So there is an overlap.
+ excludedNodes.add(datanodes.get(5));
+
+ List<DatanodeDetails> datanodeDetails = policy.chooseDatanodes(
+ excludedNodes, null, nodeNum, 0, 5);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ }
+
+ private int getRackSize(List<DatanodeDetails>... datanodeDetails) {
+ Set<Node> racks = new HashSet<>();
+ for (List<DatanodeDetails> list : datanodeDetails) {
+ for (DatanodeDetails dn : list) {
+ racks.add(cluster.getAncestor(dn, 1));
+ }
+ }
+ return racks.size();
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java
index c82006e..523efc0 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java
@@ -20,7 +20,9 @@
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
import org.apache.hadoop.hdds.scm.node.NodeManager;
@@ -29,17 +31,21 @@
import org.apache.hadoop.ozone.ClientVersion;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
+import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Mock PipelineManager implementation for testing.
*/
-public final class MockPipelineManager implements PipelineManager {
+public class MockPipelineManager implements PipelineManager {
private PipelineStateManager stateManager;
@@ -56,6 +62,14 @@
@Override
public Pipeline createPipeline(ReplicationConfig replicationConfig)
throws IOException {
+ return createPipeline(replicationConfig, Collections.emptyList(),
+ Collections.emptyList());
+ }
+
+ @Override
+ public Pipeline createPipeline(ReplicationConfig replicationConfig,
+ List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes)
+ throws IOException {
final List<DatanodeDetails> nodes = Stream.generate(
MockDatanodeDetails::randomDatanodeDetails)
.limit(replicationConfig.getRequiredNodes())
@@ -84,6 +98,25 @@
}
@Override
+ public Pipeline createPipelineForRead(
+ final ReplicationConfig replicationConfig,
+ final Set<ContainerReplica> replicas) {
+ List<DatanodeDetails> dns = new ArrayList<>();
+ Map<DatanodeDetails, Integer> map = new HashMap<>();
+ for (ContainerReplica r : replicas) {
+ map.put(r.getDatanodeDetails(), r.getReplicaIndex());
+ dns.add(r.getDatanodeDetails());
+ }
+ return Pipeline.newBuilder()
+ .setId(PipelineID.randomId())
+ .setReplicationConfig(replicationConfig)
+ .setNodes(dns)
+ .setReplicaIndexes(map)
+ .setState(Pipeline.PipelineState.CLOSED)
+ .build();
+ }
+
+ @Override
public Pipeline getPipeline(final PipelineID pipelineID)
throws PipelineNotFoundException {
return stateManager.getPipeline(pipelineID);
@@ -126,6 +159,19 @@
}
@Override
+ /**
+ * Returns the count of pipelines meeting the given ReplicationConfig and
+ * state.
+ * @param replicationConfig The ReplicationConfig of the pipelines to count
+ * @param state The current state of the pipelines to count
+ * @return The count of pipelines meeting the above criteria
+ */
+ public int getPipelineCount(ReplicationConfig replicationConfig,
+ final Pipeline.PipelineState state) {
+ return stateManager.getPipelineCount(replicationConfig, state);
+ }
+
+ @Override
public void addContainerToPipeline(final PipelineID pipelineID,
final ContainerID containerID)
throws IOException {
@@ -160,11 +206,15 @@
@Override
public void openPipeline(final PipelineID pipelineId)
throws IOException {
+ stateManager.updatePipelineState(
+ pipelineId.getProtobuf(), HddsProtos.PipelineState.PIPELINE_OPEN);
}
@Override
public void closePipeline(final Pipeline pipeline, final boolean onTimeout)
throws IOException {
+ stateManager.updatePipelineState(pipeline.getId().getProtobuf(),
+ HddsProtos.PipelineState.PIPELINE_CLOSED);
}
@Override
@@ -206,6 +256,8 @@
@Override
public void deactivatePipeline(final PipelineID pipelineID)
throws IOException {
+ stateManager.updatePipelineState(pipelineID.getProtobuf(),
+ HddsProtos.PipelineState.PIPELINE_DORMANT);
}
@Override
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestECPipelineProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestECPipelineProvider.java
new file mode 100644
index 0000000..802ed01
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestECPipelineProvider.java
@@ -0,0 +1,154 @@
+/*
+ * 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.pipeline;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
+import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.ALLOCATED;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test for the ECPipelineProvider.
+ */
+public class TestECPipelineProvider {
+
+ private PipelineProvider provider;
+ private OzoneConfiguration conf;
+ private NodeManager nodeManager = Mockito.mock(NodeManager.class);
+ private PipelineStateManager stateManager =
+ Mockito.mock(PipelineStateManager.class);
+ private PlacementPolicy placementPolicy = Mockito.mock(PlacementPolicy.class);
+ private long containerSizeBytes;
+ @Before
+ public void setup() throws IOException {
+ conf = new OzoneConfiguration();
+ provider = new ECPipelineProvider(
+ nodeManager, stateManager, conf, placementPolicy);
+ this.containerSizeBytes = (long) this.conf.getStorageSize(
+ ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+ ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
+ StorageUnit.BYTES);
+ // Placement policy will always return EC number of random nodes.
+ Mockito.when(placementPolicy.chooseDatanodes(Mockito.anyList(),
+ Mockito.anyList(), Mockito.anyInt(), Mockito.anyLong(),
+ Mockito.anyLong()))
+ .thenAnswer(invocation -> {
+ List<DatanodeDetails> dns = new ArrayList<>();
+ for (int i = 0; i < (int) invocation.getArguments()[2]; i++) {
+ dns.add(MockDatanodeDetails.randomDatanodeDetails());
+ }
+ return dns;
+ });
+
+ }
+
+
+ @Test
+ public void testSimplePipelineCanBeCreatedWithIndexes() throws IOException {
+ ECReplicationConfig ecConf = new ECReplicationConfig(3, 2);
+ Pipeline pipeline = provider.create(ecConf);
+ Assert.assertEquals(EC, pipeline.getType());
+ Assert.assertEquals(ecConf.getData() + ecConf.getParity(),
+ pipeline.getNodes().size());
+ Assert.assertEquals(ALLOCATED, pipeline.getPipelineState());
+ List<DatanodeDetails> dns = pipeline.getNodes();
+ for (int i = 0; i < ecConf.getRequiredNodes(); i++) {
+ // EC DN indexes are numbered starting from 1 to N.
+ Assert.assertEquals(i + 1, pipeline.getReplicaIndex(dns.get(i)));
+ }
+ }
+
+ @Test
+ public void testPipelineForReadCanBeCreated() {
+ ECReplicationConfig ecConf = new ECReplicationConfig(3, 2);
+
+ Set<ContainerReplica> replicas = createContainerReplicas(4);
+ Pipeline pipeline = provider.createForRead(ecConf, replicas);
+
+ Assert.assertEquals(EC, pipeline.getType());
+ Assert.assertEquals(4, pipeline.getNodes().size());
+ Assert.assertEquals(ALLOCATED, pipeline.getPipelineState());
+ for (ContainerReplica r : replicas) {
+ Assert.assertEquals(r.getReplicaIndex(),
+ pipeline.getReplicaIndex(r.getDatanodeDetails()));
+ }
+ }
+
+ @Test
+ public void testExcludedAndFavoredNodesPassedToPlacementPolicy()
+ throws IOException {
+ ECReplicationConfig ecConf = new ECReplicationConfig(3, 2);
+
+ List<DatanodeDetails> excludedNodes = new ArrayList<>();
+ excludedNodes.add(MockDatanodeDetails.randomDatanodeDetails());
+
+ List<DatanodeDetails> favoredNodes = new ArrayList<>();
+ favoredNodes.add(MockDatanodeDetails.randomDatanodeDetails());
+
+ Pipeline pipeline = provider.create(ecConf, excludedNodes, favoredNodes);
+ Assert.assertEquals(EC, pipeline.getType());
+ Assert.assertEquals(ecConf.getData() + ecConf.getParity(),
+ pipeline.getNodes().size());
+
+ verify(placementPolicy).chooseDatanodes(excludedNodes, favoredNodes,
+ ecConf.getRequiredNodes(), 0, containerSizeBytes);
+ }
+
+ private Set<ContainerReplica> createContainerReplicas(int number) {
+ Set<ContainerReplica> replicas = new HashSet<>();
+ for (int i = 0; i < number; i++) {
+ ContainerReplica r = ContainerReplica.newBuilder()
+ .setBytesUsed(1)
+ .setContainerID(ContainerID.valueOf(1))
+ .setContainerState(StorageContainerDatanodeProtocolProtos
+ .ContainerReplicaProto.State.CLOSED)
+ .setKeyCount(1)
+ .setOriginNodeId(UUID.randomUUID())
+ .setSequenceId(1)
+ .setReplicaIndex(i + 1)
+ .setDatanodeDetails(MockDatanodeDetails.randomDatanodeDetails())
+ .build();
+ replicas.add(r);
+ }
+ return replicas;
+ }
+
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java
index 3a68db1..fd29929 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java
@@ -25,10 +25,12 @@
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.scm.HddsTestUtils;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.ha.SCMHADBTransactionBuffer;
@@ -37,6 +39,7 @@
import org.apache.hadoop.hdds.scm.ha.SCMContext;
import org.apache.hadoop.hdds.scm.ha.SCMServiceManager;
import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
@@ -57,9 +60,12 @@
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT_DEFAULT;
@@ -223,12 +229,18 @@
.getPipelines(RatisReplicationConfig
.getInstance(ReplicationFactor.THREE),
Pipeline.PipelineState.OPEN).contains(pipeline));
+ Assert.assertEquals(1, pipelineManager.getPipelineCount(
+ RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
+ Pipeline.PipelineState.DORMANT));
pipelineManager.activatePipeline(pipeline.getId());
Assert.assertTrue(pipelineManager
.getPipelines(RatisReplicationConfig
.getInstance(ReplicationFactor.THREE),
Pipeline.PipelineState.OPEN).contains(pipeline));
+ Assert.assertEquals(1, pipelineManager.getPipelineCount(
+ RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
+ Pipeline.PipelineState.OPEN));
buffer.flush();
Assert.assertTrue(pipelineStore.get(pipeline.getId()).isOpen());
pipelineManager.close();
@@ -724,6 +736,43 @@
assertTrue(containerLogIdx < pipelineLogIdx);
}
+ @Test
+ public void testCreatePipelineForRead() throws IOException {
+ PipelineManager pipelineManager = createPipelineManager(true);
+ List<DatanodeDetails> dns = nodeManager
+ .getNodes(NodeStatus.inServiceHealthy())
+ .stream()
+ .limit(3)
+ .collect(Collectors.toList());
+ Set<ContainerReplica> replicas = createContainerReplicasList(dns);
+ Pipeline pipeline = pipelineManager.createPipelineForRead(
+ RatisReplicationConfig.getInstance(ReplicationFactor.THREE), replicas);
+ Assert.assertEquals(3, pipeline.getNodes().size());
+ for (DatanodeDetails dn : pipeline.getNodes()) {
+ Assert.assertTrue(dns.contains(dn));
+ }
+ }
+
+ private Set<ContainerReplica> createContainerReplicasList(
+ List <DatanodeDetails> dns) {
+ Set<ContainerReplica> replicas = new HashSet<>();
+ for (DatanodeDetails dn : dns) {
+ ContainerReplica r = ContainerReplica.newBuilder()
+ .setBytesUsed(1)
+ .setContainerID(ContainerID.valueOf(1))
+ .setContainerState(StorageContainerDatanodeProtocolProtos
+ .ContainerReplicaProto.State.CLOSED)
+ .setKeyCount(1)
+ .setOriginNodeId(UUID.randomUUID())
+ .setSequenceId(1)
+ .setReplicaIndex(0)
+ .setDatanodeDetails(dn)
+ .build();
+ replicas.add(r);
+ }
+ return replicas;
+ }
+
private void sendPipelineReport(
DatanodeDetails dn, Pipeline pipeline,
PipelineReportHandler pipelineReportHandler,
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStateMap.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStateMap.java
new file mode 100644
index 0000000..8168ce8
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStateMap.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.pipeline;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for PipelineStateMap.
+ */
+
+public class TestPipelineStateMap {
+
+ private PipelineStateMap map;
+
+ @Before
+ public void setup() {
+ map = new PipelineStateMap();
+ }
+
+ @After
+ public void teardown() throws IOException {
+ }
+
+ @Test
+ public void testCountPipelines() throws IOException {
+ Pipeline p;
+
+ // Open Stanadlone Pipelines
+ map.addPipeline(MockPipeline.createPipeline(1));
+ map.addPipeline(MockPipeline.createPipeline(1));
+ p = MockPipeline.createPipeline(1);
+ map.addPipeline(p);
+ map.updatePipelineState(p.getId(), Pipeline.PipelineState.CLOSED);
+
+ // Ratis pipeline
+ map.addPipeline(MockPipeline.createRatisPipeline());
+ p = MockPipeline.createRatisPipeline();
+ map.addPipeline(p);
+ map.updatePipelineState(p.getId(), Pipeline.PipelineState.CLOSED);
+
+ // EC Pipelines
+ map.addPipeline(MockPipeline.createEcPipeline(
+ new ECReplicationConfig(3, 2)));
+ map.addPipeline(MockPipeline.createEcPipeline(
+ new ECReplicationConfig(3, 2)));
+ p = MockPipeline.createEcPipeline(new ECReplicationConfig(3, 2));
+ map.addPipeline(p);
+ map.updatePipelineState(p.getId(), Pipeline.PipelineState.CLOSED);
+
+ assertEquals(2, map.getPipelineCount(
+ StandaloneReplicationConfig.getInstance(ONE),
+ Pipeline.PipelineState.OPEN));
+ assertEquals(1, map.getPipelineCount(
+ RatisReplicationConfig.getInstance(THREE),
+ Pipeline.PipelineState.OPEN));
+ assertEquals(2, map.getPipelineCount(new ECReplicationConfig(3, 2),
+ Pipeline.PipelineState.OPEN));
+
+ assertEquals(0, map.getPipelineCount(new ECReplicationConfig(6, 3),
+ Pipeline.PipelineState.OPEN));
+
+ assertEquals(1, map.getPipelineCount(
+ StandaloneReplicationConfig.getInstance(ONE),
+ Pipeline.PipelineState.CLOSED));
+ assertEquals(1, map.getPipelineCount(
+ RatisReplicationConfig.getInstance(THREE),
+ Pipeline.PipelineState.CLOSED));
+ assertEquals(1, map.getPipelineCount(new ECReplicationConfig(3, 2),
+ Pipeline.PipelineState.CLOSED));
+ }
+
+
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
index d2b19fb..c0ce8d4 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
@@ -27,7 +27,10 @@
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
import org.apache.hadoop.hdds.scm.container.TestContainerManagerImpl;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
@@ -46,6 +49,7 @@
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@@ -56,6 +60,7 @@
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
@@ -219,6 +224,7 @@
List<DatanodeDetails> healthyNodes = nodeManager
.getNodes(NodeStatus.inServiceHealthy()).stream()
.limit(3).collect(Collectors.toList());
+ Set<ContainerReplica> replicas = createContainerReplicas(healthyNodes);
Pipeline pipeline1 = provider.create(
RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
@@ -226,8 +232,12 @@
Pipeline pipeline2 = provider.create(
RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
healthyNodes);
+ Pipeline pipeline3 = provider.createForRead(
+ RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
+ replicas);
Assert.assertEquals(pipeline1.getNodeSet(), pipeline2.getNodeSet());
+ Assert.assertEquals(pipeline2.getNodeSet(), pipeline3.getNodeSet());
cleanup();
}
@@ -281,6 +291,31 @@
}
@Test
+ // Test excluded nodes work correctly. Note that for Ratis, the
+ // PipelinePlacementPolicy, which Ratis is hardcoded to use, does not consider
+ // favored nodes.
+ public void testCreateFactorTHREEPipelineWithExcludedDatanodes()
+ throws Exception {
+ init(1);
+ int healthyCount = nodeManager.getNodes(NodeStatus.inServiceHealthy())
+ .size();
+ // Add all but 3 nodes to the exclude list and ensure that the 3 picked
+ // nodes are not in the excluded list.
+ List<DatanodeDetails> excludedNodes = nodeManager
+ .getNodes(NodeStatus.inServiceHealthy()).stream()
+ .limit(healthyCount - 3).collect(Collectors.toList());
+
+ Pipeline pipeline1 = provider.create(
+ RatisReplicationConfig.getInstance(ReplicationFactor.THREE),
+ excludedNodes, Collections.EMPTY_LIST);
+
+ for (DatanodeDetails dn : pipeline1.getNodes()) {
+ assertFalse(excludedNodes.contains(dn));
+ }
+ }
+
+
+ @Test
public void testCreatePipelinesWhenNotEnoughSpace() throws Exception {
String expectedErrorSubstring = "Unable to find enough" +
" nodes that meet the space requirement";
@@ -291,6 +326,9 @@
largeContainerConf.set(OZONE_SCM_CONTAINER_SIZE, "100TB");
init(1, largeContainerConf);
for (ReplicationFactor factor: ReplicationFactor.values()) {
+ if (factor == ReplicationFactor.ZERO) {
+ continue;
+ }
try {
provider.create(RatisReplicationConfig.getInstance(factor));
Assert.fail("Expected SCMException for large container size with " +
@@ -304,6 +342,9 @@
largeMetadataConf.set(OZONE_DATANODE_RATIS_VOLUME_FREE_SPACE_MIN, "100TB");
init(1, largeMetadataConf);
for (ReplicationFactor factor: ReplicationFactor.values()) {
+ if (factor == ReplicationFactor.ZERO) {
+ continue;
+ }
try {
provider.create(RatisReplicationConfig.getInstance(factor));
Assert.fail("Expected SCMException for large metadata size with " +
@@ -331,4 +372,24 @@
stateManager.addPipeline(pipelineProto);
nodeManager.addPipeline(openPipeline);
}
+
+ private Set<ContainerReplica> createContainerReplicas(
+ List<DatanodeDetails> dns) {
+ Set<ContainerReplica> replicas = new HashSet<>();
+ for (DatanodeDetails dn : dns) {
+ ContainerReplica r = ContainerReplica.newBuilder()
+ .setBytesUsed(1)
+ .setContainerID(ContainerID.valueOf(1))
+ .setContainerState(StorageContainerDatanodeProtocolProtos
+ .ContainerReplicaProto.State.CLOSED)
+ .setKeyCount(1)
+ .setOriginNodeId(UUID.randomUUID())
+ .setSequenceId(1)
+ .setReplicaIndex(0)
+ .setDatanodeDetails(dn)
+ .build();
+ replicas.add(r);
+ }
+ return replicas;
+ }
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java
new file mode 100644
index 0000000..ae997a3
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java
@@ -0,0 +1,446 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.pipeline;
+
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.PipelineChoosePolicy;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.TestContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.HealthyPipelineChoosePolicy;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.apache.hadoop.hdds.conf.StorageUnit.BYTES;
+import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.CLOSED;
+import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.OPEN;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests to validate the WritableECContainerProvider works correctly.
+ */
+public class TestWritableECContainerProvider {
+
+ private static final Logger LOG = LoggerFactory
+ .getLogger(TestWritableECContainerProvider.class);
+ private static final String OWNER = "SCM";
+ private PipelineManager pipelineManager;
+ private ContainerManager containerManager
+ = Mockito.mock(ContainerManager.class);
+ private PipelineChoosePolicy pipelineChoosingPolicy
+ = new HealthyPipelineChoosePolicy();
+
+ private OzoneConfiguration conf;
+ private DBStore dbStore;
+ private SCMHAManager scmhaManager;
+ private NodeManager nodeManager;
+ private WritableContainerProvider provider;
+ private ReplicationConfig repConfig;
+ private int minPipelines;
+
+ private Map<ContainerID, ContainerInfo> containers;
+
+ @Before
+ public void setup() throws IOException {
+ repConfig = new ECReplicationConfig(3, 2);
+ conf = new OzoneConfiguration();
+ WritableECContainerProvider.WritableECContainerProviderConfig providerConf =
+ conf.getObject(WritableECContainerProvider
+ .WritableECContainerProviderConfig.class);
+ minPipelines = providerConf.getMinimumPipelines();
+ containers = new HashMap<>();
+ File testDir = GenericTestUtils.getTestDir(
+ TestContainerManagerImpl.class.getSimpleName() + UUID.randomUUID());
+ conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
+ dbStore = DBStoreBuilder.createDBStore(
+ conf, new SCMDBDefinition());
+ scmhaManager = SCMHAManagerStub.getInstance(true);
+ nodeManager = new MockNodeManager(true, 10);
+ pipelineManager =
+ new MockPipelineManager(dbStore, scmhaManager, nodeManager);
+ provider = new WritableECContainerProvider(
+ conf, pipelineManager, containerManager, pipelineChoosingPolicy);
+
+ Mockito.doAnswer(call -> {
+ Pipeline pipeline = (Pipeline)call.getArguments()[2];
+ ContainerInfo container = createContainer(pipeline,
+ repConfig, System.nanoTime());
+ pipelineManager.addContainerToPipeline(
+ pipeline.getId(), container.containerID());
+ containers.put(container.containerID(), container);
+ return container;
+ }).when(containerManager).getMatchingContainer(Matchers.anyLong(),
+ Matchers.anyString(), Matchers.any(Pipeline.class));
+
+ Mockito.doAnswer(call ->
+ containers.get((ContainerID)call.getArguments()[0]))
+ .when(containerManager).getContainer(Matchers.any(ContainerID.class));
+
+ }
+
+ @Test
+ public void testPipelinesCreatedUpToMinLimitAndRandomPipelineReturned()
+ throws IOException {
+ // The first 5 calls should return a different container
+ Set<ContainerInfo> allocatedContainers = new HashSet<>();
+ for (int i = 0; i < minPipelines; i++) {
+ ContainerInfo container =
+ provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+ assertFalse(allocatedContainers.contains(container));
+ allocatedContainers.add(container);
+ }
+
+ allocatedContainers.clear();
+ for (int i = 0; i < 20; i++) {
+ ContainerInfo container =
+ provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+ allocatedContainers.add(container);
+ }
+ // Should have minPipelines containers created
+ assertEquals(minPipelines,
+ pipelineManager.getPipelines(repConfig, OPEN).size());
+ // We should have more than 1 allocatedContainers in the set proving a
+ // random container is selected each time. Do not check for 5 here as there
+ // is a reasonable chance that in 20 turns we don't pick all 5 nodes.
+ assertTrue(allocatedContainers.size() > 2);
+ }
+
+ @Test
+ public void testPiplineLimitIgnoresExcludedPipelines() throws IOException {
+ Set<ContainerInfo> allocatedContainers = new HashSet<>();
+ for (int i = 0; i < minPipelines; i++) {
+ ContainerInfo container = provider.getContainer(
+ 1, repConfig, OWNER, new ExcludeList());
+ allocatedContainers.add(container);
+ }
+ // We have the min limit of pipelines, but then exclude one. It should use
+ // one of the existing rather than createing a new one, as the limit is
+ // checked against all pipelines, not just the filtered list
+ ExcludeList exclude = new ExcludeList();
+ PipelineID excludedID = allocatedContainers
+ .stream().findFirst().get().getPipelineID();
+ exclude.addPipeline(excludedID);
+
+ ContainerInfo c = provider.getContainer(1, repConfig, OWNER, exclude);
+ assertNotEquals(excludedID, c.getPipelineID());
+ assertTrue(allocatedContainers.contains(c));
+ }
+
+ @Test
+ public void testNewPipelineCreatedIfAllPipelinesExcluded()
+ throws IOException {
+ Set<ContainerInfo> allocatedContainers = new HashSet<>();
+ for (int i = 0; i < minPipelines; i++) {
+ ContainerInfo container = provider.getContainer(
+ 1, repConfig, OWNER, new ExcludeList());
+ allocatedContainers.add(container);
+ }
+ // We have the min limit of pipelines, but then exclude one. It should use
+ // one of the existing rather than creating a new one, as the limit is
+ // checked against all pipelines, not just the filtered list
+ ExcludeList exclude = new ExcludeList();
+ for (ContainerInfo c : allocatedContainers) {
+ exclude.addPipeline(c.getPipelineID());
+ }
+ ContainerInfo c = provider.getContainer(1, repConfig, OWNER, exclude);
+ assertFalse(allocatedContainers.contains(c));
+ }
+
+ @Test
+ public void testNewPipelineCreatedIfAllContainersExcluded()
+ throws IOException {
+ Set<ContainerInfo> allocatedContainers = new HashSet<>();
+ for (int i = 0; i < minPipelines; i++) {
+ ContainerInfo container = provider.getContainer(
+ 1, repConfig, OWNER, new ExcludeList());
+ allocatedContainers.add(container);
+ }
+ // We have the min limit of pipelines, but then exclude one. It should use
+ // one of the existing rather than createing a new one, as the limit is
+ // checked against all pipelines, not just the filtered list
+ ExcludeList exclude = new ExcludeList();
+ for (ContainerInfo c : allocatedContainers) {
+ exclude.addConatinerId(c.containerID());
+ }
+ ContainerInfo c = provider.getContainer(1, repConfig, OWNER, exclude);
+ assertFalse(allocatedContainers.contains(c));
+ }
+
+ @Test
+ public void testUnableToCreateAnyPipelinesReturnsNull() throws IOException {
+ pipelineManager = new MockPipelineManager(
+ dbStore, scmhaManager, nodeManager) {
+ @Override
+ public Pipeline createPipeline(ReplicationConfig repConf,
+ List<DatanodeDetails> excludedNodes,
+ List<DatanodeDetails> favoredNodes) throws IOException {
+ throw new IOException("Cannot create pipelines");
+ }
+ };
+ provider = new WritableECContainerProvider(
+ conf, pipelineManager, containerManager, pipelineChoosingPolicy);
+
+ ContainerInfo container =
+ provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+ assertNull(container);
+ }
+
+ @Test
+ public void testExistingPipelineReturnedWhenNewCannotBeCreated()
+ throws IOException {
+ pipelineManager = new MockPipelineManager(
+ dbStore, scmhaManager, nodeManager) {
+
+ private boolean throwError = false;
+
+ @Override
+ public Pipeline createPipeline(ReplicationConfig repConf,
+ List<DatanodeDetails> excludedNodes,
+ List<DatanodeDetails> favoredNodes) throws IOException {
+ if (throwError) {
+ throw new IOException("Cannot create pipelines");
+ }
+ throwError = true;
+ return super.createPipeline(repConfig);
+ }
+ };
+ provider = new WritableECContainerProvider(
+ conf, pipelineManager, containerManager, pipelineChoosingPolicy);
+
+ ContainerInfo container =
+ provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+ for (int i = 0; i < 5; i++) {
+ ContainerInfo nextContainer =
+ provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+ assertEquals(container, nextContainer);
+ }
+ }
+
+ @Test
+ public void testNewContainerAllocatedAndPipelinesClosedIfNoSpaceInExisting()
+ throws IOException {
+ Set<ContainerInfo> allocatedContainers = new HashSet<>();
+ for (int i = 0; i < minPipelines; i++) {
+ ContainerInfo container =
+ provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+ assertFalse(allocatedContainers.contains(container));
+ allocatedContainers.add(container);
+ }
+ // Update all the containers to make them nearly full, but with enough space
+ // for an EC block to be striped across them.
+ for (ContainerInfo c : allocatedContainers) {
+ c.setUsedBytes(getMaxContainerSize() - 30 * 1024 * 1024);
+ }
+
+ // Get a new container of size 50 and ensure it is one of the original set.
+ // We ask for a space of 50, but as it is stripped across the EC group it
+ // will actually need 50 / dataNum space
+ ContainerInfo newContainer =
+ provider.getContainer(50 * 1024 * 1024, repConfig, OWNER,
+ new ExcludeList());
+ assertNotNull(newContainer);
+ assertTrue(allocatedContainers.contains(newContainer));
+ // Now get a new container where there is not enough space in the existing
+ // and ensure a new container gets created.
+ newContainer = provider.getContainer(
+ 128 * 1024 * 1024, repConfig, OWNER, new ExcludeList());
+ assertNotNull(newContainer);
+ assertFalse(allocatedContainers.contains(newContainer));
+ // The original pipelines should all be closed, triggered by the lack of
+ // space.
+ for (ContainerInfo c : allocatedContainers) {
+ Pipeline pipeline = pipelineManager.getPipeline(c.getPipelineID());
+ assertEquals(CLOSED, pipeline.getPipelineState());
+ }
+ }
+
+ @Test
+ public void testPipelineNotFoundWhenAttemptingToUseExisting()
+ throws IOException {
+ // Ensure PM throws PNF exception when we ask for the containers in the
+ // pipeline
+ pipelineManager = new MockPipelineManager(
+ dbStore, scmhaManager, nodeManager) {
+
+ @Override
+ public NavigableSet<ContainerID> getContainersInPipeline(
+ PipelineID pipelineID) throws IOException {
+ throw new PipelineNotFoundException("Simulated exception");
+ }
+ };
+ provider = new WritableECContainerProvider(
+ conf, pipelineManager, containerManager, pipelineChoosingPolicy);
+
+ Set<ContainerInfo> allocatedContainers = new HashSet<>();
+ for (int i = 0; i < minPipelines; i++) {
+ ContainerInfo container =
+ provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+ assertFalse(allocatedContainers.contains(container));
+ allocatedContainers.add(container);
+ }
+ // Now attempt to get a container - any attempt to use an existing with
+ // throw PNF and then we must allocate a new one
+ ContainerInfo newContainer =
+ provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+ assertNotNull(newContainer);
+ assertFalse(allocatedContainers.contains(newContainer));
+ }
+
+ @Test
+ public void testContainerNotFoundWhenAttemptingToUseExisting()
+ throws IOException {
+ Set<ContainerInfo> allocatedContainers = new HashSet<>();
+ for (int i = 0; i < minPipelines; i++) {
+ ContainerInfo container =
+ provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+ assertFalse(allocatedContainers.contains(container));
+ allocatedContainers.add(container);
+ }
+
+ // Ensure ContainerManager always throws when a container is requested so
+ // existing pipelines cannot be used
+ Mockito.doAnswer(call -> {
+ throw new ContainerNotFoundException();
+ }).when(containerManager).getContainer(Matchers.any(ContainerID.class));
+
+ ContainerInfo newContainer =
+ provider.getContainer(1, repConfig, OWNER, new ExcludeList());
+ assertNotNull(newContainer);
+ assertFalse(allocatedContainers.contains(newContainer));
+
+ // Ensure all the existing pipelines are closed
+ for (ContainerInfo c : allocatedContainers) {
+ Pipeline pipeline = pipelineManager.getPipeline(c.getPipelineID());
+ assertEquals(CLOSED, pipeline.getPipelineState());
+ }
+ }
+
+ @Test
+ public void testPipelineOpenButContainerRemovedFromIt() throws IOException {
+ // This can happen if the container close process is triggered from the DN.
+ // When tha happens, CM will change the container state to CLOSING and
+ // remove it from the container list in pipeline Manager.
+ Set<ContainerInfo> allocatedContainers = new HashSet<>();
+ for (int i = 0; i < minPipelines; i++) {
+ ContainerInfo container = provider.getContainer(
+ 1, repConfig, OWNER, new ExcludeList());
+ assertFalse(allocatedContainers.contains(container));
+ allocatedContainers.add(container);
+ // Remove the container from the pipeline to simulate closing it
+ pipelineManager.removeContainerFromPipeline(
+ container.getPipelineID(), container.containerID());
+ }
+ ContainerInfo newContainer = provider.getContainer(
+ 1, repConfig, OWNER, new ExcludeList());
+ assertFalse(allocatedContainers.contains(newContainer));
+ for (ContainerInfo c : allocatedContainers) {
+ Pipeline pipeline = pipelineManager.getPipeline(c.getPipelineID());
+ assertEquals(CLOSED, pipeline.getPipelineState());
+ }
+ }
+
+ @Test
+ public void testExcludedNodesPassedToCreatePipelineIfProvided()
+ throws IOException {
+ PipelineManager pipelineManagerSpy = Mockito.spy(pipelineManager);
+ provider = new WritableECContainerProvider(
+ conf, pipelineManagerSpy, containerManager, pipelineChoosingPolicy);
+ ExcludeList excludeList = new ExcludeList();
+
+ // EmptyList should be passed if there are no nodes excluded.
+ ContainerInfo container = provider.getContainer(
+ 1, repConfig, OWNER, excludeList);
+ assertNotNull(container);
+
+ verify(pipelineManagerSpy).createPipeline(repConfig,
+ Collections.emptyList(), Collections.emptyList());
+
+ // If nodes are excluded then the excluded nodes should be passed through to
+ // the create pipeline call.
+ excludeList.addDatanode(MockDatanodeDetails.randomDatanodeDetails());
+ List<DatanodeDetails> excludedNodes =
+ new ArrayList<>(excludeList.getDatanodes());
+
+ container = provider.getContainer(
+ 1, repConfig, OWNER, excludeList);
+ assertNotNull(container);
+ verify(pipelineManagerSpy).createPipeline(repConfig, excludedNodes,
+ Collections.emptyList());
+ }
+
+ private ContainerInfo createContainer(Pipeline pipeline,
+ ReplicationConfig repConf, long containerID) {
+ return new ContainerInfo.Builder()
+ .setContainerID(containerID)
+ .setOwner(OWNER)
+ .setReplicationConfig(repConf)
+ .setState(HddsProtos.LifeCycleState.OPEN)
+ .setPipelineID(pipeline.getId())
+ .setNumberOfKeys(0)
+ .setUsedBytes(0)
+ .setSequenceId(0)
+ .setDeleteTransactionId(0)
+ .build();
+ }
+
+ private long getMaxContainerSize() {
+ return (long)conf.getStorageSize(
+ ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+ ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, BYTES);
+ }
+
+}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
index af7337a..d72682a 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
@@ -21,6 +21,7 @@
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto;
@@ -409,9 +410,10 @@
@Override
public List<ContainerInfo> listContainer(long startContainerID,
int count, HddsProtos.LifeCycleState state,
- HddsProtos.ReplicationFactor factor) throws IOException {
+ HddsProtos.ReplicationType repType,
+ ReplicationConfig replicationConfig) throws IOException {
return storageContainerLocationClient.listContainer(
- startContainerID, count, state, factor);
+ startContainerID, count, state, repType, replicationConfig);
}
/**
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
index fd97153d..37bea18 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
@@ -124,6 +124,9 @@
private static String buildReplicaDetails(ContainerReplicaInfo replica) {
StringBuilder sb = new StringBuilder();
sb.append("State: " + replica.getState() + ";");
+ if (replica.getReplicaIndex() != -1) {
+ sb.append(" ReplicaIndex: " + replica.getReplicaIndex() + ";");
+ }
sb.append(" Origin: " + replica.getPlaceOfBirth().toString() + ";");
sb.append(" Location: "
+ buildDatanodeDetails(replica.getDatanodeDetails()));
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 3d776bb..b120fe4 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
@@ -20,7 +20,11 @@
import java.io.IOException;
import java.util.List;
+import com.google.common.base.Strings;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
@@ -65,9 +69,14 @@
"DELETING, DELETED)")
private HddsProtos.LifeCycleState state;
- @Option(names = {"--factor"},
- description = "Container factor(ONE, THREE)")
- private HddsProtos.ReplicationFactor factor;
+ @Option(names = {"-t", "--type"},
+ description = "Replication Type (RATIS, STAND_ALONE or EC)")
+ private HddsProtos.ReplicationType type;
+
+ @Option(names = {"-r", "--replication", "--factor"},
+ description = "Container replication (ONE, THREE for Ratis, " +
+ "rs-6-3-1024k for EC)")
+ private String replication;
private static final ObjectWriter WRITER;
@@ -90,8 +99,19 @@
@Override
public void execute(ScmClient scmClient) throws IOException {
+ if (!Strings.isNullOrEmpty(replication) && type == null) {
+ // Set type to RATIS as that is what any command prior to this change
+ // would have expected.
+ type = HddsProtos.ReplicationType.RATIS;
+ }
+ ReplicationConfig repConfig = null;
+ if (!Strings.isNullOrEmpty(replication)) {
+ repConfig = ReplicationConfig.parse(
+ ReplicationType.fromProto(type),
+ replication, new OzoneConfiguration());
+ }
List<ContainerInfo> containerList =
- scmClient.listContainer(startId, count, state, factor);
+ scmClient.listContainer(startId, count, state, type, repConfig);
// Output data list
for (ContainerInfo container : containerList) {
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java
index faea21e..86299ee 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java
@@ -55,7 +55,14 @@
@Override
public void execute(ScmClient scmClient) throws IOException {
- if (type == HddsProtos.ReplicationType.CHAINED) {
+ // Once we support creating EC containers/pipelines from the client, the
+ // client should check if SCM is able to fulfil the request, and
+ // understands an EcReplicationConfig. For that we also need to have SCM's
+ // version here from ScmInfo response.
+ // As I see there is no way to specify ECReplicationConfig properly here
+ // so failing the request if type is EC, seems to be safe.
+ if (type == HddsProtos.ReplicationType.CHAINED
+ || type == HddsProtos.ReplicationType.EC) {
throw new IllegalArgumentException(type.name()
+ " is not supported yet.");
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java
index a91af50..28e0eb9 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java
@@ -21,6 +21,8 @@
import com.google.common.base.Strings;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -39,23 +41,42 @@
versionProvider = HddsVersionProvider.class)
public class ListPipelinesSubcommand extends ScmSubcommand {
- @CommandLine.Option(names = {"-ffc", "--filterByFactor"},
- description = "Filter listed pipelines by Factor(ONE/one)",
+ @CommandLine.Option(names = {"-t", "--type"},
+ description = "Filter listed pipelines by replication type, RATIS or EC",
defaultValue = "")
- private String factor;
+ private String replicationType;
- @CommandLine.Option(names = {"-fst", "--filterByState"},
- description = "Filter listed pipelines by State(OPEN/CLOSE)",
+ @CommandLine.Option(
+ names = {"-r", "--replication", "-ffc", "--filterByFactor"},
+ description = "Filter listed pipelines by replication, eg ONE, THREE or "
+ + "for EC rs-3-2-1024k",
+ defaultValue = "")
+ private String replication;
+
+ @CommandLine.Option(names = {"-s", "--state", "-fst", "--filterByState"},
+ description = "Filter listed pipelines by State, eg OPEN, CLOSED",
defaultValue = "")
private String state;
@Override
public void execute(ScmClient scmClient) throws IOException {
Stream<Pipeline> stream = scmClient.listPipelines().stream();
- if (!Strings.isNullOrEmpty(factor)) {
+ if (!Strings.isNullOrEmpty(replication)) {
+ if (Strings.isNullOrEmpty(replicationType)) {
+ throw new IOException(
+ "ReplicationType cannot be null if replication is passed");
+ }
+ ReplicationConfig repConfig =
+ ReplicationConfig.parse(ReplicationType.valueOf(replicationType),
+ replication, new OzoneConfiguration());
stream = stream.filter(
- p -> ReplicationConfig.getLegacyFactor(p.getReplicationConfig())
- .toString().compareToIgnoreCase(factor) == 0);
+ p -> p.getReplicationConfig().equals(repConfig));
+ } else if (!Strings.isNullOrEmpty(replicationType)) {
+ stream = stream.filter(
+ p -> p.getReplicationConfig()
+ .getReplicationType()
+ .toString()
+ .compareToIgnoreCase(replicationType) == 0);
}
if (!Strings.isNullOrEmpty(state)) {
stream = stream.filter(p -> p.getPipelineState().toString()
diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestInfoSubCommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestInfoSubCommand.java
index 23a1a0b..c604019 100644
--- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestInfoSubCommand.java
+++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestInfoSubCommand.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hdds.scm.cli.container;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -81,8 +82,19 @@
@Test
public void testReplicasIncludedInOutput() throws Exception {
+ testReplicaIncludedInOutput(false);
+ }
+
+ @Test
+ public void testReplicaIndexInOutput() throws Exception {
+ testReplicaIncludedInOutput(true);
+ }
+
+
+ private void testReplicaIncludedInOutput(boolean includeIndex)
+ throws IOException {
Mockito.when(scmClient.getContainerReplicas(anyLong()))
- .thenReturn(getReplicas());
+ .thenReturn(getReplicas(includeIndex));
cmd = new InfoSubcommand();
CommandLine c = new CommandLine(cmd);
c.parseArgs("1", "--replicas");
@@ -94,7 +106,7 @@
.filter(m -> m.getRenderedMessage().matches("(?s)^Replicas:.*"))
.collect(Collectors.toList());
Assert.assertEquals(1, replica.size());
-
+
// Ensure each DN UUID is mentioned in the message:
for (DatanodeDetails dn : datanodes) {
Pattern pattern = Pattern.compile(".*" + dn.getUuid().toString() + ".*",
@@ -102,6 +114,11 @@
Matcher matcher = pattern.matcher(replica.get(0).getRenderedMessage());
Assert.assertTrue(matcher.matches());
}
+ // Ensure ReplicaIndex is not mentioned as it was not passed in the proto:
+ Pattern pattern = Pattern.compile(".*ReplicaIndex.*",
+ Pattern.DOTALL);
+ Matcher matcher = pattern.matcher(replica.get(0).getRenderedMessage());
+ Assert.assertEquals(includeIndex, matcher.matches());
}
@Test
@@ -151,7 +168,20 @@
@Test
public void testReplicasOutputWithJson() throws IOException {
Mockito.when(scmClient.getContainerReplicas(anyLong()))
- .thenReturn(getReplicas());
+ .thenReturn(getReplicas(true));
+ testJsonOutput();
+ }
+
+ @Test
+ public void testECContainerReplicasOutputWithJson() throws IOException {
+ Mockito.when(scmClient.getContainerReplicas(anyLong()))
+ .thenReturn(getReplicas(true));
+ Mockito.when(scmClient.getContainerWithPipeline(anyLong()))
+ .thenReturn(getECContainerWithPipeline());
+ testJsonOutput();
+ }
+
+ private void testJsonOutput() throws IOException {
cmd = new InfoSubcommand();
CommandLine c = new CommandLine(cmd);
c.parseArgs("1", "--json");
@@ -169,20 +199,29 @@
Matcher matcher = pattern.matcher(json);
Assert.assertTrue(matcher.matches());
}
+ Pattern pattern = Pattern.compile(".*replicaIndex.*",
+ Pattern.DOTALL);
+ Matcher matcher = pattern.matcher(json);
+ Assert.assertTrue(matcher.matches());
}
- private List<ContainerReplicaInfo> getReplicas() {
+
+ private List<ContainerReplicaInfo> getReplicas(boolean includeIndex) {
List<ContainerReplicaInfo> replicas = new ArrayList<>();
for (DatanodeDetails dn : datanodes) {
- ContainerReplicaInfo container = new ContainerReplicaInfo.Builder()
+ ContainerReplicaInfo.Builder container
+ = new ContainerReplicaInfo.Builder()
.setContainerID(1)
.setBytesUsed(1234)
.setState("CLOSED")
.setPlaceOfBirth(dn.getUuid())
.setDatanodeDetails(dn)
.setKeyCount(1)
- .setSequenceId(1).build();
- replicas.add(container);
+ .setSequenceId(1);
+ if (includeIndex) {
+ container.setReplicaIndex(4);
+ }
+ replicas.add(container.build());
}
return replicas;
}
@@ -207,6 +246,26 @@
return new ContainerWithPipeline(container, pipeline);
}
+ private ContainerWithPipeline getECContainerWithPipeline() {
+ Pipeline pipeline = new Pipeline.Builder()
+ .setState(Pipeline.PipelineState.CLOSED)
+ .setReplicationConfig(new ECReplicationConfig(3, 2))
+ .setId(PipelineID.randomId())
+ .setNodes(datanodes)
+ .build();
+
+ ContainerInfo container = new ContainerInfo.Builder()
+ .setSequenceId(1)
+ .setPipelineID(pipeline.getId())
+ .setUsedBytes(1234)
+ .setReplicationConfig(RatisReplicationConfig.getInstance(THREE))
+ .setNumberOfKeys(1)
+ .setState(CLOSED)
+ .build();
+
+ return new ContainerWithPipeline(container, pipeline);
+ }
+
private List<DatanodeDetails> createDatanodeDetails(int count) {
List<DatanodeDetails> dns = new ArrayList<>();
for (int i = 0; i < count; i++) {
diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/pipeline/TestListPipelinesSubCommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/pipeline/TestListPipelinesSubCommand.java
new file mode 100644
index 0000000..acc3bda
--- /dev/null
+++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/pipeline/TestListPipelinesSubCommand.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.cli.pipeline;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import picocli.CommandLine;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the ListPipelineSubCommand class.
+ */
+public class TestListPipelinesSubCommand {
+
+ private ListPipelinesSubcommand cmd;
+ private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+ private final ByteArrayOutputStream errContent = new ByteArrayOutputStream();
+ private final PrintStream originalOut = System.out;
+ private final PrintStream originalErr = System.err;
+ private static final String DEFAULT_ENCODING = StandardCharsets.UTF_8.name();
+ private ScmClient scmClient;
+
+ @Before
+ public void setup() throws IOException {
+ cmd = new ListPipelinesSubcommand();
+ System.setOut(new PrintStream(outContent, false, DEFAULT_ENCODING));
+ System.setErr(new PrintStream(errContent, false, DEFAULT_ENCODING));
+
+ scmClient = mock(ScmClient.class);
+ Mockito.when(scmClient.listPipelines())
+ .thenAnswer(invocation -> createPipelines());
+ }
+
+ @After
+ public void tearDown() {
+ System.setOut(originalOut);
+ System.setErr(originalErr);
+ }
+
+ @Test
+ public void testAllPipelinesReturnedWithNoFilter() throws IOException {
+ CommandLine c = new CommandLine(cmd);
+ c.parseArgs();
+ cmd.execute(scmClient);
+ Assert.assertEquals(6, outContent.toString(DEFAULT_ENCODING)
+ .split(System.getProperty("line.separator")).length);
+ }
+
+ @Test
+ public void testOnlyOpenReturned() throws IOException {
+ CommandLine c = new CommandLine(cmd);
+ c.parseArgs("-s", "OPEN");
+ cmd.execute(scmClient);
+ String output = outContent.toString(DEFAULT_ENCODING);
+ Assert.assertEquals(3, output.split(
+ System.getProperty("line.separator")).length);
+ Assert.assertEquals(-1, output.indexOf("CLOSED"));
+ }
+
+ @Test(expected = IOException.class)
+ public void testExceptionIfReplicationWithoutType() throws IOException {
+ CommandLine c = new CommandLine(cmd);
+ c.parseArgs("-r", "THREE");
+ cmd.execute(scmClient);
+ }
+
+ @Test
+ public void testReplicationAndType() throws IOException {
+ CommandLine c = new CommandLine(cmd);
+ c.parseArgs("-r", "THREE", "-t", "RATIS");
+ cmd.execute(scmClient);
+
+ String output = outContent.toString(DEFAULT_ENCODING);
+ Assert.assertEquals(2, output.split(
+ System.getProperty("line.separator")).length);
+ Assert.assertEquals(-1, output.indexOf("EC"));
+ }
+
+ @Test
+ public void testReplicationAndTypeEC() throws IOException {
+ CommandLine c = new CommandLine(cmd);
+ c.parseArgs("-r", "rs-6-3-1024k", "-t", "EC");
+ cmd.execute(scmClient);
+
+ String output = outContent.toString(DEFAULT_ENCODING);
+ Assert.assertEquals(1, output.split(
+ System.getProperty("line.separator")).length);
+ Assert.assertEquals(-1,
+ output.indexOf("ReplicationConfig: RATIS"));
+ }
+
+ @Test
+ public void testReplicationAndTypeAndState() throws IOException {
+ CommandLine c = new CommandLine(cmd);
+ c.parseArgs("-r", "THREE", "-t", "RATIS", "-s", "OPEN");
+ cmd.execute(scmClient);
+
+ String output = outContent.toString(DEFAULT_ENCODING);
+ Assert.assertEquals(1, output.split(
+ System.getProperty("line.separator")).length);
+ Assert.assertEquals(-1, output.indexOf("CLOSED"));
+ Assert.assertEquals(-1, output.indexOf("EC"));
+ }
+
+ private List<Pipeline> createPipelines() {
+ List<Pipeline> pipelines = new ArrayList<>();
+ pipelines.add(createPipeline(StandaloneReplicationConfig.getInstance(ONE),
+ Pipeline.PipelineState.OPEN));
+ pipelines.add(createPipeline(RatisReplicationConfig.getInstance(THREE),
+ Pipeline.PipelineState.OPEN));
+ pipelines.add(createPipeline(RatisReplicationConfig.getInstance(THREE),
+ Pipeline.PipelineState.CLOSED));
+
+ pipelines.add(createPipeline(
+ new ECReplicationConfig(3, 2), Pipeline.PipelineState.OPEN));
+ pipelines.add(createPipeline(
+ new ECReplicationConfig(3, 2), Pipeline.PipelineState.CLOSED));
+ pipelines.add(createPipeline(
+ new ECReplicationConfig(6, 3), Pipeline.PipelineState.CLOSED));
+
+ return pipelines;
+ }
+
+ private Pipeline createPipeline(ReplicationConfig repConfig,
+ Pipeline.PipelineState state) {
+ return new Pipeline.Builder()
+ .setId(PipelineID.randomId())
+ .setCreateTimestamp(System.currentTimeMillis())
+ .setState(state)
+ .setReplicationConfig(repConfig)
+ .setNodes(createDatanodeDetails(1))
+ .build();
+ }
+
+ private List<DatanodeDetails> createDatanodeDetails(int count) {
+ List<DatanodeDetails> dns = new ArrayList<>();
+ for (int i = 0; i < count; i++) {
+ HddsProtos.DatanodeDetailsProto dnd =
+ HddsProtos.DatanodeDetailsProto.newBuilder()
+ .setHostName("host" + i)
+ .setIpAddress("1.2.3." + i + 1)
+ .setNetworkLocation("/default")
+ .setNetworkName("host" + i)
+ .addPorts(HddsProtos.Port.newBuilder()
+ .setName("ratis").setValue(5678).build())
+ .setUuid(UUID.randomUUID().toString())
+ .build();
+ dns.add(DatanodeDetails.getFromProtoBuf(dnd));
+ }
+ return dns;
+ }
+}
+
+
diff --git a/hadoop-ozone/client/pom.xml b/hadoop-ozone/client/pom.xml
index 9ba3af6..f072308 100644
--- a/hadoop-ozone/client/pom.xml
+++ b/hadoop-ozone/client/pom.xml
@@ -39,6 +39,10 @@
<artifactId>ozone-common</artifactId>
</dependency>
<dependency>
+ <groupId>org.apache.ozone</groupId>
+ <artifactId>hdds-erasurecode</artifactId>
+ </dependency>
+ <dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
index 43d28f0..6876059 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.ozone.client;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConsts;
@@ -56,6 +57,7 @@
* Bucket encryption key name.
*/
private String bucketEncryptionKey;
+ private DefaultReplicationConfig defaultReplicationConfig;
private final String sourceVolume;
private final String sourceBucket;
@@ -80,15 +82,16 @@
* @param sourceBucket
* @param quotaInBytes Bucket quota in bytes.
* @param quotaInNamespace Bucket quota in counts.
- * @param bucketLayout Bucket Layouts.
+ * @param bucketLayout bucket layout.
* @param owner owner of the bucket.
+ * @param defaultReplicationConfig default replication config.
*/
@SuppressWarnings("parameternumber")
private BucketArgs(Boolean versioning, StorageType storageType,
List<OzoneAcl> acls, Map<String, String> metadata,
String bucketEncryptionKey, String sourceVolume, String sourceBucket,
long quotaInBytes, long quotaInNamespace, BucketLayout bucketLayout,
- String owner) {
+ String owner, DefaultReplicationConfig defaultReplicationConfig) {
this.acls = acls;
this.versioning = versioning;
this.storageType = storageType;
@@ -100,6 +103,7 @@
this.quotaInNamespace = quotaInNamespace;
this.bucketLayout = bucketLayout;
this.owner = owner;
+ this.defaultReplicationConfig = defaultReplicationConfig;
}
/**
@@ -144,6 +148,14 @@
}
/**
+ * Returns the bucket default replication config.
+ * @return bucket's default Replication Config.
+ */
+ public DefaultReplicationConfig getDefaultReplicationConfig() {
+ return this.defaultReplicationConfig;
+ }
+
+ /**
* Returns new builder class that builds a OmBucketInfo.
*
* @return Builder
@@ -205,6 +217,7 @@
private long quotaInNamespace;
private BucketLayout bucketLayout;
private String owner;
+ private DefaultReplicationConfig defaultReplicationConfig;
public Builder() {
metadata = new HashMap<>();
@@ -267,6 +280,12 @@
return this;
}
+ public BucketArgs.Builder setDefaultReplicationConfig(
+ DefaultReplicationConfig defaultReplConfig) {
+ defaultReplicationConfig = defaultReplConfig;
+ return this;
+ }
+
/**
* Constructs the BucketArgs.
* @return instance of BucketArgs.
@@ -274,7 +293,7 @@
public BucketArgs build() {
return new BucketArgs(versioning, storageType, acls, metadata,
bucketEncryptionKey, sourceVolume, sourceBucket, quotaInBytes,
- quotaInNamespace, bucketLayout, owner);
+ quotaInNamespace, bucketLayout, owner, defaultReplicationConfig);
}
}
}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index e49aeb7..9ae4185 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -22,6 +22,7 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
import org.apache.hadoop.hdds.client.OzoneQuota;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
@@ -80,7 +81,7 @@
/**
* Default replication factor to be used while creating keys.
*/
- private final ReplicationConfig defaultReplication;
+ private ReplicationConfig defaultReplication;
/**
* Type of storage to be used for this bucket.
@@ -151,7 +152,8 @@
this.volumeName = volumeName;
this.name = bucketName;
- this.defaultReplication = ReplicationConfig.getDefault(conf);
+ // Bucket level replication is not configured by default.
+ this.defaultReplication = null;
this.proxy = proxy;
this.ozoneObj = OzoneObjInfo.Builder.newBuilder()
@@ -225,6 +227,32 @@
this.bucketLayout = bucketLayout;
}
+ @SuppressWarnings("parameternumber")
+ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
+ String volumeName, String bucketName, StorageType storageType,
+ Boolean versioning, long creationTime, long modificationTime,
+ Map<String, String> metadata, String encryptionKeyName,
+ String sourceVolume, String sourceBucket, long usedBytes,
+ long usedNamespace, long quotaInBytes, long quotaInNamespace,
+ BucketLayout bucketLayout, String owner,
+ DefaultReplicationConfig defaultReplicationConfig) {
+ this(conf, proxy, volumeName, bucketName, storageType, versioning,
+ creationTime, modificationTime, metadata, encryptionKeyName,
+ sourceVolume, sourceBucket, usedBytes, usedNamespace, quotaInBytes,
+ quotaInNamespace, bucketLayout, owner);
+ this.bucketLayout = bucketLayout;
+ if (defaultReplicationConfig != null) {
+ this.defaultReplication =
+ defaultReplicationConfig.getType() == ReplicationType.EC ?
+ defaultReplicationConfig.getEcReplicationConfig() :
+ ReplicationConfig
+ .fromTypeAndFactor(defaultReplicationConfig.getType(),
+ defaultReplicationConfig.getFactor());
+ } else {
+ // Bucket level replication is not configured by default.
+ this.defaultReplication = null;
+ }
+ }
@SuppressWarnings("checkstyle:ParameterNumber")
public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
@@ -508,6 +536,17 @@
}
/**
+ * Sets/Changes the default replication config of this Bucket.
+ *
+ * @param replicationConfig Replication config that can be applied to bucket.
+ * @throws IOException
+ */
+ public void setReplicationConfig(ReplicationConfig replicationConfig)
+ throws IOException {
+ proxy.setReplicationConfig(volumeName, name, replicationConfig);
+ }
+
+ /**
* Creates a new key in the bucket, with default replication type RATIS and
* with replication factor THREE.
* @param key Name of the key to be created.
@@ -1269,4 +1308,8 @@
public BucketLayout getBucketLayout() {
return bucketLayout;
}
+
+ public ReplicationConfig getReplicationConfig() {
+ return this.defaultReplication;
+ }
}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java
index e37969d..f0302c8 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKey.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.ozone.client;
+import com.fasterxml.jackson.annotation.JsonIgnore;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
@@ -154,12 +155,14 @@
*/
@Deprecated
+ @JsonIgnore
public ReplicationType getReplicationType() {
return ReplicationType
.fromProto(replicationConfig.getReplicationType());
}
@Deprecated
+ @JsonIgnore
public int getReplicationFactor() {
return replicationConfig.getRequiredNodes();
}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BadDataLocationException.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BadDataLocationException.java
new file mode 100644
index 0000000..dc8f7b0
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BadDataLocationException.java
@@ -0,0 +1,56 @@
+/**
+ * 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.client.io;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+
+import java.io.IOException;
+
+/**
+ * Exception used to indicate a problem with a specific block location, allowing
+ * the failed location to be communicated back to the caller.
+ */
+public class BadDataLocationException extends IOException {
+
+ private DatanodeDetails failedLocation;
+
+ public BadDataLocationException(DatanodeDetails dn) {
+ super();
+ failedLocation = dn;
+ }
+
+ public BadDataLocationException(DatanodeDetails dn, String message) {
+ super(message);
+ failedLocation = dn;
+ }
+
+ public BadDataLocationException(DatanodeDetails dn, String message,
+ Throwable ex) {
+ super(message, ex);
+ failedLocation = dn;
+ }
+
+ public BadDataLocationException(DatanodeDetails dn, Throwable ex) {
+ super(ex);
+ failedLocation = dn;
+ }
+
+ public DatanodeDetails getFailedLocation() {
+ return failedLocation;
+ }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
new file mode 100644
index 0000000..d1bf7f3
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java
@@ -0,0 +1,55 @@
+/*
+ * 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.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+
+import java.util.function.Function;
+
+/**
+ * Interface used by classes which need to obtain BlockStream instances.
+ */
+public interface BlockInputStreamFactory {
+
+ /**
+ * Create a new BlockInputStream based on the replication Config. If the
+ * replication Config indicates the block is EC, then it will create an
+ * ECBlockInputStream, otherwise a BlockInputStream will be returned.
+ * @param repConfig The replication Config
+ * @param blockInfo The blockInfo representing the block.
+ * @param pipeline The pipeline to be used for reading the block
+ * @param token The block Access Token
+ * @param verifyChecksum Whether to verify checksums or not.
+ * @param xceiverFactory Factory to create the xceiver in the client
+ * @param refreshFunction Function to refresh the pipeline if needed
+ * @return BlockExtendedInputStream of the correct type.
+ */
+ BlockExtendedInputStream create(ReplicationConfig repConfig,
+ OmKeyLocationInfo blockInfo, Pipeline pipeline,
+ Token<OzoneBlockTokenIdentifier> token, boolean verifyChecksum,
+ XceiverClientFactory xceiverFactory,
+ Function<BlockID, Pipeline> refreshFunction);
+
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java
new file mode 100644
index 0000000..104e5bc
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Factory class to create various BlockStream instances.
+ */
+public class BlockInputStreamFactoryImpl implements BlockInputStreamFactory {
+
+ private ECBlockInputStreamFactory ecBlockStreamFactory;
+
+ public static BlockInputStreamFactory getInstance(
+ ByteBufferPool byteBufferPool,
+ Supplier<ExecutorService> ecReconstructExecutorSupplier) {
+ return new BlockInputStreamFactoryImpl(byteBufferPool,
+ ecReconstructExecutorSupplier);
+ }
+
+ public BlockInputStreamFactoryImpl() {
+ this(new ElasticByteBufferPool(), Executors::newSingleThreadExecutor);
+ }
+
+ public BlockInputStreamFactoryImpl(ByteBufferPool byteBufferPool,
+ Supplier<ExecutorService> ecReconstructExecutorSupplier) {
+ this.ecBlockStreamFactory =
+ ECBlockInputStreamFactoryImpl.getInstance(this, byteBufferPool,
+ ecReconstructExecutorSupplier);
+ }
+
+ /**
+ * Create a new BlockInputStream based on the replication Config. If the
+ * replication Config indicates the block is EC, then it will create an
+ * ECBlockInputStream, otherwise a BlockInputStream will be returned.
+ * @param repConfig The replication Config
+ * @param blockInfo The blockInfo representing the block.
+ * @param pipeline The pipeline to be used for reading the block
+ * @param token The block Access Token
+ * @param verifyChecksum Whether to verify checksums or not.
+ * @param xceiverFactory Factory to create the xceiver in the client
+ * @param refreshFunction Function to refresh the pipeline if needed
+ * @return BlockExtendedInputStream of the correct type.
+ */
+ public BlockExtendedInputStream create(ReplicationConfig repConfig,
+ OmKeyLocationInfo blockInfo, Pipeline pipeline,
+ Token<OzoneBlockTokenIdentifier> token, boolean verifyChecksum,
+ XceiverClientFactory xceiverFactory,
+ Function<BlockID, Pipeline> refreshFunction) {
+ if (repConfig.getReplicationType().equals(HddsProtos.ReplicationType.EC)) {
+ return new ECBlockInputStreamProxy((ECReplicationConfig)repConfig,
+ blockInfo, verifyChecksum, xceiverFactory, refreshFunction,
+ ecBlockStreamFactory);
+ } else {
+ return new BlockInputStream(blockInfo.getBlockID(), blockInfo.getLength(),
+ pipeline, token, verifyChecksum, xceiverFactory, refreshFunction);
+ }
+ }
+
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java
index 5aff685..2b3b756 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java
@@ -42,7 +42,7 @@
*
* The base implementation is handling Ratis-3 writes, with a single stream,
* but there can be other implementations that are using a different way.
- * */
+ */
public class BlockOutputStreamEntry extends OutputStream {
private final OzoneClientConfig config;
@@ -276,7 +276,8 @@
* here.
* @param id the last know ID of the block.
*/
- void updateBlockID(BlockID id) {
+ @VisibleForTesting
+ protected void updateBlockID(BlockID id) {
this.blockID = id;
}
@@ -352,13 +353,11 @@
return this;
}
-
public Builder setLength(long len) {
this.length = len;
return this;
}
-
public Builder setBufferPool(BufferPool pool) {
this.bufferPool = pool;
return this;
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
index 7985bf7..bdfa3f2 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
@@ -102,7 +102,7 @@
.setMultipartUploadPartNumber(partNumber).build();
this.requestID = requestId;
this.openID = openID;
- this.excludeList = new ExcludeList();
+ this.excludeList = createExcludeList();
this.bufferPool =
new BufferPool(config.getStreamBufferSize(),
@@ -112,6 +112,10 @@
.createByteBufferConversion(unsafeByteBufferConversion));
}
+ ExcludeList createExcludeList() {
+ return new ExcludeList();
+ }
+
/**
* A constructor for testing purpose only.
*
@@ -216,7 +220,7 @@
.setLength(streamEntry.getCurrentPosition())
.setOffset(0)
.setToken(streamEntry.getToken())
- .setPipeline(streamEntry.getPipelineForOMLocationReport())
+ .setPipeline(streamEntry.getPipeline())
.build();
locationInfoList.add(info);
}
@@ -319,7 +323,8 @@
if (keyArgs != null) {
// in test, this could be null
long length = getKeyLength();
- Preconditions.checkArgument(offset == length);
+ Preconditions.checkArgument(offset == length,
+ "Epected offset: " + offset + " expected len: " + length);
keyArgs.setDataSize(length);
keyArgs.setLocationInfoList(getLocationInfoList());
// When the key is multipart upload part file upload, we should not
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
new file mode 100644
index 0000000..b0e9755
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java
@@ -0,0 +1,420 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.io;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+/**
+ * Class to read data from an EC Block Group.
+ */
+public class ECBlockInputStream extends BlockExtendedInputStream {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ECBlockInputStream.class);
+
+ private final ECReplicationConfig repConfig;
+ private final int ecChunkSize;
+ private final long stripeSize;
+ private final BlockInputStreamFactory streamFactory;
+ private final boolean verifyChecksum;
+ private final XceiverClientFactory xceiverClientFactory;
+ private final Function<BlockID, Pipeline> refreshFunction;
+ private final OmKeyLocationInfo blockInfo;
+ private final DatanodeDetails[] dataLocations;
+ private final BlockExtendedInputStream[] blockStreams;
+ private final int maxLocations;
+
+ private long position = 0;
+ private boolean closed = false;
+ private boolean seeked = false;
+
+ protected OmKeyLocationInfo getBlockInfo() {
+ return blockInfo;
+ }
+
+ protected ECReplicationConfig getRepConfig() {
+ return repConfig;
+ }
+
+ protected DatanodeDetails[] getDataLocations() {
+ return dataLocations;
+ }
+
+ protected long getStripeSize() {
+ return stripeSize;
+ }
+
+ /**
+ * Returns the number of available data locations, taking account of the
+ * expected number of locations. Eg, if the block is less than 1 EC chunk,
+ * we only expect 1 data location. If it is between 1 and 2 chunks, we expect
+ * there to be 2 locations, and so on.
+ * @param expectedLocations The maximum number of allowed data locations,
+ * depending on the block size.
+ * @return The number of available data locations.
+ */
+ protected int availableDataLocations(int expectedLocations) {
+ int count = 0;
+ for (int i = 0; i < repConfig.getData() && i < expectedLocations; i++) {
+ if (dataLocations[i] != null) {
+ count++;
+ }
+ }
+ return count;
+ }
+
+ protected int availableParityLocations() {
+ int count = 0;
+ for (int i = repConfig.getData();
+ i < repConfig.getData() + repConfig.getParity(); i++) {
+ if (dataLocations[i] != null) {
+ count++;
+ }
+ }
+ return count;
+ }
+
+ public ECBlockInputStream(ECReplicationConfig repConfig,
+ OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+ XceiverClientFactory xceiverClientFactory, Function<BlockID,
+ Pipeline> refreshFunction, BlockInputStreamFactory streamFactory) {
+ this.repConfig = repConfig;
+ this.ecChunkSize = repConfig.getEcChunkSize();
+ this.verifyChecksum = verifyChecksum;
+ this.blockInfo = blockInfo;
+ this.streamFactory = streamFactory;
+ this.xceiverClientFactory = xceiverClientFactory;
+ this.refreshFunction = refreshFunction;
+ this.maxLocations = repConfig.getData() + repConfig.getParity();
+ this.dataLocations = new DatanodeDetails[repConfig.getRequiredNodes()];
+ this.blockStreams =
+ new BlockExtendedInputStream[repConfig.getRequiredNodes()];
+
+ this.stripeSize = (long)ecChunkSize * repConfig.getData();
+ setBlockLocations(this.blockInfo.getPipeline());
+ }
+
+ public synchronized boolean hasSufficientLocations() {
+ // The number of locations needed is a function of the EC Chunk size. If the
+ // block length is <= the chunk size, we should only have location 1. If it
+ // is greater than the chunk size but less than chunk_size * 2, then we must
+ // have two locations. If it is greater than chunk_size * data_num, then we
+ // must have all data_num locations.
+ // We only consider data locations here.
+ int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+ return expectedDataBlocks == availableDataLocations(expectedDataBlocks);
+ }
+
+ protected int calculateExpectedDataBlocks(ECReplicationConfig rConfig) {
+ return ECBlockInputStreamProxy.expectedDataLocations(rConfig, getLength());
+ }
+
+ /**
+ * Using the current position, returns the index of the blockStream we should
+ * be reading from. This is the index in the internal array holding the
+ * stream reference. The block group index will be one greater than this.
+ * @return
+ */
+ protected int currentStreamIndex() {
+ return (int)((position / ecChunkSize) % repConfig.getData());
+ }
+
+ /**
+ * Uses the current position and ecChunkSize to determine which of the
+ * internal block streams the next read should come from. Also opens the
+ * stream if it has not been opened already.
+ * @return BlockInput stream to read from.
+ */
+ protected BlockExtendedInputStream getOrOpenStream(int locationIndex) {
+ BlockExtendedInputStream stream = blockStreams[locationIndex];
+ if (stream == null) {
+ // To read an EC block, we create a STANDALONE pipeline that contains the
+ // single location for the block index we want to read. The EC blocks are
+ // indexed from 1 to N, however the data locations are stored in the
+ // dataLocations array indexed from zero.
+ Pipeline pipeline = Pipeline.newBuilder()
+ .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE))
+ .setNodes(Arrays.asList(dataLocations[locationIndex]))
+ .setId(PipelineID.randomId())
+ .setState(Pipeline.PipelineState.CLOSED)
+ .build();
+
+ OmKeyLocationInfo blkInfo = new OmKeyLocationInfo.Builder()
+ .setBlockID(blockInfo.getBlockID())
+ .setLength(internalBlockLength(locationIndex + 1))
+ .setPipeline(blockInfo.getPipeline())
+ .setToken(blockInfo.getToken())
+ .setPartNumber(blockInfo.getPartNumber())
+ .build();
+ stream = streamFactory.create(
+ StandaloneReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE),
+ blkInfo, pipeline,
+ blockInfo.getToken(), verifyChecksum, xceiverClientFactory,
+ refreshFunction);
+ blockStreams[locationIndex] = stream;
+ }
+ return stream;
+ }
+
+ /**
+ * Returns the length of the Nth block in the block group, taking account of a
+ * potentially partial last stripe. Note that the internal block index is
+ * numbered starting from 1.
+ * @param index - Index number of the internal block, starting from 1
+ * @return
+ */
+ protected long internalBlockLength(int index) {
+ long lastStripe = blockInfo.getLength() % stripeSize;
+ long blockSize = (blockInfo.getLength() - lastStripe) / repConfig.getData();
+ long lastCell = lastStripe / ecChunkSize + 1;
+ long lastCellLength = lastStripe % ecChunkSize;
+
+ if (index > repConfig.getData()) {
+ // Its a parity block and their size is driven by the size of the
+ // first block of the block group. All parity blocks have the same size
+ // as block_1.
+ index = 1;
+ }
+
+ if (index < lastCell) {
+ return blockSize + ecChunkSize;
+ } else if (index == lastCell) {
+ return blockSize + lastCellLength;
+ } else {
+ return blockSize;
+ }
+ }
+
+ private void setBlockLocations(Pipeline pipeline) {
+ for (DatanodeDetails node : pipeline.getNodes()) {
+ int index = pipeline.getReplicaIndex(node);
+ addBlockLocation(index, node);
+ }
+ }
+
+ private void addBlockLocation(int index, DatanodeDetails location) {
+ if (index > maxLocations) {
+ throw new IndexOutOfBoundsException("The index " + index + " is greater "
+ + "than the EC Replication Config (" + repConfig + ")");
+ }
+ dataLocations[index - 1] = location;
+ }
+
+ protected long blockLength() {
+ return blockInfo.getLength();
+ }
+
+ protected long remaining() {
+ return blockLength() - position;
+ }
+
+ /**
+ * Read from the internal BlockInputStreams one EC cell at a time into the
+ * strategy buffer. This call may read from several internal BlockInputStreams
+ * if there is sufficient space in the buffer.
+ * @param strategy
+ * @return
+ * @throws IOException
+ */
+ @Override
+ protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+ throws IOException {
+ Preconditions.checkArgument(strategy != null);
+ checkOpen();
+
+ if (remaining() == 0) {
+ return EOF;
+ }
+
+ int totalRead = 0;
+ while (strategy.getTargetLength() > 0 && remaining() > 0) {
+ try {
+ int currentIndex = currentStreamIndex();
+ BlockExtendedInputStream stream = getOrOpenStream(currentIndex);
+ int read = readFromStream(stream, strategy);
+ totalRead += read;
+ position += read;
+ } catch (IOException ioe) {
+ throw new BadDataLocationException(
+ dataLocations[currentStreamIndex()], ioe);
+ }
+ }
+ return totalRead;
+ }
+
+ @Override
+ public synchronized long getRemaining() {
+ return blockInfo.getLength() - position;
+ }
+
+ @Override
+ public synchronized long getLength() {
+ return blockInfo.getLength();
+ }
+
+ @Override
+ public BlockID getBlockID() {
+ return blockInfo.getBlockID();
+ }
+
+ protected void seekStreamIfNecessary(BlockExtendedInputStream stream,
+ long partialChunkSize) throws IOException {
+ if (seeked) {
+ // Seek on the underlying streams is performed lazily, as there is a
+ // possibility a read after a seek may only read a small amount of data.
+ // Once this block stream has been seeked, we always check the position,
+ // but in the usual case, where there are no seeks at all, we don't need
+ // to do this extra work.
+ long basePosition = (position / stripeSize) * (long)ecChunkSize;
+ long streamPosition = basePosition + partialChunkSize;
+ if (streamPosition != stream.getPos()) {
+ // This ECBlockInputStream has been seeked, so the underlying
+ // block stream is no longer at the correct position. Therefore we need
+ // to seek it too.
+ stream.seek(streamPosition);
+ }
+ }
+ }
+
+ /**
+ * Read the most allowable amount of data from the current stream. This
+ * ensures we don't read past the end of an EC cell or the overall block
+ * group length.
+ * @param stream Stream to read from
+ * @param strategy The ReaderStrategy to read data into
+ * @return
+ * @throws IOException
+ */
+ private int readFromStream(BlockExtendedInputStream stream,
+ ByteReaderStrategy strategy)
+ throws IOException {
+ long partialPosition = position % ecChunkSize;
+ seekStreamIfNecessary(stream, partialPosition);
+ long ecLimit = ecChunkSize - partialPosition;
+ // Free space in the buffer to read into
+ long bufLimit = strategy.getTargetLength();
+ // How much we can read, the lower of the EC Cell, buffer and overall block
+ // remaining.
+ int expectedRead = (int)Math.min(Math.min(ecLimit, bufLimit), remaining());
+ int actualRead = strategy.readFromBlock(stream, expectedRead);
+ if (actualRead == -1) {
+ // The Block Stream reached EOF, but we did not expect it to, so the block
+ // might be corrupt.
+ throw new IOException("Expected to read " + expectedRead + " but got EOF"
+ + " from blockGroup " + stream.getBlockID() + " index "
+ + currentStreamIndex() + 1);
+ }
+ return actualRead;
+ }
+
+ /**
+ * Verify that the input stream is open.
+ * @throws IOException if the connection is closed.
+ */
+ private void checkOpen() throws IOException {
+ if (closed) {
+ throw new IOException(
+ ": " + FSExceptionMessages.STREAM_IS_CLOSED + " Block: "
+ + blockInfo.getBlockID());
+ }
+ }
+
+ @Override
+ public synchronized void close() {
+ closeStreams();
+ closed = true;
+ }
+
+ protected synchronized void closeStreams() {
+ for (int i = 0; i < blockStreams.length; i++) {
+ if (blockStreams[i] != null) {
+ try {
+ blockStreams[i].close();
+ blockStreams[i] = null;
+ } catch (IOException e) {
+ LOG.error("Failed to close stream {}", blockStreams[i], e);
+ }
+ }
+ }
+ // If the streams have been closed outside of a close() call, then it may
+ // be due to freeing resources. If they are reopened, then we will need to
+ // seek the stream to its expected position when the next read is attempted.
+ seeked = true;
+ }
+
+ @Override
+ public synchronized void unbuffer() {
+ for (BlockExtendedInputStream stream : blockStreams) {
+ if (stream != null) {
+ stream.unbuffer();
+ }
+ }
+ }
+
+ @Override
+ public synchronized void seek(long pos) throws IOException {
+ checkOpen();
+ if (pos < 0 || pos >= getLength()) {
+ if (pos == 0) {
+ // It is possible for length and pos to be zero in which case
+ // seek should return instead of throwing exception
+ return;
+ }
+ throw new EOFException(
+ "EOF encountered at pos: " + pos + " for block: "
+ + blockInfo.getBlockID());
+ }
+ position = pos;
+ seeked = true;
+ }
+
+ @Override
+ public synchronized long getPos() {
+ return position;
+ }
+
+ protected synchronized void setPos(long pos) {
+ position = pos;
+ }
+
+ @Override
+ public synchronized boolean seekToNewSource(long l) throws IOException {
+ return false;
+ }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java
new file mode 100644
index 0000000..6c39e93
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactory.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Interface used by factories which create ECBlockInput streams for
+ * reconstruction or non-reconstruction reads.
+ */
+public interface ECBlockInputStreamFactory {
+
+ /**
+ * Create a new EC InputStream based on the missingLocations boolean. If it is
+ * set to false, it indicates all locations are available and an
+ * ECBlockInputStream will be created. Otherwise an
+ * ECBlockReconstructedInputStream will be created.
+ * @param missingLocations Indicates if all the data locations are available
+ * or not, controlling the type of stream created
+ * @param failedLocations List of DatanodeDetails indicating locations we
+ * know are bad and should not be used.
+ * @param repConfig The replication Config
+ * @param blockInfo The blockInfo representing the block.
+ * @param verifyChecksum Whether to verify checksums or not.
+ * @param xceiverFactory Factory to create the xceiver in the client
+ * @param refreshFunction Function to refresh the pipeline if needed
+ * @return BlockExtendedInputStream of the correct type.
+ */
+ BlockExtendedInputStream create(boolean missingLocations,
+ List<DatanodeDetails> failedLocations, ReplicationConfig repConfig,
+ OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+ XceiverClientFactory xceiverFactory,
+ Function<BlockID, Pipeline> refreshFunction);
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java
new file mode 100644
index 0000000..470df0c
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamFactoryImpl.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Factory class to create various BlockStream instances.
+ */
+public final class ECBlockInputStreamFactoryImpl implements
+ ECBlockInputStreamFactory {
+
+ private final BlockInputStreamFactory inputStreamFactory;
+ private final ByteBufferPool byteBufferPool;
+ private final Supplier<ExecutorService> ecReconstructExecutorSupplier;
+
+ public static ECBlockInputStreamFactory getInstance(
+ BlockInputStreamFactory streamFactory, ByteBufferPool byteBufferPool,
+ Supplier<ExecutorService> ecReconstructExecutorSupplier) {
+ return new ECBlockInputStreamFactoryImpl(streamFactory, byteBufferPool,
+ ecReconstructExecutorSupplier);
+ }
+
+ private ECBlockInputStreamFactoryImpl(BlockInputStreamFactory streamFactory,
+ ByteBufferPool byteBufferPool,
+ Supplier<ExecutorService> ecReconstructExecutorSupplier) {
+ this.byteBufferPool = byteBufferPool;
+ this.inputStreamFactory = streamFactory;
+ this.ecReconstructExecutorSupplier = ecReconstructExecutorSupplier;
+ }
+
+ /**
+ * Create a new EC InputStream based on the missingLocations boolean. If it is
+ * set to false, it indicates all locations are available and an
+ * ECBlockInputStream will be created. Otherwise an
+ * ECBlockReconstructedInputStream will be created.
+ * @param missingLocations Indicates if all the data locations are available
+ * or not, controlling the type of stream created
+ * @param failedLocations List of DatanodeDetails indicating locations we
+ * know are bad and should not be used.
+ * @param repConfig The replication Config
+ * @param blockInfo The blockInfo representing the block.
+ * @param verifyChecksum Whether to verify checksums or not.
+ * @param xceiverFactory Factory to create the xceiver in the client
+ * @param refreshFunction Function to refresh the pipeline if needed
+ * @return BlockExtendedInputStream of the correct type.
+ */
+ public BlockExtendedInputStream create(boolean missingLocations,
+ List<DatanodeDetails> failedLocations, ReplicationConfig repConfig,
+ OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+ XceiverClientFactory xceiverFactory,
+ Function<BlockID, Pipeline> refreshFunction) {
+ if (missingLocations) {
+ // We create the reconstruction reader
+ ECBlockReconstructedStripeInputStream sis =
+ new ECBlockReconstructedStripeInputStream(
+ (ECReplicationConfig)repConfig, blockInfo, verifyChecksum,
+ xceiverFactory, refreshFunction, inputStreamFactory,
+ byteBufferPool, ecReconstructExecutorSupplier.get());
+ if (failedLocations != null) {
+ sis.addFailedDatanodes(failedLocations);
+ }
+ return new ECBlockReconstructedInputStream(
+ (ECReplicationConfig) repConfig, byteBufferPool, sis);
+ } else {
+ // Otherwise create the more efficient non-reconstruction reader
+ return new ECBlockInputStream((ECReplicationConfig)repConfig, blockInfo,
+ verifyChecksum, xceiverFactory, refreshFunction, inputStreamFactory);
+ }
+ }
+
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
new file mode 100644
index 0000000..ecde9c6
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStreamProxy.java
@@ -0,0 +1,239 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Top level class used to read data from EC Encoded blocks. This class decides,
+ * based on the block availability, whether to use a reconstruction or non
+ * reconstruction read and also handles errors from the non-reconstruction reads
+ * failing over to a reconstruction read when they happen.
+ */
+public class ECBlockInputStreamProxy extends BlockExtendedInputStream {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ECBlockInputStreamProxy.class);
+
+ private final ECReplicationConfig repConfig;
+ private final boolean verifyChecksum;
+ private final XceiverClientFactory xceiverClientFactory;
+ private final Function<BlockID, Pipeline> refreshFunction;
+ private final OmKeyLocationInfo blockInfo;
+ private final ECBlockInputStreamFactory ecBlockInputStreamFactory;
+
+ private BlockExtendedInputStream blockReader;
+ private boolean reconstructionReader = false;
+ private List<DatanodeDetails> failedLocations = new ArrayList<>();
+ private boolean closed = false;
+
+ /**
+ * Given the ECReplicationConfig and the block length, calculate how many
+ * data locations the block should have.
+ * @param repConfig The EC Replication Config
+ * @param blockLength The length of the data block in bytes
+ * @return The number of expected data locations
+ */
+ public static int expectedDataLocations(ECReplicationConfig repConfig,
+ long blockLength) {
+ return (int)Math.min(
+ Math.ceil((double)blockLength / repConfig.getEcChunkSize()),
+ repConfig.getData());
+ }
+
+ /**
+ * From the Pipeline and expected number of locations, determine the number
+ * of data locations available.
+ * @param pipeline The pipeline for the data block, givings its locations and
+ * the index of each location.
+ * @param expectedLocs The number of locations we expect for the block to have
+ * based on its block length and replication config. The
+ * max value should be the repConfig data number.
+ * @return The number of locations available
+ */
+ public static int availableDataLocations(Pipeline pipeline,
+ int expectedLocs) {
+ Set<Integer> locations = new HashSet<>();
+ for (DatanodeDetails dn : pipeline.getNodes()) {
+ int index = pipeline.getReplicaIndex(dn);
+ if (index > 0 && index <= expectedLocs) {
+ locations.add(index);
+ }
+ }
+ return locations.size();
+ }
+
+ public ECBlockInputStreamProxy(ECReplicationConfig repConfig,
+ OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+ XceiverClientFactory xceiverClientFactory, Function<BlockID,
+ Pipeline> refreshFunction, ECBlockInputStreamFactory streamFactory) {
+ this.repConfig = repConfig;
+ this.verifyChecksum = verifyChecksum;
+ this.blockInfo = blockInfo;
+ this.ecBlockInputStreamFactory = streamFactory;
+ this.xceiverClientFactory = xceiverClientFactory;
+ this.refreshFunction = refreshFunction;
+
+ setReaderType();
+ createBlockReader();
+ }
+
+ private synchronized void setReaderType() {
+ int expected = expectedDataLocations(repConfig, getLength());
+ int available = availableDataLocations(blockInfo.getPipeline(), expected);
+ reconstructionReader = available < expected;
+ }
+
+ private void createBlockReader() {
+ blockReader = ecBlockInputStreamFactory.create(reconstructionReader,
+ failedLocations, repConfig, blockInfo, verifyChecksum,
+ xceiverClientFactory, refreshFunction);
+ }
+
+ @Override
+ public synchronized BlockID getBlockID() {
+ return blockInfo.getBlockID();
+ }
+
+ @Override
+ public synchronized long getRemaining() {
+ return blockReader.getRemaining();
+ }
+
+ @Override
+ public synchronized long getLength() {
+ return blockInfo.getLength();
+ }
+
+ @Override
+ public synchronized int read(byte[] b, int off, int len)
+ throws IOException {
+ return read(ByteBuffer.wrap(b, off, len));
+ }
+
+ @Override
+ public synchronized int read(ByteBuffer buf) throws IOException {
+ ensureNotClosed();
+ if (blockReader.getRemaining() == 0) {
+ return EOF;
+ }
+ int totalRead = 0;
+ long lastPosition = 0;
+ try {
+ while (buf.hasRemaining() && getRemaining() > 0) {
+ buf.mark();
+ lastPosition = blockReader.getPos();
+ totalRead += blockReader.read(buf);
+ }
+ } catch (IOException e) {
+ if (reconstructionReader) {
+ // If we get an error from the reconstruction reader, there
+ // is nothing left to try. It will re-try until it has insufficient
+ // locations internally, so if an error comes here, just re-throw it.
+ throw e;
+ }
+ if (e instanceof BadDataLocationException) {
+ LOG.warn("Failing over to reconstruction read due to an error in " +
+ "ECBlockReader", e);
+ failoverToReconstructionRead(
+ ((BadDataLocationException) e).getFailedLocation(), lastPosition);
+ buf.reset();
+ totalRead += read(buf);
+ } else {
+ throw e;
+ }
+ }
+ return totalRead;
+ }
+
+ private synchronized void failoverToReconstructionRead(
+ DatanodeDetails badLocation, long lastPosition) throws IOException {
+ if (badLocation != null) {
+ failedLocations.add(badLocation);
+ }
+ blockReader.close();
+ reconstructionReader = true;
+ createBlockReader();
+ if (lastPosition != 0) {
+ blockReader.seek(lastPosition);
+ }
+ }
+
+ /**
+ * Should never be called in this class.
+ */
+ @Override
+ protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+ throws IOException {
+ throw new IOException("Not Implemented");
+ }
+
+ @Override
+ public synchronized void unbuffer() {
+ blockReader.unbuffer();
+ }
+
+ @Override
+ public synchronized long getPos() throws IOException {
+ return blockReader != null ? blockReader.getPos() : 0;
+ }
+
+ @Override
+ public synchronized void seek(long pos) throws IOException {
+ ensureNotClosed();
+ try {
+ blockReader.seek(pos);
+ } catch (IOException e) {
+ if (reconstructionReader) {
+ throw e;
+ }
+ failoverToReconstructionRead(null, pos);
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (blockReader != null) {
+ blockReader.close();
+ }
+ closed = true;
+ }
+
+ private void ensureNotClosed() throws IOException {
+ if (closed) {
+ throw new IOException("The stream is closed");
+ }
+ }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java
new file mode 100644
index 0000000..ebfa5cc
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java
@@ -0,0 +1,455 @@
+/*
+ * 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.client.io;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
+import org.apache.hadoop.hdds.scm.storage.BufferPool;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static com.google.common.base.Preconditions.checkState;
+import static org.apache.ratis.util.Preconditions.assertInstanceOf;
+
+/**
+ * ECBlockOutputStreamEntry manages write into EC keys' data block groups.
+ * A block group consists of data and parity blocks. For every block we have
+ * an internal ECBlockOutputStream instance with a single node pipeline, that
+ * is derived from the original EC pipeline.
+ */
+public class ECBlockOutputStreamEntry extends BlockOutputStreamEntry {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ECBlockOutputStreamEntry.class);
+
+ private final ECReplicationConfig replicationConfig;
+ private final long length;
+
+ private ECBlockOutputStream[] blockOutputStreams;
+ private int currentStreamIdx = 0;
+ private long successfulBlkGrpAckedLen;
+
+ @SuppressWarnings({"parameternumber", "squid:S00107"})
+ ECBlockOutputStreamEntry(BlockID blockID, String key,
+ XceiverClientFactory xceiverClientManager, Pipeline pipeline, long length,
+ BufferPool bufferPool, Token<OzoneBlockTokenIdentifier> token,
+ OzoneClientConfig config) {
+ super(blockID, key, xceiverClientManager, pipeline, length, bufferPool,
+ token, config);
+ assertInstanceOf(
+ pipeline.getReplicationConfig(), ECReplicationConfig.class);
+ this.replicationConfig =
+ (ECReplicationConfig) pipeline.getReplicationConfig();
+ this.length = replicationConfig.getData() * length;
+ }
+
+ @Override
+ void checkStream() throws IOException {
+ if (!isInitialized()) {
+ blockOutputStreams =
+ new ECBlockOutputStream[replicationConfig.getRequiredNodes()];
+ }
+ if (blockOutputStreams[currentStreamIdx] == null) {
+ createOutputStream();
+ }
+ }
+
+ @Override
+ void createOutputStream() throws IOException {
+ Pipeline ecPipeline = getPipeline();
+ List<DatanodeDetails> nodes = getPipeline().getNodes();
+ blockOutputStreams[currentStreamIdx] = new ECBlockOutputStream(
+ getBlockID(),
+ getXceiverClientManager(),
+ createSingleECBlockPipeline(
+ ecPipeline, nodes.get(currentStreamIdx), currentStreamIdx + 1),
+ getBufferPool(),
+ getConf(),
+ getToken());
+ }
+
+ @Override
+ public OutputStream getOutputStream() {
+ if (!isInitialized()) {
+ return null;
+ }
+ checkState(blockOutputStreams[currentStreamIdx] != null);
+ return blockOutputStreams[currentStreamIdx];
+ }
+
+ @Override
+ boolean isInitialized() {
+ return blockOutputStreams != null;
+ }
+
+ @Override
+ public long getLength() {
+ return length;
+ }
+
+ public int getCurrentStreamIdx() {
+ return currentStreamIdx;
+ }
+
+ public void useNextBlockStream() {
+ currentStreamIdx =
+ (currentStreamIdx + 1) % replicationConfig.getRequiredNodes();
+ }
+
+ public void markFailed(Exception e) {
+ if (blockOutputStreams[currentStreamIdx] != null) {
+ blockOutputStreams[currentStreamIdx].setIoException(e);
+ }
+ }
+
+ public void forceToFirstParityBlock() {
+ currentStreamIdx = replicationConfig.getData();
+ }
+
+ public void resetToFirstEntry() {
+ currentStreamIdx = 0;
+ }
+
+ @Override
+ void incCurrentPosition() {
+ if (isWritingParity()) {
+ return;
+ }
+ super.incCurrentPosition();
+ }
+
+ @Override
+ void incCurrentPosition(long len) {
+ if (isWritingParity()) {
+ return;
+ }
+ super.incCurrentPosition(len);
+ }
+
+ @Override
+ public void flush() throws IOException {
+ if (!isInitialized()) {
+ return;
+ }
+ for (int i = 0;
+ i <= currentStreamIdx && i < blockOutputStreams.length; i++) {
+ if (blockOutputStreams[i] != null) {
+ blockOutputStreams[i].flush();
+ }
+ }
+ }
+
+ @Override
+ boolean isClosed() {
+ if (!isInitialized()) {
+ return false;
+ }
+ return blockStreams().allMatch(BlockOutputStream::isClosed);
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (!isInitialized()) {
+ return;
+ }
+ for (ECBlockOutputStream stream : blockOutputStreams) {
+ if (stream != null) {
+ stream.close();
+ }
+ }
+ updateBlockID(underlyingBlockID());
+ }
+
+ @Override
+ long getTotalAckDataLength() {
+ if (!isInitialized()) {
+ return 0;
+ }
+ updateBlockID(underlyingBlockID());
+
+ return this.successfulBlkGrpAckedLen;
+ }
+
+ void updateBlockGroupToAckedPosition(long len) {
+ if (isWritingParity()) {
+ return;
+ }
+ this.successfulBlkGrpAckedLen = len;
+ }
+
+ /**
+ * Returns the amount of bytes that were attempted to be sent through towards
+ * the DataNodes, and the write call succeeded without an exception.
+ * In EC entries the parity writes does not count into this, as the written
+ * data length represents the attempts of the classes using the entry, and
+ * not the attempts of the entry itself.
+ *
+ * @return 0 if the stream is not initialized, the amount of data bytes that
+ * were attempted to be written to the entry.
+ */
+ //TODO: this might become problematic, and should be tested during the
+ // implementation of retries and error handling, as if there is a retry,
+ // then some data might have to be written twice.
+ // This current implementation is an assumption here.
+ // We might need to account the parity bytes written here, or elsewhere.
+ @Override
+ long getWrittenDataLength() {
+ if (!isInitialized()) {
+ return 0;
+ }
+ return dataStreams()
+ .mapToLong(BlockOutputStream::getWrittenDataLength)
+ .sum();
+ }
+
+ @Override
+ Collection<DatanodeDetails> getFailedServers() {
+ if (!isInitialized()) {
+ return Collections.emptyList();
+ }
+
+ return blockStreams()
+ .flatMap(outputStream -> outputStream.getFailedServers().stream())
+ .collect(Collectors.toList());
+ }
+
+ @VisibleForTesting
+ Pipeline createSingleECBlockPipeline(Pipeline ecPipeline,
+ DatanodeDetails node, int replicaIndex) {
+ Map<DatanodeDetails, Integer> indiciesForSinglePipeline = new HashMap<>();
+ indiciesForSinglePipeline.put(node, replicaIndex);
+ return Pipeline.newBuilder()
+ .setId(ecPipeline.getId())
+ .setReplicationConfig(ecPipeline.getReplicationConfig())
+ .setState(ecPipeline.getPipelineState())
+ .setNodes(ImmutableList.of(node))
+ .setReplicaIndexes(indiciesForSinglePipeline)
+ .build();
+ }
+
+ void executePutBlock(boolean isClose) {
+ if (!isInitialized()) {
+ return;
+ }
+ for (ECBlockOutputStream stream : blockOutputStreams) {
+ if (stream == null) {
+ continue;
+ }
+ try {
+ stream.executePutBlock(isClose, true);
+ } catch (Exception e) {
+ stream.setIoException(e);
+ }
+ }
+ }
+
+ private BlockID underlyingBlockID() {
+ if (blockOutputStreams[0] == null) {
+ return null;
+ }
+ // blockID is the same for EC blocks inside one block group managed by
+ // this entry, so updating based on the first stream, as when we write any
+ // data that is surely exists.
+ return blockOutputStreams[0].getBlockID();
+ }
+
+ public List<ECBlockOutputStream> streamsWithWriteFailure() {
+ return getFailedStreams(false);
+ }
+
+ public List<ECBlockOutputStream> streamsWithPutBlockFailure() {
+ return getFailedStreams(true);
+ }
+
+ /**
+ * In EC, we will do async write calls for writing data in the scope of a
+ * stripe. After every stripe write finishes, use this method to validate the
+ * responses of current stripe data writes. This method can also be used to
+ * validate the stripe put block responses.
+ * @param forPutBlock If true, it will validate the put block response
+ * futures. It will validate stripe data write response
+ * futures if false.
+ * @return
+ */
+ private List<ECBlockOutputStream> getFailedStreams(boolean forPutBlock) {
+ final Iterator<ECBlockOutputStream> iter = blockStreams().iterator();
+ List<ECBlockOutputStream> failedStreams = new ArrayList<>();
+ while (iter.hasNext()) {
+ final ECBlockOutputStream stream = iter.next();
+ CompletableFuture<ContainerProtos.ContainerCommandResponseProto>
+ responseFuture = null;
+ if (forPutBlock) {
+ responseFuture = stream.getCurrentPutBlkResponseFuture();
+ } else {
+ responseFuture = stream.getCurrentChunkResponseFuture();
+ }
+ if (isFailed(stream, responseFuture)) {
+ failedStreams.add(stream);
+ }
+ }
+ return failedStreams;
+ }
+
+ private boolean isFailed(
+ ECBlockOutputStream outputStream,
+ CompletableFuture<ContainerProtos.
+ ContainerCommandResponseProto> chunkWriteResponseFuture) {
+
+ if (chunkWriteResponseFuture == null) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Failed to reap response from datanode {}",
+ outputStream.getDatanodeDetails());
+ }
+ return true;
+ }
+
+ ContainerProtos.ContainerCommandResponseProto containerCommandResponseProto
+ = null;
+ try {
+ containerCommandResponseProto = chunkWriteResponseFuture.get();
+ } catch (InterruptedException e) {
+ outputStream.setIoException(e);
+ Thread.currentThread().interrupt();
+ } catch (ExecutionException e) {
+ outputStream.setIoException(e);
+ }
+
+ if (outputStream.getIoException() != null) {
+ return true;
+ }
+
+ if (containerCommandResponseProto == null) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Empty response from datanode {}",
+ outputStream.getDatanodeDetails());
+ }
+ return true;
+ }
+
+ return false;
+ }
+
+ private boolean isWritingParity() {
+ return currentStreamIdx >= replicationConfig.getData();
+ }
+
+ private Stream<ECBlockOutputStream> blockStreams() {
+ return Arrays.stream(blockOutputStreams).filter(Objects::nonNull);
+ }
+
+ private Stream<ECBlockOutputStream> dataStreams() {
+ return Arrays.stream(blockOutputStreams)
+ .limit(replicationConfig.getData())
+ .filter(Objects::nonNull);
+ }
+
+ /**
+ * Builder class for ChunkGroupOutputStreamEntry.
+ * */
+ public static class Builder {
+ private BlockID blockID;
+ private String key;
+ private XceiverClientFactory xceiverClientManager;
+ private Pipeline pipeline;
+ private long length;
+ private BufferPool bufferPool;
+ private Token<OzoneBlockTokenIdentifier> token;
+ private OzoneClientConfig config;
+
+ public ECBlockOutputStreamEntry.Builder setBlockID(BlockID bID) {
+ this.blockID = bID;
+ return this;
+ }
+
+ public ECBlockOutputStreamEntry.Builder setKey(String keys) {
+ this.key = keys;
+ return this;
+ }
+
+ public ECBlockOutputStreamEntry.Builder setXceiverClientManager(
+ XceiverClientFactory
+ xClientManager) {
+ this.xceiverClientManager = xClientManager;
+ return this;
+ }
+
+ public ECBlockOutputStreamEntry.Builder setPipeline(Pipeline ppln) {
+ this.pipeline = ppln;
+ return this;
+ }
+
+ public ECBlockOutputStreamEntry.Builder setLength(long len) {
+ this.length = len;
+ return this;
+ }
+
+ public ECBlockOutputStreamEntry.Builder setBufferPool(BufferPool pool) {
+ this.bufferPool = pool;
+ return this;
+ }
+
+ public ECBlockOutputStreamEntry.Builder setConfig(
+ OzoneClientConfig clientConfig) {
+ this.config = clientConfig;
+ return this;
+ }
+
+ public ECBlockOutputStreamEntry.Builder setToken(
+ Token<OzoneBlockTokenIdentifier> bToken) {
+ this.token = bToken;
+ return this;
+ }
+
+ public ECBlockOutputStreamEntry build() {
+ return new ECBlockOutputStreamEntry(blockID,
+ key,
+ xceiverClientManager,
+ pipeline,
+ length,
+ bufferPool,
+ token, config);
+ }
+ }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java
new file mode 100644
index 0000000..4651896
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java
@@ -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.
+ */
+package org.apache.hadoop.ozone.client.io;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.ozone.common.MonotonicClock;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+
+import java.time.ZoneOffset;
+
+/**
+ * {@link BlockOutputStreamEntryPool} is responsible to manage OM communication
+ * regarding writing a block to Ozone in a non-EC write case.
+ * The basic operations are fine for us but we need a specific
+ * {@link ECBlockOutputStreamEntry} implementation to handle writing EC block
+ * groups, this class implements the logic that handles the specific EC entries'
+ * instantiation and retrieval from the pool.
+ *
+ * @see ECKeyOutputStream
+ * @see BlockOutputStreamEntryPool
+ * @see ECBlockOutputStreamEntry
+ */
+public class ECBlockOutputStreamEntryPool extends BlockOutputStreamEntryPool {
+
+ @SuppressWarnings({"parameternumber", "squid:S00107"})
+ public ECBlockOutputStreamEntryPool(OzoneClientConfig config,
+ OzoneManagerProtocol omClient,
+ String requestId,
+ ReplicationConfig replicationConfig,
+ String uploadID,
+ int partNumber,
+ boolean isMultipart,
+ OmKeyInfo info,
+ boolean unsafeByteBufferConversion,
+ XceiverClientFactory xceiverClientFactory,
+ long openID) {
+ super(config, omClient, requestId, replicationConfig, uploadID, partNumber,
+ isMultipart, info, unsafeByteBufferConversion, xceiverClientFactory,
+ openID);
+ assert replicationConfig instanceof ECReplicationConfig;
+ }
+
+ @Override
+ ExcludeList createExcludeList() {
+ return new ExcludeList(getConfig().getExcludeNodesExpiryTime(),
+ new MonotonicClock(ZoneOffset.UTC));
+ }
+
+ @Override
+ BlockOutputStreamEntry createStreamEntry(OmKeyLocationInfo subKeyInfo) {
+ return
+ new ECBlockOutputStreamEntry.Builder()
+ .setBlockID(subKeyInfo.getBlockID())
+ .setKey(getKeyName())
+ .setXceiverClientManager(getXceiverClientFactory())
+ .setPipeline(subKeyInfo.getPipeline())
+ .setConfig(getConfig())
+ .setLength(subKeyInfo.getLength())
+ .setBufferPool(getBufferPool())
+ .setToken(subKeyInfo.getToken())
+ .build();
+ }
+
+ @Override
+ public ECBlockOutputStreamEntry getCurrentStreamEntry() {
+ return (ECBlockOutputStreamEntry) super.getCurrentStreamEntry();
+ }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
new file mode 100644
index 0000000..fe93b2e
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedInputStream.java
@@ -0,0 +1,243 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.io.ByteBufferPool;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Input stream which wraps a ECBlockReconstructedStripeInputStream to allow
+ * a EC Block to be read via the traditional InputStream read methods.
+ */
+public class ECBlockReconstructedInputStream extends BlockExtendedInputStream {
+
+ private ECReplicationConfig repConfig;
+ private ECBlockReconstructedStripeInputStream stripeReader;
+ private ByteBuffer[] bufs;
+ private final ByteBufferPool byteBufferPool;
+ private boolean closed = false;
+ private boolean unBuffered = false;
+
+ private long position = 0;
+
+ public ECBlockReconstructedInputStream(ECReplicationConfig repConfig,
+ ByteBufferPool byteBufferPool,
+ ECBlockReconstructedStripeInputStream stripeReader) {
+ this.repConfig = repConfig;
+ this.byteBufferPool = byteBufferPool;
+ this.stripeReader = stripeReader;
+ }
+
+ @Override
+ public synchronized BlockID getBlockID() {
+ return stripeReader.getBlockID();
+ }
+
+ @Override
+ public synchronized long getRemaining() {
+ return getLength() - position;
+ }
+
+ @Override
+ public synchronized long getLength() {
+ return stripeReader.getLength();
+ }
+
+ @Override
+ public synchronized int read(byte[] b, int off, int len)
+ throws IOException {
+ return read(ByteBuffer.wrap(b, off, len));
+ }
+
+ @Override
+ public synchronized int read(ByteBuffer buf) throws IOException {
+ ensureNotClosed();
+ if (!hasRemaining()) {
+ return EOF;
+ }
+ allocateBuffers();
+ if (unBuffered) {
+ seek(getPos());
+ unBuffered = false;
+ }
+ int totalRead = 0;
+ while (buf.hasRemaining() && getRemaining() > 0) {
+ ByteBuffer b = selectNextBuffer();
+ if (b == null) {
+ // This should not happen, so if it does abort.
+ throw new IOException(getRemaining() + " bytes remaining but unable " +
+ "to select a buffer with data");
+ }
+ long read = readBufferToDest(b, buf);
+ totalRead += read;
+ }
+ if (!hasRemaining()) {
+ // We have reached the end of the block. While the block is still open
+ // and could be seeked back, it is most likely the block will be closed.
+ // KeyInputStream does not call close on the block until all blocks in the
+ // key have been read, so releasing the resources here helps to avoid
+ // excessive memory usage.
+ freeBuffers();
+ }
+ return totalRead;
+ }
+
+ private void ensureNotClosed() throws IOException {
+ if (closed) {
+ throw new IOException("The input stream is closed");
+ }
+ }
+
+ private ByteBuffer selectNextBuffer() throws IOException {
+ for (ByteBuffer b : bufs) {
+ if (b.hasRemaining()) {
+ return b;
+ }
+ }
+ // If we get here, then no buffer has any remaining, so we need to
+ // fill them.
+ long read = readStripe();
+ if (read == EOF) {
+ return null;
+ }
+ return selectNextBuffer();
+ }
+
+ private long readBufferToDest(ByteBuffer src, ByteBuffer dest) {
+ int initialRemaining = dest.remaining();
+ while (dest.hasRemaining() && src.hasRemaining()) {
+ dest.put(src.get());
+ }
+ int read = initialRemaining - dest.remaining();
+ position += read;
+ return read;
+ }
+
+ @Override
+ protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+ throws IOException {
+ throw new IOException("Not Implemented");
+ }
+
+ @Override
+ public synchronized void unbuffer() {
+ stripeReader.unbuffer();
+ freeBuffers();
+ unBuffered = true;
+ }
+
+ @Override
+ public synchronized long getPos() throws IOException {
+ return position;
+ }
+
+ @Override
+ public synchronized void close() throws IOException {
+ stripeReader.close();
+ freeBuffers();
+ closed = true;
+ }
+
+ private void freeBuffers() {
+ if (bufs != null) {
+ for (int i = 0; i < bufs.length; i++) {
+ byteBufferPool.putBuffer(bufs[i]);
+ bufs[i] = null;
+ }
+ bufs = null;
+ }
+ }
+
+ @Override
+ public synchronized void seek(long pos) throws IOException {
+ ensureNotClosed();
+ if (pos < 0 || pos >= getLength()) {
+ if (pos == 0) {
+ // It is possible for length and pos to be zero in which case
+ // seek should return instead of throwing exception
+ return;
+ }
+ throw new EOFException(
+ "EOF encountered at pos: " + pos + " for block: " + getBlockID());
+ }
+ long stripeSize = (long)repConfig.getEcChunkSize() * repConfig.getData();
+ long stripeNum = pos / stripeSize;
+ int partial = (int)(pos % stripeSize);
+ // Seek the stripe reader to the beginning of the new current stripe
+ stripeReader.seek(stripeNum * stripeSize);
+ // Now reload the data buffers and adjust their position to the partial
+ // stripe offset.
+ readAndSeekStripe(partial);
+ position = pos;
+ }
+
+ private void readAndSeekStripe(int offset) throws IOException {
+ allocateBuffers();
+ readStripe();
+ if (offset == 0) {
+ return;
+ }
+ for (ByteBuffer b : bufs) {
+ int newPos = Math.min(b.remaining(), offset);
+ b.position(newPos);
+ offset -= newPos;
+ if (offset == 0) {
+ break;
+ }
+ }
+ }
+
+ private long readStripe() throws IOException {
+ clearBuffers();
+ return stripeReader.readStripe(bufs);
+ }
+
+ private void allocateBuffers() {
+ if (bufs != null) {
+ return;
+ }
+ bufs = new ByteBuffer[repConfig.getData()];
+ for (int i = 0; i < repConfig.getData(); i++) {
+ bufs[i] = byteBufferPool.getBuffer(false, repConfig.getEcChunkSize());
+ // Initially set the limit to 0 so there is no remaining space.
+ bufs[i].limit(0);
+ }
+ }
+
+ private void clearBuffers() {
+ for (ByteBuffer b : bufs) {
+ b.clear();
+ // As we are getting buffers from a bufferPool, we may get buffers with a
+ // capacity larger than what we asked for. After calling clear(), the
+ // buffer limit will become the capacity so we need to reset it back to
+ // the desired limit.
+ b.limit(repConfig.getEcChunkSize());
+ }
+ }
+
+ private boolean hasRemaining() {
+ return getRemaining() > 0;
+ }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
new file mode 100644
index 0000000..d5ec6db
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockReconstructedStripeInputStream.java
@@ -0,0 +1,631 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.io;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.ozone.erasurecode.rawcoder.util.CodecUtil;
+import org.apache.ratis.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.function.Function;
+
+/**
+ * Class to read EC encoded data from blocks a stripe at a time, when some of
+ * the data blocks are not available. The public API for this class is:
+ *
+ * readStripe(ByteBuffer[] bufs)
+ *
+ * The other inherited public APIs will throw a NotImplementedException. This is
+ * because this class is intended to only read full stripes into a reusable set
+ * of bytebuffers, and the tradition read APIs do not facilitate this.
+ *
+ * The caller should pass an array of ByteBuffers to readStripe() which:
+ *
+ * 1. Have EC DataNum buffers in the array.
+ * 2. Each buffer should have its position set to zero
+ * 3. Each buffer should have ecChunkSize remaining
+ *
+ * These buffers are either read into directly from the data blocks on the
+ * datanodes, or they will be reconstructed from parity data using the EC
+ * decoder.
+ *
+ * The EC Decoder expects to receive an array of elements matching EC Data + EC
+ * Parity elements long. Missing or not needed elements should be set to null
+ * in the array. The elements should be assigned to the array in EC index order.
+ *
+ * Assuming we have n missing data locations, where n <= parity locations, the
+ * ByteBuffers passed in from the client are either assigned to the decoder
+ * input array, or they are assigned to the decoder output array, where
+ * reconstructed data is written. The required number of parity buffers will be
+ * assigned and added to the decoder input so it has sufficient locations to
+ * reconstruct the data. After reconstruction the byte buffers received will
+ * have the data for a full stripe populated, either by reading directly from
+ * the block or by reconstructing the data.
+ *
+ * The buffers are returned "ready to read" with the position at zero and
+ * remaining() indicating how much data was read. If the remaining data is less
+ * than a full stripe, the client can simply read upto remaining from each
+ * buffer in turn. If there is a full stripe, each buffer should have ecChunk
+ * size remaining.
+ */
+public class ECBlockReconstructedStripeInputStream extends ECBlockInputStream {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ECBlockReconstructedStripeInputStream.class);
+
+ // List of buffers, data + parity long, needed by the EC decoder. Missing
+ // or not-need locations will be null.
+ private ByteBuffer[] decoderInputBuffers;
+ // Missing chunks are recovered into these buffers.
+ private ByteBuffer[] decoderOutputBuffers;
+ // Missing indexes to be recovered into the recovered buffers. Required by the
+ // EC decoder
+ private int[] missingIndexes;
+ // The blockLocation indexes to use to read data into the dataBuffers.
+ private List<Integer> dataIndexes = new ArrayList<>();
+ // Data Indexes we have tried to read from, and failed for some reason
+ private Set<Integer> failedDataIndexes = new HashSet<>();
+ private ByteBufferPool byteBufferPool;
+
+ private RawErasureDecoder decoder;
+
+ private boolean initialized = false;
+
+ private ExecutorService executor;
+
+ @SuppressWarnings("checkstyle:ParameterNumber")
+ public ECBlockReconstructedStripeInputStream(ECReplicationConfig repConfig,
+ OmKeyLocationInfo blockInfo, boolean verifyChecksum,
+ XceiverClientFactory xceiverClientFactory, Function<BlockID,
+ Pipeline> refreshFunction, BlockInputStreamFactory streamFactory,
+ ByteBufferPool byteBufferPool,
+ ExecutorService ecReconstructExecutor) {
+ super(repConfig, blockInfo, verifyChecksum, xceiverClientFactory,
+ refreshFunction, streamFactory);
+ this.byteBufferPool = byteBufferPool;
+ this.executor = ecReconstructExecutor;
+ }
+
+ /**
+ * Provide a list of datanodes that are known to be bad, and no attempt will
+ * be made to read from them. If too many failed nodes are passed, then the
+ * reader may not have sufficient locations available to reconstruct the data.
+ *
+ * Note this call must be made before any attempt it made to read data,
+ * as that is when the reader is initialized. Attempting to call this method
+ * after a read will result in a runtime exception.
+ *
+ * @param dns A list of DatanodeDetails that are known to be bad.
+ */
+ public synchronized void addFailedDatanodes(List<DatanodeDetails> dns) {
+ if (initialized) {
+ throw new RuntimeException("Cannot add failed datanodes after the " +
+ "reader has been initialized");
+ }
+ DatanodeDetails[] locations = getDataLocations();
+ for (DatanodeDetails dn : dns) {
+ for (int i = 0; i < locations.length; i++) {
+ if (locations[i] != null && locations[i].equals(dn)) {
+ failedDataIndexes.add(i);
+ break;
+ }
+ }
+ }
+ }
+
+ private void init() throws InsufficientLocationsException {
+ if (decoder == null) {
+ decoder = CodecUtil.createRawDecoderWithFallback(getRepConfig());
+ }
+ if (decoderInputBuffers == null) {
+ // The EC decoder needs an array data+parity long, with missing or not
+ // needed indexes set to null.
+ decoderInputBuffers = new ByteBuffer[getRepConfig().getRequiredNodes()];
+ }
+ if (!hasSufficientLocations()) {
+ throw new InsufficientLocationsException("There are insufficient " +
+ "datanodes to read the EC block");
+ }
+ dataIndexes.clear();
+ ECReplicationConfig repConfig = getRepConfig();
+ DatanodeDetails[] locations = getDataLocations();
+ setMissingIndexesAndDataLocations(locations);
+ List<Integer> parityIndexes =
+ selectParityIndexes(locations, missingIndexes.length);
+ // We read from the selected parity blocks, so add them to the data indexes.
+ dataIndexes.addAll(parityIndexes);
+ // The decoder inputs originally start as all nulls. Then we populate the
+ // pieces we have data for. The parity buffers are reused for the block
+ // so we can allocated them now. On re-init, we reuse any parity buffers
+ // already allocated.
+ for (int i = repConfig.getData(); i < repConfig.getRequiredNodes(); i++) {
+ if (parityIndexes.contains(i)) {
+ if (decoderInputBuffers[i] == null) {
+ decoderInputBuffers[i] = allocateBuffer(repConfig);
+ }
+ } else {
+ decoderInputBuffers[i] = null;
+ }
+ }
+ decoderOutputBuffers = new ByteBuffer[missingIndexes.length];
+ initialized = true;
+ }
+
+ /**
+ * Determine which indexes are missing, taking into account the length of the
+ * block. For a block shorter than a full EC stripe, it is expected that
+ * some of the data locations will not be present.
+ * Populates the missingIndex and dataIndexes instance variables.
+ * @param locations Available locations for the block group
+ */
+ private void setMissingIndexesAndDataLocations(DatanodeDetails[] locations) {
+ ECReplicationConfig repConfig = getRepConfig();
+ int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+ List<Integer> missingInd = new ArrayList<>();
+ for (int i = 0; i < repConfig.getData(); i++) {
+ if ((locations[i] == null || failedDataIndexes.contains(i))
+ && i < expectedDataBlocks) {
+ missingInd.add(i);
+ } else if (locations[i] != null && !failedDataIndexes.contains(i)) {
+ dataIndexes.add(i);
+ }
+ }
+ missingIndexes = missingInd.stream().mapToInt(Integer::valueOf).toArray();
+ }
+
+ private void assignBuffers(ByteBuffer[] bufs) {
+ ECReplicationConfig repConfig = getRepConfig();
+ Preconditions.assertTrue(bufs.length == repConfig.getData());
+ int recoveryIndex = 0;
+ // Here bufs come from the caller and will be filled with data read from
+ // the blocks or recovered. Therefore, if the index is missing, we assign
+ // the buffer to the decoder outputs, where data is recovered via EC
+ // decoding. Otherwise the buffer is set to the input. Note, it may be a
+ // buffer which needs padded.
+ for (int i = 0; i < repConfig.getData(); i++) {
+ if (isMissingIndex(i)) {
+ decoderOutputBuffers[recoveryIndex++] = bufs[i];
+ decoderInputBuffers[i] = null;
+ } else {
+ decoderInputBuffers[i] = bufs[i];
+ }
+ }
+ }
+
+ private boolean isMissingIndex(int ind) {
+ for (int i : missingIndexes) {
+ if (i == ind) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ * This method should be passed a list of byteBuffers which must contain EC
+ * Data Number entries. Each Bytebuffer should be at position 0 and have EC
+ * ChunkSize bytes remaining. After returning, the buffers will contain the
+ * data for the next stripe in the block. The buffers will be returned
+ * "ready to read" with their position set to zero and the limit set
+ * according to how much data they contain.
+ *
+ * @param bufs A list of byteBuffers which must contain EC Data Number
+ * entries. Each Bytebuffer should be at position 0 and have
+ * EC ChunkSize bytes remaining.
+ *
+ * @return The number of bytes read
+ * @throws IOException
+ */
+ public synchronized int readStripe(ByteBuffer[] bufs) throws IOException {
+ int toRead = (int)Math.min(getRemaining(), getStripeSize());
+ if (toRead == 0) {
+ return EOF;
+ }
+ if (!initialized) {
+ init();
+ }
+ validateBuffers(bufs);
+ while (true) {
+ try {
+ assignBuffers(bufs);
+ clearParityBuffers();
+ // Set the read limits on the buffers so we do not read any garbage data
+ // from the end of the block that is unexpected.
+ setBufferReadLimits(bufs, toRead);
+ loadDataBuffersFromStream();
+ break;
+ } catch (IOException e) {
+ // Re-init now the bad block has been excluded. If we have ran out of
+ // locations, init will throw an InsufficientLocations exception.
+ init();
+ // seek to the current position so it rewinds any blocks we read
+ // already.
+ seek(getPos());
+ // Reset the input positions back to zero
+ for (ByteBuffer b : bufs) {
+ b.position(0);
+ }
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ throw new IOException("Interrupted waiting for reads to complete", ie);
+ }
+ }
+ if (missingIndexes.length > 0) {
+ padBuffers(toRead);
+ flipInputs();
+ decodeStripe();
+ // Reset the buffer positions and limits to remove any padding added
+ // before EC Decode.
+ setBufferReadLimits(bufs, toRead);
+ } else {
+ // If we have no missing indexes, then the buffers will be at their
+ // limits after reading so we need to flip them to ensure they are ready
+ // to read by the caller.
+ flipInputs();
+ }
+ setPos(getPos() + toRead);
+ if (remaining() == 0) {
+ // If we reach the end of the block (ie remaining is zero) we free
+ // the underlying streams and buffers. This is because KeyInputStream,
+ // which reads from the EC streams does not close the blocks until it has
+ // read all blocks in the key.
+ freeAllResourcesWithoutClosing();
+ }
+ return toRead;
+ }
+
+ private void validateBuffers(ByteBuffer[] bufs) {
+ Preconditions.assertTrue(bufs.length == getRepConfig().getData());
+ int chunkSize = getRepConfig().getEcChunkSize();
+ for (ByteBuffer b : bufs) {
+ Preconditions.assertTrue(b.remaining() == chunkSize);
+ }
+ }
+
+ private void padBuffers(int toRead) {
+ int dataNum = getRepConfig().getData();
+ int parityNum = getRepConfig().getParity();
+ int chunkSize = getRepConfig().getEcChunkSize();
+ if (toRead >= getStripeSize()) {
+ // There is no padding to do - we are reading a full stripe.
+ return;
+ }
+ int fullChunks = toRead / chunkSize;
+ // The size of each chunk is governed by the size of the first chunk.
+ // The parity always matches the first chunk size.
+ int paritySize = Math.min(toRead, chunkSize);
+ // We never need to pad the first chunk - its length dictates the length
+ // of all others.
+ fullChunks = Math.max(1, fullChunks);
+ for (int i = fullChunks; i < dataNum; i++) {
+ ByteBuffer buf = decoderInputBuffers[i];
+ if (buf != null) {
+ buf.limit(paritySize);
+ zeroFill(buf);
+ }
+ }
+ // Ensure the available parity buffers are the expected length
+ for (int i = dataNum; i < dataNum + parityNum; i++) {
+ ByteBuffer b = decoderInputBuffers[i];
+ if (b != null) {
+ Preconditions.assertTrue(b.position() == paritySize);
+ }
+ }
+ // The output buffers need their limit set to the parity size
+ for (ByteBuffer b : decoderOutputBuffers) {
+ b.limit(paritySize);
+ }
+ }
+
+ private void setBufferReadLimits(ByteBuffer[] bufs, int toRead) {
+ int chunkSize = getRepConfig().getEcChunkSize();
+ int fullChunks = toRead / chunkSize;
+ if (fullChunks == getRepConfig().getData()) {
+ // We are reading a full stripe, no concerns over padding.
+ return;
+ }
+
+ if (fullChunks == 0) {
+ bufs[0].limit(toRead);
+ // All buffers except the first contain no data.
+ for (int i = 1; i < bufs.length; i++) {
+ bufs[i].position(0);
+ bufs[i].limit(0);
+ }
+ // If we have less than 1 chunk, then the parity buffers are the size
+ // of the first chunk.
+ for (int i = getRepConfig().getData();
+ i < getRepConfig().getRequiredNodes(); i++) {
+ ByteBuffer b = decoderInputBuffers[i];
+ if (b != null) {
+ b.limit(toRead);
+ }
+ }
+ } else {
+ int remainingLength = toRead % chunkSize;
+ // The first partial has the remaining length
+ bufs[fullChunks].limit(remainingLength);
+ // All others have a zero limit
+ for (int i = fullChunks + 1; i < bufs.length; i++) {
+ bufs[i].position(0);
+ bufs[i].limit(0);
+ }
+ }
+ }
+
+ private void zeroFill(ByteBuffer buf) {
+ // fill with zeros from pos to limit.
+ if (buf.hasArray()) {
+ byte[] a = buf.array();
+ Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+ buf.position(buf.limit());
+ } else {
+ while (buf.hasRemaining()) {
+ buf.put((byte)0);
+ }
+ }
+ }
+
+ /**
+ * Take the parity indexes which are already used, and the others which are
+ * available, and select random indexes to meet numRequired. The resulting
+ * list is sorted in ascending order of the indexes.
+ * @param locations The list of locations for all blocks in the block group/
+ * @param numRequired The number of parity chunks needed for reconstruction
+ * @return A list of indexes indicating which parity locations to read.
+ */
+ private List<Integer> selectParityIndexes(
+ DatanodeDetails[] locations, int numRequired) {
+ List<Integer> indexes = new ArrayList<>();
+ List<Integer> selected = new ArrayList<>();
+ ECReplicationConfig repConfig = getRepConfig();
+ for (int i = repConfig.getData(); i < repConfig.getRequiredNodes(); i++) {
+ if (locations[i] != null && !failedDataIndexes.contains(i)
+ && decoderInputBuffers[i] == null) {
+ indexes.add(i);
+ }
+ // If we are re-initializing, we want to make sure we are re-using any
+ // previously selected good parity indexes, as the block stream is already
+ // opened.
+ if (decoderInputBuffers[i] != null && !failedDataIndexes.contains(i)) {
+ selected.add(i);
+ }
+ }
+ Preconditions.assertTrue(indexes.size() + selected.size() >= numRequired);
+ Random rand = new Random();
+ while (selected.size() < numRequired) {
+ selected.add(indexes.remove(rand.nextInt(indexes.size())));
+ }
+ Collections.sort(selected);
+ return selected;
+ }
+
+ private ByteBuffer allocateBuffer(ECReplicationConfig repConfig) {
+ ByteBuffer buf = byteBufferPool.getBuffer(
+ false, repConfig.getEcChunkSize());
+ return buf;
+ }
+
+ private void flipInputs() {
+ for (ByteBuffer b : decoderInputBuffers) {
+ if (b != null) {
+ b.flip();
+ }
+ }
+ }
+
+ private void clearParityBuffers() {
+ for (int i = getRepConfig().getData();
+ i < getRepConfig().getRequiredNodes(); i++) {
+ if (decoderInputBuffers[i] != null) {
+ decoderInputBuffers[i].clear();
+ }
+ }
+ }
+
+ protected void loadDataBuffersFromStream()
+ throws IOException, InterruptedException {
+ Queue<ImmutablePair<Integer, Future<Void>>> pendingReads
+ = new ArrayDeque<>();
+ for (int i : dataIndexes) {
+ pendingReads.add(new ImmutablePair<>(i, executor.submit(() -> {
+ readIntoBuffer(i, decoderInputBuffers[i]);
+ return null;
+ })));
+ }
+ boolean exceptionOccurred = false;
+ while (!pendingReads.isEmpty()) {
+ int index = -1;
+ try {
+ ImmutablePair<Integer, Future<Void>> pair = pendingReads.poll();
+ index = pair.getKey();
+ // Should this future.get() have a timeout? At the end of the call chain
+ // we eventually call a grpc or ratis client to read the block data. Its
+ // the call to the DNs which could potentially block. There is a timeout
+ // on that call controlled by:
+ // OZONE_CLIENT_READ_TIMEOUT = "ozone.client.read.timeout";
+ // Which defaults to 30s. So if there is a DN communication problem, it
+ // should timeout in the client which should propagate up the stack as
+ // an IOException.
+ pair.getValue().get();
+ } catch (ExecutionException ee) {
+ LOG.warn("Failed to read from block {} EC index {}. Excluding the " +
+ "block", getBlockID(), index + 1, ee.getCause());
+ failedDataIndexes.add(index);
+ exceptionOccurred = true;
+ } catch (InterruptedException ie) {
+ // Catch each InterruptedException to ensure all the futures have been
+ // handled, and then throw the exception later
+ LOG.debug("Interrupted while waiting for reads to complete", ie);
+ Thread.currentThread().interrupt();
+ }
+ }
+ if (Thread.currentThread().isInterrupted()) {
+ throw new InterruptedException(
+ "Interrupted while waiting for reads to complete");
+ }
+ if (exceptionOccurred) {
+ throw new IOException("One or more errors occurred reading blocks");
+ }
+ }
+
+ private void readIntoBuffer(int ind, ByteBuffer buf) throws IOException {
+ BlockExtendedInputStream stream = getOrOpenStream(ind);
+ seekStreamIfNecessary(stream, 0);
+ while (buf.hasRemaining()) {
+ int read = stream.read(buf);
+ if (read == EOF) {
+ // We should not reach EOF, as the block should have enough data to
+ // fill the buffer. If the block does not, then it indicates the
+ // block is not as long as it should be, based on the block length
+ // stored in OM. Therefore if there is any remaining space in the
+ // buffer, we should throw an exception.
+ if (buf.hasRemaining()) {
+ throw new IOException("Expected to read " + buf.remaining() +
+ " bytes from block " + getBlockID() + " EC index " + (ind + 1) +
+ " but reached EOF");
+ }
+ break;
+ }
+ }
+ }
+
+ /**
+ * Take the populated input buffers and missing indexes and create the
+ * outputs. Note that the input buffers have to be "ready for read", ie they
+ * need to have been flipped after their data was loaded. The created outputs
+ * are "ready to read" by the underlying decoder API, so there is no need to
+ * flip them after the call. The decoder reads all the inputs leaving the
+ * buffer position at the end, so the inputs are flipped after the decode so
+ * we have a complete set of "outputs" for the EC Stripe which are ready to
+ * read.
+ * @throws IOException
+ */
+ private void decodeStripe() throws IOException {
+ decoder.decode(decoderInputBuffers, missingIndexes, decoderOutputBuffers);
+ flipInputs();
+ }
+
+ @Override
+ public synchronized boolean hasSufficientLocations() {
+ // The number of locations needed is a function of the EC Chunk size. If the
+ // block length is <= the chunk size, we should only have one data location.
+ // If it is greater than the chunk size but less than chunk_size * 2, then
+ // we must have two locations. If it is greater than chunk_size * data_num,
+ // then we must have all data_num locations.
+ // The remaining data locations (for small block lengths) can be assumed to
+ // be all zeros.
+ // Then we need a total of dataNum blocks available across the available
+ // data, parity and padding blocks.
+ ECReplicationConfig repConfig = getRepConfig();
+ int expectedDataBlocks = calculateExpectedDataBlocks(repConfig);
+ int availableLocations =
+ availableDataLocations(expectedDataBlocks) + availableParityLocations();
+ int paddedLocations = repConfig.getData() - expectedDataBlocks;
+ int failedLocations = failedDataIndexes.size();
+
+ if (availableLocations + paddedLocations - failedLocations
+ >= repConfig.getData()) {
+ return true;
+ } else {
+ LOG.error("There are insufficient locations. {} available; {} padded;" +
+ " {} failed; {} expected;", availableLocations, paddedLocations,
+ failedLocations, expectedDataBlocks);
+ return false;
+ }
+ }
+
+ @Override
+ protected int readWithStrategy(ByteReaderStrategy strategy) {
+ throw new NotImplementedException("readWithStrategy is not implemented. " +
+ "Use readStripe() instead");
+ }
+
+ @Override
+ public synchronized void close() {
+ super.close();
+ freeBuffers();
+ }
+
+ @Override
+ public synchronized void unbuffer() {
+ super.unbuffer();
+ freeBuffers();
+ }
+
+ private void freeBuffers() {
+ // Inside this class, we only allocate buffers to read parity into. Data
+ // is reconstructed or read into a set of buffers passed in from the calling
+ // class. Therefore we only need to ensure we free the parity buffers here.
+ if (decoderInputBuffers != null) {
+ for (int i = getRepConfig().getData();
+ i < getRepConfig().getRequiredNodes(); i++) {
+ ByteBuffer buf = decoderInputBuffers[i];
+ if (buf != null) {
+ byteBufferPool.putBuffer(buf);
+ decoderInputBuffers[i] = null;
+ }
+ }
+ }
+ initialized = false;
+ }
+
+ private void freeAllResourcesWithoutClosing() throws IOException {
+ LOG.debug("Freeing all resources while leaving the block open");
+ freeBuffers();
+ closeStreams();
+ }
+
+ @Override
+ public synchronized void seek(long pos) throws IOException {
+ if (pos % getStripeSize() != 0) {
+ // As this reader can only return full stripes, we only seek to the start
+ // stripe offsets
+ throw new IOException("Requested position " + pos
+ + " does not align with a stripe offset");
+ }
+ super.seek(pos);
+ }
+
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java
new file mode 100644
index 0000000..2c51962
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECKeyOutputStream.java
@@ -0,0 +1,706 @@
+/*
+ * 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.client.io;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.ozone.erasurecode.rawcoder.util.CodecUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * ECKeyOutputStream handles the EC writes by writing the data into underlying
+ * block output streams chunk by chunk.
+ */
+public final class ECKeyOutputStream extends KeyOutputStream {
+ private OzoneClientConfig config;
+ private ECChunkBuffers ecChunkBufferCache;
+ private int ecChunkSize;
+ private final int numDataBlks;
+ private final int numParityBlks;
+ private final ByteBufferPool bufferPool;
+ private final RawErasureEncoder encoder;
+
+ private enum StripeWriteStatus {
+ SUCCESS,
+ FAILED
+ }
+
+ public static final Logger LOG =
+ LoggerFactory.getLogger(KeyOutputStream.class);
+
+ private boolean closed;
+ // how much of data is actually written yet to underlying stream
+ private long offset;
+ // how much data has been ingested into the stream
+ private long writeOffset;
+ private final ECBlockOutputStreamEntryPool blockOutputStreamEntryPool;
+
+ @VisibleForTesting
+ public List<BlockOutputStreamEntry> getStreamEntries() {
+ return blockOutputStreamEntryPool.getStreamEntries();
+ }
+
+ @VisibleForTesting
+ public XceiverClientFactory getXceiverClientFactory() {
+ return blockOutputStreamEntryPool.getXceiverClientFactory();
+ }
+
+ @VisibleForTesting
+ public List<OmKeyLocationInfo> getLocationInfoList() {
+ return blockOutputStreamEntryPool.getLocationInfoList();
+ }
+
+ private ECKeyOutputStream(Builder builder) {
+ this.config = builder.getClientConfig();
+ this.bufferPool = builder.getByteBufferPool();
+ // For EC, cell/chunk size and buffer size can be same for now.
+ ecChunkSize = builder.getReplicationConfig().getEcChunkSize();
+ this.config.setStreamBufferMaxSize(ecChunkSize);
+ this.config.setStreamBufferFlushSize(ecChunkSize);
+ this.config.setStreamBufferSize(ecChunkSize);
+ this.numDataBlks = builder.getReplicationConfig().getData();
+ this.numParityBlks = builder.getReplicationConfig().getParity();
+ ecChunkBufferCache = new ECChunkBuffers(
+ ecChunkSize, numDataBlks, numParityBlks, bufferPool);
+ OmKeyInfo info = builder.getOpenHandler().getKeyInfo();
+ blockOutputStreamEntryPool =
+ new ECBlockOutputStreamEntryPool(config,
+ builder.getOmClient(), builder.getRequestID(),
+ builder.getReplicationConfig(),
+ builder.getMultipartUploadID(), builder.getMultipartNumber(),
+ builder.isMultipartKey(),
+ info, builder.isUnsafeByteBufferConversionEnabled(),
+ builder.getXceiverManager(), builder.getOpenHandler().getId());
+
+ this.writeOffset = 0;
+ this.encoder = CodecUtil.createRawEncoderWithFallback(
+ builder.getReplicationConfig());
+ }
+
+ /**
+ * When a key is opened, it is possible that there are some blocks already
+ * allocated to it for this open session. In this case, to make use of these
+ * blocks, we need to add these blocks to stream entries. But, a key's version
+ * also includes blocks from previous versions, we need to avoid adding these
+ * old blocks to stream entries, because these old blocks should not be picked
+ * for write. To do this, the following method checks that, only those
+ * blocks created in this particular open version are added to stream entries.
+ *
+ * @param version the set of blocks that are pre-allocated.
+ * @param openVersion the version corresponding to the pre-allocation.
+ * @throws IOException
+ */
+ public void addPreallocateBlocks(OmKeyLocationInfoGroup version,
+ long openVersion) throws IOException {
+ blockOutputStreamEntryPool.addPreallocateBlocks(version, openVersion);
+ }
+
+ /**
+ * Try to write the bytes sequence b[off:off+len) to underlying EC block
+ * streams.
+ *
+ * @param b byte data
+ * @param off starting offset
+ * @param len length to write
+ * @throws IOException
+ */
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ checkNotClosed();
+ if (b == null) {
+ throw new NullPointerException();
+ }
+ if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length)
+ || ((off + len) < 0)) {
+ throw new IndexOutOfBoundsException();
+ }
+ int rem = len;
+ while (rem > 0) {
+ try {
+ blockOutputStreamEntryPool.allocateBlockIfNeeded();
+ int currentStreamIdx = blockOutputStreamEntryPool
+ .getCurrentStreamEntry().getCurrentStreamIdx();
+ int bufferRem =
+ ecChunkBufferCache.dataBuffers[currentStreamIdx].remaining();
+ int writeLen = Math.min(rem, Math.min(bufferRem, ecChunkSize));
+ int pos = handleDataWrite(currentStreamIdx, b, off, writeLen);
+ checkAndWriteParityCells(pos);
+ rem -= writeLen;
+ off += writeLen;
+ } catch (Exception e) {
+ markStreamClosed();
+ throw new IOException(e.getMessage());
+ }
+ }
+ writeOffset += len;
+ }
+
+ private StripeWriteStatus rewriteStripeToNewBlockGroup(
+ long failedStripeDataSize, boolean close) throws IOException {
+ int[] failedDataStripeChunkLens = new int[numDataBlks];
+ int[] failedParityStripeChunkLens = new int[numParityBlks];
+ final ByteBuffer[] dataBuffers = ecChunkBufferCache.getDataBuffers();
+ for (int i = 0; i < numDataBlks; i++) {
+ failedDataStripeChunkLens[i] = dataBuffers[i].limit();
+ }
+ final ByteBuffer[] parityBuffers = ecChunkBufferCache.getParityBuffers();
+ for (int i = 0; i < numParityBlks; i++) {
+ failedParityStripeChunkLens[i] = parityBuffers[i].limit();
+ }
+
+ // Rollback the length/offset updated as part of this failed stripe write.
+ offset -= failedStripeDataSize;
+
+ final ECBlockOutputStreamEntry failedStreamEntry =
+ blockOutputStreamEntryPool.getCurrentStreamEntry();
+ failedStreamEntry.resetToFirstEntry();
+ failedStreamEntry.resetToAckedPosition();
+ // All pre-allocated blocks from the same pipeline
+ // should be dropped to eliminate worthless retries.
+ blockOutputStreamEntryPool.discardPreallocatedBlocks(-1,
+ failedStreamEntry.getPipeline().getId());
+ // Let's close the current entry.
+ failedStreamEntry.close();
+
+ // Let's rewrite the last stripe, so that it will be written to new block
+ // group.
+ // TODO: we can improve to write partial stripe failures. In that case,
+ // we just need to write only available buffers.
+ blockOutputStreamEntryPool.allocateBlockIfNeeded();
+ final ECBlockOutputStreamEntry currentStreamEntry =
+ blockOutputStreamEntryPool.getCurrentStreamEntry();
+ long totalLenToWrite = failedStripeDataSize;
+ for (int i = 0; i < numDataBlks; i++) {
+ int currentLen = (int) (totalLenToWrite < failedDataStripeChunkLens[i] ?
+ totalLenToWrite :
+ failedDataStripeChunkLens[i]);
+ if (currentLen > 0) {
+ handleOutputStreamWrite(i, currentLen, false);
+ }
+ currentStreamEntry.useNextBlockStream();
+ totalLenToWrite -= currentLen;
+ }
+ for (int i = 0; i < (numParityBlks); i++) {
+ handleOutputStreamWrite(i + numDataBlks, failedParityStripeChunkLens[i],
+ true);
+ currentStreamEntry.useNextBlockStream();
+ }
+
+ if (hasWriteFailure()) {
+ handleFailedStreams(false);
+ return StripeWriteStatus.FAILED;
+ }
+ currentStreamEntry.executePutBlock(close);
+
+ if (hasPutBlockFailure()) {
+ handleFailedStreams(true);
+ return StripeWriteStatus.FAILED;
+ }
+ ECBlockOutputStreamEntry newBlockGroupStreamEntry =
+ blockOutputStreamEntryPool.getCurrentStreamEntry();
+ newBlockGroupStreamEntry
+ .updateBlockGroupToAckedPosition(failedStripeDataSize);
+ ecChunkBufferCache.clear();
+
+ if (newBlockGroupStreamEntry.getRemaining() <= 0) {
+ // In most cases this should not happen except in the case stripe size and
+ // block size same.
+ newBlockGroupStreamEntry.close();
+ } else {
+ newBlockGroupStreamEntry.resetToFirstEntry();
+ }
+
+ return StripeWriteStatus.SUCCESS;
+ }
+
+ private void checkAndWriteParityCells(int lastDataBuffPos)
+ throws IOException {
+ // Check data blocks finished
+ // If index > numDataBlks
+ ECBlockOutputStreamEntry currentStreamEntry =
+ blockOutputStreamEntryPool.getCurrentStreamEntry();
+ int currentStreamIdx = currentStreamEntry.getCurrentStreamIdx();
+ if (currentStreamIdx == numDataBlks && lastDataBuffPos == ecChunkSize) {
+ //Lets encode and write
+ boolean shouldClose = currentStreamEntry.getRemaining() <= 0;
+ if (handleParityWrites(ecChunkSize, shouldClose)
+ == StripeWriteStatus.FAILED) {
+ handleStripeFailure(numDataBlks * ecChunkSize, shouldClose);
+ } else {
+ // At this stage stripe write is successful.
+ currentStreamEntry.updateBlockGroupToAckedPosition(
+ currentStreamEntry.getCurrentPosition());
+ }
+
+ }
+ }
+
+ private StripeWriteStatus handleParityWrites(int parityCellSize,
+ boolean isLastStripe) throws IOException {
+ writeParityCells(parityCellSize);
+ if (hasWriteFailure()) {
+ handleFailedStreams(false);
+ return StripeWriteStatus.FAILED;
+ }
+
+ // By this time, we should have finished full stripe. So, lets call
+ // executePutBlock for all.
+ // TODO: we should alter the put block calls to share CRC to each stream.
+ ECBlockOutputStreamEntry streamEntry =
+ blockOutputStreamEntryPool.getCurrentStreamEntry();
+ streamEntry
+ .executePutBlock(isLastStripe);
+
+ if (hasPutBlockFailure()) {
+ handleFailedStreams(true);
+ return StripeWriteStatus.FAILED;
+ }
+ ecChunkBufferCache.clear();
+
+ if (streamEntry.getRemaining() <= 0) {
+ streamEntry.close();
+ } else {
+ streamEntry.resetToFirstEntry();
+ }
+
+ return StripeWriteStatus.SUCCESS;
+ }
+
+ private boolean hasWriteFailure() {
+ return !blockOutputStreamEntryPool.getCurrentStreamEntry()
+ .streamsWithWriteFailure().isEmpty();
+ }
+
+ private boolean hasPutBlockFailure() {
+ return !blockOutputStreamEntryPool.getCurrentStreamEntry()
+ .streamsWithPutBlockFailure().isEmpty();
+ }
+
+ private void handleFailedStreams(boolean forPutBlock) {
+ ECBlockOutputStreamEntry currentStreamEntry =
+ blockOutputStreamEntryPool.getCurrentStreamEntry();
+ List<ECBlockOutputStream> failedStreams = forPutBlock
+ ? currentStreamEntry.streamsWithPutBlockFailure()
+ : currentStreamEntry.streamsWithWriteFailure();
+
+ // Since writes are async, let's check the failures once.
+ boolean containerToExcludeAll = true;
+ for (ECBlockOutputStream failedStream : failedStreams) {
+ Throwable cause = HddsClientUtils.checkForException(
+ failedStream.getIoException());
+ Preconditions.checkNotNull(cause);
+ if (!checkIfContainerToExclude(cause)) {
+ blockOutputStreamEntryPool.getExcludeList()
+ .addDatanode(failedStream.getDatanodeDetails());
+ containerToExcludeAll = false;
+ }
+ }
+
+ // NOTE: For now, this is mainly for ContainerNotOpenException
+ // due to container full, but may also for those cases that
+ // a DN do respond but with one with certain failures.
+ // In such cases we don't treat the replied DNs as failed.
+ if (containerToExcludeAll) {
+ blockOutputStreamEntryPool.getExcludeList()
+ .addPipeline(currentStreamEntry.getPipeline().getId());
+ }
+ }
+
+ @Override
+ protected boolean checkIfContainerToExclude(Throwable t) {
+ return super.checkIfContainerToExclude(t)
+ && t instanceof ContainerNotOpenException;
+ }
+
+ void writeParityCells(int parityCellSize) throws IOException {
+ final ByteBuffer[] buffers = ecChunkBufferCache.getDataBuffers();
+ final ByteBuffer[] parityBuffers = ecChunkBufferCache.getParityBuffers();
+
+ for (ByteBuffer b : parityBuffers) {
+ b.limit(parityCellSize);
+ }
+ for (ByteBuffer b : buffers) {
+ b.flip();
+ }
+ encoder.encode(buffers, parityBuffers);
+ blockOutputStreamEntryPool
+ .getCurrentStreamEntry().forceToFirstParityBlock();
+ for (int i =
+ numDataBlks; i < (this.numDataBlks + this.numParityBlks); i++) {
+ // Move the stream entry cursor to parity block index
+ handleParityWrite(i, parityCellSize);
+ }
+ }
+
+ private int handleDataWrite(int currIdx, byte[] b, int off, int len) {
+ int pos = ecChunkBufferCache.addToDataBuffer(currIdx, b, off, len);
+ if (pos == ecChunkSize) {
+ handleOutputStreamWrite(currIdx, pos, false);
+ blockOutputStreamEntryPool.getCurrentStreamEntry().useNextBlockStream();
+ }
+ return pos;
+ }
+
+ private void handleParityWrite(int currIdx, int len) {
+ handleOutputStreamWrite(currIdx, len, true);
+ blockOutputStreamEntryPool.getCurrentStreamEntry().useNextBlockStream();
+ }
+
+ private void handleOutputStreamWrite(int currIdx, int len, boolean isParity) {
+ ByteBuffer bytesToWrite = isParity ?
+ ecChunkBufferCache.getParityBuffers()[currIdx - numDataBlks] :
+ ecChunkBufferCache.getDataBuffers()[currIdx];
+ try {
+ // Since it's a full cell, let's write all content from buffer.
+ // At a time we write max cell size in EC. So, it should safe to cast
+ // the len to int to use the super class defined write API.
+ // The len cannot be bigger than cell buffer size.
+ assert len <= ecChunkSize : " The len: " + len + ". EC chunk size: "
+ + ecChunkSize;
+ assert len <= bytesToWrite
+ .limit() : " The len: " + len + ". Chunk buffer limit: "
+ + bytesToWrite.limit();
+ writeToOutputStream(blockOutputStreamEntryPool.getCurrentStreamEntry(),
+ bytesToWrite.array(), len, 0, isParity);
+ } catch (Exception e) {
+ markStreamAsFailed(e);
+ }
+ }
+
+ private long writeToOutputStream(ECBlockOutputStreamEntry current,
+ byte[] b, int writeLen, int off, boolean isParity)
+ throws IOException {
+ try {
+ if (!isParity) {
+ // In case if exception while writing, this length will be updated back
+ // as part of handleStripeFailure.
+ offset += writeLen;
+ }
+ current.write(b, off, writeLen);
+ } catch (IOException ioe) {
+ LOG.debug(
+ "Exception while writing the cell buffers. The writeLen: " + writeLen
+ + ". The block internal index is: "
+ + current
+ .getCurrentStreamIdx(), ioe);
+ handleException(current, ioe);
+ }
+ return writeLen;
+ }
+
+ private void handleException(BlockOutputStreamEntry streamEntry,
+ IOException exception) throws IOException {
+ Throwable t = HddsClientUtils.checkForException(exception);
+ Preconditions.checkNotNull(t);
+ boolean containerExclusionException = checkIfContainerToExclude(t);
+ if (containerExclusionException) {
+ blockOutputStreamEntryPool.getExcludeList()
+ .addPipeline(streamEntry.getPipeline().getId());
+ }
+ markStreamAsFailed(exception);
+ }
+
+ private void markStreamClosed() {
+ blockOutputStreamEntryPool.cleanup();
+ closed = true;
+ }
+
+ private void markStreamAsFailed(Exception e) {
+ blockOutputStreamEntryPool.getCurrentStreamEntry().markFailed(e);
+ }
+
+ @Override
+ public void flush() {
+ LOG.debug("ECKeyOutputStream does not support flush.");
+ }
+
+ private void closeCurrentStreamEntry()
+ throws IOException {
+ if (!blockOutputStreamEntryPool.isEmpty()) {
+ while (true) {
+ try {
+ BlockOutputStreamEntry entry =
+ blockOutputStreamEntryPool.getCurrentStreamEntry();
+ if (entry != null) {
+ try {
+ entry.close();
+ } catch (IOException ioe) {
+ handleException(entry, ioe);
+ continue;
+ }
+ }
+ return;
+ } catch (Exception e) {
+ markStreamClosed();
+ throw e;
+ }
+ }
+ }
+ }
+
+ /**
+ * Commit the key to OM, this will add the blocks as the new key blocks.
+ *
+ * @throws IOException
+ */
+ @Override
+ public void close() throws IOException {
+ if (closed) {
+ return;
+ }
+ closed = true;
+ try {
+ final long lastStripeSize = getCurrentDataStripeSize();
+ if (isPartialStripe(lastStripeSize)) {
+ ByteBuffer bytesToWrite =
+ ecChunkBufferCache.getDataBuffers()[blockOutputStreamEntryPool
+ .getCurrentStreamEntry().getCurrentStreamIdx()];
+
+ // Finish writing the current partial cached chunk
+ if (bytesToWrite.position() % ecChunkSize != 0) {
+ final ECBlockOutputStreamEntry current =
+ blockOutputStreamEntryPool.getCurrentStreamEntry();
+ try {
+ byte[] array = bytesToWrite.array();
+ writeToOutputStream(current, array,
+ bytesToWrite.position(), 0, false);
+ } catch (Exception e) {
+ markStreamAsFailed(e);
+ }
+ }
+
+ final int parityCellSize =
+ (int) (lastStripeSize < ecChunkSize ? lastStripeSize : ecChunkSize);
+ addPadding(parityCellSize);
+ if (handleParityWrites(parityCellSize, true)
+ == StripeWriteStatus.FAILED) {
+ handleStripeFailure(lastStripeSize, true);
+ } else {
+ blockOutputStreamEntryPool.getCurrentStreamEntry()
+ .updateBlockGroupToAckedPosition(
+ blockOutputStreamEntryPool.getCurrentStreamEntry()
+ .getCurrentPosition());
+ }
+
+ }
+
+ closeCurrentStreamEntry();
+ Preconditions.checkArgument(writeOffset == offset,
+ "Expected writeOffset= " + writeOffset
+ + " Expected offset=" + offset);
+ blockOutputStreamEntryPool.commitKey(offset);
+ } finally {
+ blockOutputStreamEntryPool.cleanup();
+ }
+ ecChunkBufferCache.release();
+ }
+
+ private void handleStripeFailure(long lastStripeSize, boolean isClose)
+ throws IOException {
+ StripeWriteStatus stripeWriteStatus;
+ for (int i = 0; i < this.config.getMaxECStripeWriteRetries(); i++) {
+ stripeWriteStatus = rewriteStripeToNewBlockGroup(lastStripeSize, isClose);
+ if (stripeWriteStatus == StripeWriteStatus.SUCCESS) {
+ return;
+ }
+ }
+ throw new IOException("Completed max allowed retries " + this.config
+ .getMaxECStripeWriteRetries() + " on stripe failures.");
+
+ }
+
+ private void addPadding(int parityCellSize) {
+ ByteBuffer[] buffers = ecChunkBufferCache.getDataBuffers();
+
+ for (int i = 1; i < numDataBlks; i++) {
+ final int position = buffers[i].position();
+ assert position <= parityCellSize : "If an internal block is smaller"
+ + " than parity block, then its last cell should be small than last"
+ + " parity cell";
+ padBufferToLimit(buffers[i], parityCellSize);
+ }
+ }
+
+ public static void padBufferToLimit(ByteBuffer buf, int limit) {
+ int pos = buf.position();
+ if (pos >= limit) {
+ return;
+ }
+ Arrays.fill(buf.array(), pos, limit, (byte)0);
+ buf.position(limit);
+ }
+
+ private boolean isPartialStripe(long stripeSize) {
+ return stripeSize > 0 && stripeSize < (numDataBlks * ecChunkSize);
+ }
+
+ private long getCurrentDataStripeSize() {
+ final ByteBuffer[] dataBuffers = ecChunkBufferCache.getDataBuffers();
+ long lastStripeSize = 0;
+ for (int i = 0; i < numDataBlks; i++) {
+ lastStripeSize += dataBuffers[i].position();
+ }
+ return lastStripeSize;
+ }
+
+ public OmMultipartCommitUploadPartInfo getCommitUploadPartInfo() {
+ return blockOutputStreamEntryPool.getCommitUploadPartInfo();
+ }
+
+ @VisibleForTesting
+ public ExcludeList getExcludeList() {
+ return blockOutputStreamEntryPool.getExcludeList();
+ }
+
+ /**
+ * Builder class of ECKeyOutputStream.
+ */
+ public static class Builder extends KeyOutputStream.Builder {
+ private ECReplicationConfig replicationConfig;
+ private ByteBufferPool byteBufferPool;
+
+ @Override
+ public ECReplicationConfig getReplicationConfig() {
+ return replicationConfig;
+ }
+
+ public ECKeyOutputStream.Builder setReplicationConfig(
+ ECReplicationConfig replConfig) {
+ this.replicationConfig = replConfig;
+ return this;
+ }
+
+ public ByteBufferPool getByteBufferPool() {
+ return byteBufferPool;
+ }
+
+ public ECKeyOutputStream.Builder setByteBufferPool(
+ ByteBufferPool bufferPool) {
+ this.byteBufferPool = bufferPool;
+ return this;
+ }
+
+ @Override
+ public ECKeyOutputStream build() {
+ return new ECKeyOutputStream(this);
+ }
+ }
+
+ /**
+ * Verify that the output stream is open. Non blocking; this gives
+ * the last state of the volatile {@link #closed} field.
+ *
+ * @throws IOException if the connection is closed.
+ */
+ private void checkNotClosed() throws IOException {
+ if (closed) {
+ throw new IOException(
+ ": " + FSExceptionMessages.STREAM_IS_CLOSED + " Key: "
+ + blockOutputStreamEntryPool.getKeyName());
+ }
+ }
+
+ private static class ECChunkBuffers {
+ private final ByteBuffer[] dataBuffers;
+ private final ByteBuffer[] parityBuffers;
+ private int cellSize;
+ private ByteBufferPool byteBufferPool;
+
+ ECChunkBuffers(int cellSize, int numData, int numParity,
+ ByteBufferPool byteBufferPool) {
+ this.cellSize = cellSize;
+ dataBuffers = new ByteBuffer[numData];
+ parityBuffers = new ByteBuffer[numParity];
+ this.byteBufferPool = byteBufferPool;
+ allocateBuffers(dataBuffers, this.cellSize);
+ allocateBuffers(parityBuffers, this.cellSize);
+ }
+
+ private ByteBuffer[] getDataBuffers() {
+ return dataBuffers;
+ }
+
+ private ByteBuffer[] getParityBuffers() {
+ return parityBuffers;
+ }
+
+ private int addToDataBuffer(int i, byte[] b, int off, int len) {
+ final ByteBuffer buf = dataBuffers[i];
+ final int pos = buf.position() + len;
+ Preconditions.checkState(pos <= cellSize,
+ "Position(" + pos + ") is greater than the cellSize("
+ + cellSize + ").");
+ buf.put(b, off, len);
+ return pos;
+ }
+
+ private void clear() {
+ clearBuffers(dataBuffers);
+ clearBuffers(parityBuffers);
+ }
+
+ private void release() {
+ releaseBuffers(dataBuffers);
+ releaseBuffers(parityBuffers);
+ }
+
+ private void allocateBuffers(ByteBuffer[] buffers, int bufferSize) {
+ for (int i = 0; i < buffers.length; i++) {
+ buffers[i] = byteBufferPool.getBuffer(false, cellSize);
+ buffers[i].limit(bufferSize);
+ }
+ }
+
+ private void clearBuffers(ByteBuffer[] buffers) {
+ for (int i = 0; i < buffers.length; i++) {
+ buffers[i].clear();
+ buffers[i].limit(cellSize);
+ }
+ }
+
+ private void releaseBuffers(ByteBuffer[] buffers) {
+ for (int i = 0; i < buffers.length; i++) {
+ if (buffers[i] != null) {
+ byteBufferPool.putBuffer(buffers[i]);
+ buffers[i] = null;
+ }
+ }
+ }
+ }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
new file mode 100644
index 0000000..956ed90
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/InsufficientLocationsException.java
@@ -0,0 +1,43 @@
+/**
+ * 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.client.io;
+
+import java.io.IOException;
+
+/**
+ * Exception thrown by EC Input Streams if there are not enough locations to
+ * read the EC data successfully.
+ */
+public class InsufficientLocationsException extends IOException {
+
+ public InsufficientLocationsException() {
+ super();
+ }
+
+ public InsufficientLocationsException(String message) {
+ super(message);
+ }
+
+ public InsufficientLocationsException(String message, Throwable ex) {
+ super(message, ex);
+ }
+
+ public InsufficientLocationsException(Throwable ex) {
+ super(ex);
+ }
+}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
index f440cf9..20cb2b5 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
@@ -19,7 +19,6 @@
import java.io.EOFException;
import java.io.IOException;
-import java.io.InputStream;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
@@ -30,17 +29,17 @@
import java.util.stream.Collectors;
import com.google.common.base.Preconditions;
-import org.apache.hadoop.fs.ByteBufferReadable;
-import org.apache.hadoop.fs.CanUnbuffer;
import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
import org.apache.hadoop.hdds.scm.storage.ByteArrayReader;
import org.apache.hadoop.hdds.scm.storage.ByteBufferReader;
import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.scm.storage.ExtendedInputStream;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
@@ -52,8 +51,7 @@
/**
* Maintaining a list of BlockInputStream. Read based on offset.
*/
-public class KeyInputStream extends InputStream
- implements Seekable, CanUnbuffer, ByteBufferReadable {
+public class KeyInputStream extends ExtendedInputStream {
private static final Logger LOG =
LoggerFactory.getLogger(KeyInputStream.class);
@@ -65,7 +63,7 @@
private boolean closed = false;
// List of BlockInputStreams, one for each block in the key
- private final List<BlockInputStream> blockStreams;
+ private final List<BlockExtendedInputStream> blockStreams;
// blockOffsets[i] stores the index of the first data byte in
// blockStream w.r.t the key data.
@@ -93,20 +91,23 @@
*/
public static LengthInputStream getFromOmKeyInfo(OmKeyInfo keyInfo,
XceiverClientFactory xceiverClientFactory,
- boolean verifyChecksum, Function<OmKeyInfo, OmKeyInfo> retryFunction) {
+ boolean verifyChecksum, Function<OmKeyInfo, OmKeyInfo> retryFunction,
+ BlockInputStreamFactory blockStreamFactory) {
List<OmKeyLocationInfo> keyLocationInfos = keyInfo
.getLatestVersionLocations().getBlocksLatestVersionOnly();
KeyInputStream keyInputStream = new KeyInputStream();
keyInputStream.initialize(keyInfo, keyLocationInfos,
- xceiverClientFactory, verifyChecksum, retryFunction);
+ xceiverClientFactory, verifyChecksum, retryFunction,
+ blockStreamFactory);
return new LengthInputStream(keyInputStream, keyInputStream.length);
}
public static List<LengthInputStream> getStreamsFromKeyInfo(OmKeyInfo keyInfo,
XceiverClientFactory xceiverClientFactory, boolean verifyChecksum,
- Function<OmKeyInfo, OmKeyInfo> retryFunction) {
+ Function<OmKeyInfo, OmKeyInfo> retryFunction,
+ BlockInputStreamFactory blockStreamFactory) {
List<OmKeyLocationInfo> keyLocationInfos = keyInfo
.getLatestVersionLocations().getBlocksLatestVersionOnly();
@@ -137,7 +138,8 @@
partsToBlocksMap.entrySet()) {
KeyInputStream keyInputStream = new KeyInputStream();
keyInputStream.initialize(keyInfo, entry.getValue(),
- xceiverClientFactory, verifyChecksum, retryFunction);
+ xceiverClientFactory, verifyChecksum, retryFunction,
+ blockStreamFactory);
lengthInputStreams.add(new LengthInputStream(keyInputStream,
partsLengthMap.get(entry.getKey())));
}
@@ -148,7 +150,8 @@
private synchronized void initialize(OmKeyInfo keyInfo,
List<OmKeyLocationInfo> blockInfos,
XceiverClientFactory xceiverClientFactory,
- boolean verifyChecksum, Function<OmKeyInfo, OmKeyInfo> retryFunction) {
+ boolean verifyChecksum, Function<OmKeyInfo, OmKeyInfo> retryFunction,
+ BlockInputStreamFactory blockStreamFactory) {
this.key = keyInfo.getKeyName();
this.blockOffsets = new long[blockInfos.size()];
long keyLength = 0;
@@ -161,7 +164,8 @@
// We also pass in functional reference which is used to refresh the
// pipeline info for a given OM Key location info.
- addStream(omKeyLocationInfo, xceiverClientFactory,
+ addStream(keyInfo.getReplicationConfig(), omKeyLocationInfo,
+ xceiverClientFactory,
verifyChecksum, keyLocationInfo -> {
OmKeyInfo newKeyInfo = retryFunction.apply(keyInfo);
BlockID blockID = keyLocationInfo.getBlockID();
@@ -176,7 +180,7 @@
} else {
return null;
}
- });
+ }, blockStreamFactory);
this.blockOffsets[i] = keyLength;
keyLength += omKeyLocationInfo.getLength();
@@ -190,12 +194,13 @@
* BlockInputStream is initialized when a read operation is performed on
* the block for the first time.
*/
- private synchronized void addStream(OmKeyLocationInfo blockInfo,
- XceiverClientFactory xceiverClientFactory,
- boolean verifyChecksum,
- Function<OmKeyLocationInfo, Pipeline> refreshPipelineFunction) {
- blockStreams.add(new BlockInputStream(blockInfo.getBlockID(),
- blockInfo.getLength(), blockInfo.getPipeline(), blockInfo.getToken(),
+ private synchronized void addStream(ReplicationConfig repConfig,
+ OmKeyLocationInfo blockInfo,
+ XceiverClientFactory xceiverClientFactory, boolean verifyChecksum,
+ Function<OmKeyLocationInfo, Pipeline> refreshPipelineFunction,
+ BlockInputStreamFactory blockStreamFactory) {
+ blockStreams.add(blockStreamFactory.create(repConfig, blockInfo,
+ blockInfo.getPipeline(), blockInfo.getToken(),
verifyChecksum, xceiverClientFactory,
blockID -> refreshPipelineFunction.apply(blockInfo)));
}
@@ -205,6 +210,11 @@
blockStreams.add(blockInputStream);
}
+ @VisibleForTesting
+ public void addStream(BlockExtendedInputStream blockInputStream) {
+ blockStreams.add(blockInputStream);
+ }
+
/**
* {@inheritDoc}
*/
@@ -240,8 +250,9 @@
return readWithStrategy(strategy);
}
- synchronized int readWithStrategy(ByteReaderStrategy strategy) throws
- IOException {
+ @Override
+ protected synchronized int readWithStrategy(ByteReaderStrategy strategy)
+ throws IOException {
Preconditions.checkArgument(strategy != null);
checkOpen();
@@ -257,8 +268,8 @@
}
// Get the current blockStream and read data from it
- BlockInputStream current = blockStreams.get(blockIndex);
- int numBytesToRead = Math.min(buffLen, (int)current.getRemaining());
+ BlockExtendedInputStream current = blockStreams.get(blockIndex);
+ int numBytesToRead = (int)Math.min(buffLen, current.getRemaining());
int numBytesRead = strategy.readFromBlock(current, numBytesToRead);
if (numBytesRead != numBytesToRead) {
// This implies that there is either data loss or corruption in the
@@ -326,7 +337,7 @@
}
// Reset the previous blockStream's position
- blockStreams.get(blockIndexOfPrevPosition).resetPosition();
+ blockStreams.get(blockIndexOfPrevPosition).seek(0);
// Reset all the blockStreams above the blockIndex. We do this to reset
// any previous reads which might have updated the blockPosition and
@@ -360,7 +371,7 @@
@Override
public synchronized void close() throws IOException {
closed = true;
- for (BlockInputStream blockStream : blockStreams) {
+ for (ExtendedInputStream blockStream : blockStreams) {
blockStream.close();
}
}
@@ -400,13 +411,13 @@
@Override
public synchronized void unbuffer() {
- for (BlockInputStream is : blockStreams) {
+ for (ExtendedInputStream is : blockStreams) {
is.unbuffer();
}
}
@VisibleForTesting
- public List<BlockInputStream> getBlockStreams() {
+ public List<BlockExtendedInputStream> getBlockStreams() {
return blockStreams;
}
}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java
index d5f6f5d..ee69ca1 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java
@@ -150,7 +150,6 @@
unsafeByteBufferConversion,
xceiverClientManager,
handler.getId());
-
this.retryPolicyMap = HddsClientUtils.getRetryPolicyByException(
config.getMaxRetryCount(), config.getRetryInterval());
this.retryCount = 0;
@@ -432,7 +431,7 @@
// Every container specific exception from datatnode will be seen as
// StorageContainerException
- private boolean checkIfContainerToExclude(Throwable t) {
+ protected boolean checkIfContainerToExclude(Throwable t) {
return t instanceof StorageContainerException;
}
@@ -554,56 +553,99 @@
private OzoneClientConfig clientConfig;
private ReplicationConfig replicationConfig;
+ public String getMultipartUploadID() {
+ return multipartUploadID;
+ }
+
public Builder setMultipartUploadID(String uploadID) {
this.multipartUploadID = uploadID;
return this;
}
+ public int getMultipartNumber() {
+ return multipartNumber;
+ }
+
public Builder setMultipartNumber(int partNumber) {
this.multipartNumber = partNumber;
return this;
}
+ public OpenKeySession getOpenHandler() {
+ return openHandler;
+ }
+
public Builder setHandler(OpenKeySession handler) {
this.openHandler = handler;
return this;
}
+ public XceiverClientFactory getXceiverManager() {
+ return xceiverManager;
+ }
+
public Builder setXceiverClientManager(XceiverClientFactory manager) {
this.xceiverManager = manager;
return this;
}
+ public OzoneManagerProtocol getOmClient() {
+ return omClient;
+ }
+
public Builder setOmClient(OzoneManagerProtocol client) {
this.omClient = client;
return this;
}
+ public int getChunkSize() {
+ return chunkSize;
+ }
+
public Builder setChunkSize(int size) {
this.chunkSize = size;
return this;
}
+ public String getRequestID() {
+ return requestID;
+ }
+
public Builder setRequestID(String id) {
this.requestID = id;
return this;
}
+ public boolean isMultipartKey() {
+ return isMultipartKey;
+ }
+
public Builder setIsMultipartKey(boolean isMultipart) {
this.isMultipartKey = isMultipart;
return this;
}
+ public OzoneClientConfig getClientConfig() {
+ return clientConfig;
+ }
+
public Builder setConfig(OzoneClientConfig config) {
this.clientConfig = config;
return this;
}
+ public boolean isUnsafeByteBufferConversionEnabled() {
+ return unsafeByteBufferConversion;
+ }
+
public Builder enableUnsafeByteBufferConversion(boolean enabled) {
this.unsafeByteBufferConversion = enabled;
return this;
}
+ public ReplicationConfig getReplicationConfig() {
+ return replicationConfig;
+ }
public Builder setReplicationConfig(ReplicationConfig replConfig) {
this.replicationConfig = replConfig;
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
index e2a8679..725eb3a 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
@@ -743,6 +743,17 @@
long quotaInNamespace, long quotaInBytes) throws IOException;
/**
+ * Set Bucket replication configuration.
+ *
+ * @param volumeName Name of the Volume.
+ * @param bucketName Name of the Bucket.
+ * @param replicationConfig The replication config to set on bucket.
+ * @throws IOException
+ */
+ void setReplicationConfig(String volumeName, String bucketName,
+ ReplicationConfig replicationConfig) throws IOException;
+
+ /**
* Returns OzoneKey that contains the application generated/visible
* metadata for an Ozone Object.
*
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 3a460de..213868c 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
@@ -37,14 +37,20 @@
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.stream.Collectors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.crypto.CryptoInputStream;
import org.apache.hadoop.crypto.CryptoOutputStream;
import org.apache.hadoop.crypto.key.KeyProvider;
import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
@@ -62,6 +68,8 @@
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.tracing.TracingUtil;
import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -78,6 +86,9 @@
import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
import org.apache.hadoop.ozone.client.io.KeyInputStream;
import org.apache.hadoop.ozone.client.io.KeyOutputStream;
import org.apache.hadoop.ozone.client.io.LengthInputStream;
@@ -156,6 +167,11 @@
private static final Logger LOG =
LoggerFactory.getLogger(RpcClient.class);
+ // For the minimal recommended EC policy rs-3-2-1024k,
+ // we should have at least 1 core thread for each necessary chunk
+ // for reconstruction.
+ private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3;
+
private final ConfigurationSource conf;
private final OzoneManagerClientProtocol ozoneManagerClient;
private final XceiverClientFactory xceiverClientManager;
@@ -172,6 +188,10 @@
private final OzoneClientConfig clientConfig;
private final Cache<URI, KeyProvider> keyProviderCache;
private final boolean getLatestVersionLocation;
+ private final ByteBufferPool byteBufferPool;
+ private final BlockInputStreamFactory blockInputStreamFactory;
+ private final OzoneManagerVersion omVersion;
+ private volatile ExecutorService ecReconstructExecutor;
/**
* Creates RpcClient instance with the given configuration.
@@ -202,8 +222,9 @@
OzoneManagerClientProtocol.class, conf);
dtService = omTransport.getDelegationTokenService();
List<X509Certificate> x509Certificates = null;
+ ServiceInfoEx serviceInfoEx = ozoneManagerClient.getServiceInfo();
+ omVersion = getOmVersion(serviceInfoEx);
if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
- ServiceInfoEx serviceInfoEx = ozoneManagerClient.getServiceInfo();
// If the client is authenticating using S3 style auth, all future
// requests serviced by this client will need S3 Auth set.
boolean isS3 = conf.getBoolean(S3Auth.S3_AUTH_CHECK, false);
@@ -288,12 +309,30 @@
}
}
}).build();
+ this.byteBufferPool = new ElasticByteBufferPool();
+ this.blockInputStreamFactory = BlockInputStreamFactoryImpl
+ .getInstance(byteBufferPool, this::getECReconstructExecutor);
}
public XceiverClientFactory getXceiverClientManager() {
return xceiverClientManager;
}
+ private OzoneManagerVersion getOmVersion(ServiceInfoEx info) {
+ OzoneManagerVersion version = OzoneManagerVersion.CURRENT;
+ for (ServiceInfo si : info.getServiceInfoList()) {
+ if (si.getNodeType() == HddsProtos.NodeType.OM) {
+ OzoneManagerVersion current =
+ OzoneManagerVersion.fromProtoValue(si.getProtobuf().getOMVersion());
+ if (version.compareTo(current) > 0) {
+ version = current;
+ }
+ }
+ }
+ LOG.trace("Ozone Manager version is {}", version.name());
+ return version;
+ }
+
static boolean validateOmVersion(OzoneManagerVersion minimumVersion,
List<ServiceInfo> serviceInfoList) {
if (minimumVersion == OzoneManagerVersion.FUTURE_VERSION) {
@@ -532,6 +571,16 @@
Preconditions.checkNotNull(bucketArgs);
verifyCountsQuota(bucketArgs.getQuotaInNamespace());
verifySpaceQuota(bucketArgs.getQuotaInBytes());
+ if (omVersion
+ .compareTo(OzoneManagerVersion.ERASURE_CODED_STORAGE_SUPPORT) < 0) {
+ if (bucketArgs.getDefaultReplicationConfig() != null &&
+ bucketArgs.getDefaultReplicationConfig().getType()
+ == ReplicationType.EC) {
+ throw new IOException("Can not set the default replication of the"
+ + " bucket to Erasure Coded replication, as OzoneManager does"
+ + " not support Erasure Coded replication.");
+ }
+ }
final String owner;
// If S3 auth exists, set owner name to the short user name derived from the
@@ -579,6 +628,12 @@
builder.setBucketEncryptionKey(bek);
}
+ DefaultReplicationConfig defaultReplicationConfig =
+ bucketArgs.getDefaultReplicationConfig();
+ if (defaultReplicationConfig != null) {
+ builder.setDefaultReplicationConfig(defaultReplicationConfig);
+ }
+
LOG.info("Creating Bucket: {}/{}, with {} as owner and Versioning {} and " +
"Storage Type set to {} and Encryption set to {} ",
volumeName, bucketName, owner, isVersionEnabled,
@@ -769,6 +824,30 @@
}
@Override
+ public void setReplicationConfig(
+ String volumeName, String bucketName, ReplicationConfig replicationConfig)
+ throws IOException {
+ verifyVolumeName(volumeName);
+ verifyBucketName(bucketName);
+ Preconditions.checkNotNull(replicationConfig);
+ if (omVersion
+ .compareTo(OzoneManagerVersion.ERASURE_CODED_STORAGE_SUPPORT) < 0) {
+ if (replicationConfig.getReplicationType()
+ == HddsProtos.ReplicationType.EC) {
+ throw new IOException("Can not set the default replication of the"
+ + " bucket to Erasure Coded replication, as OzoneManager does"
+ + " not support Erasure Coded replication.");
+ }
+ }
+ OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
+ builder.setVolumeName(volumeName)
+ .setBucketName(bucketName)
+ .setDefaultReplicationConfig(
+ new DefaultReplicationConfig(replicationConfig));
+ ozoneManagerClient.setBucketProperty(builder.build());
+ }
+
+ @Override
public void deleteBucket(
String volumeName, String bucketName) throws IOException {
verifyVolumeName(volumeName);
@@ -808,7 +887,8 @@
bucketInfo.getQuotaInBytes(),
bucketInfo.getQuotaInNamespace(),
bucketInfo.getBucketLayout(),
- bucketInfo.getOwner()
+ bucketInfo.getOwner(),
+ bucketInfo.getDefaultReplicationConfig()
);
}
@@ -838,7 +918,8 @@
bucket.getQuotaInBytes(),
bucket.getQuotaInNamespace(),
bucket.getBucketLayout(),
- bucket.getOwner()))
+ bucket.getOwner(),
+ bucket.getDefaultReplicationConfig()))
.collect(Collectors.toList());
}
@@ -863,7 +944,17 @@
if (checkKeyNameEnabled) {
HddsClientUtils.verifyKeyName(keyName);
}
- HddsClientUtils.checkNotNull(keyName, replicationConfig);
+ HddsClientUtils.checkNotNull(keyName);
+ if (omVersion
+ .compareTo(OzoneManagerVersion.ERASURE_CODED_STORAGE_SUPPORT) < 0) {
+ if (replicationConfig != null &&
+ replicationConfig.getReplicationType()
+ == HddsProtos.ReplicationType.EC) {
+ throw new IOException("Can not set the replication of the key to"
+ + " Erasure Coded replication, as OzoneManager does not support"
+ + " Erasure Coded replication.");
+ }
+ }
String requestId = UUID.randomUUID().toString();
OmKeyArgs.Builder builder = new OmKeyArgs.Builder()
@@ -891,7 +982,7 @@
}
OpenKeySession openKey = ozoneManagerClient.openKey(builder.build());
- return createOutputStream(openKey, requestId, replicationConfig);
+ return createOutputStream(openKey, requestId);
}
private KeyProvider.KeyVersion getDEK(FileEncryptionInfo feInfo)
@@ -1133,6 +1224,10 @@
@Override
public void close() throws IOException {
+ if (ecReconstructExecutor != null) {
+ ecReconstructExecutor.shutdownNow();
+ ecReconstructExecutor = null;
+ }
IOUtils.cleanupWithLogger(LOG, ozoneManagerClient, xceiverClientManager);
keyProviderCache.invalidateAll();
keyProviderCache.cleanUp();
@@ -1155,7 +1250,16 @@
throws IOException {
verifyVolumeName(volumeName);
verifyBucketName(bucketName);
- HddsClientUtils.checkNotNull(keyName, replicationConfig);
+ HddsClientUtils.checkNotNull(keyName);
+ if (omVersion
+ .compareTo(OzoneManagerVersion.ERASURE_CODED_STORAGE_SUPPORT) < 0) {
+ if (replicationConfig.getReplicationType()
+ == HddsProtos.ReplicationType.EC) {
+ throw new IOException("Can not set the replication of the file to"
+ + " Erasure Coded replication, as OzoneManager does not support"
+ + " Erasure Coded replication.");
+ }
+ }
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
.setVolumeName(volumeName)
.setBucketName(bucketName)
@@ -1199,19 +1303,11 @@
.build();
OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs);
- KeyOutputStream keyOutputStream =
- new KeyOutputStream.Builder()
- .setHandler(openKey)
- .setXceiverClientManager(xceiverClientManager)
- .setOmClient(ozoneManagerClient)
- .setRequestID(requestId)
- .setReplicationConfig(openKey.getKeyInfo().getReplicationConfig())
- .setMultipartNumber(partNumber)
- .setMultipartUploadID(uploadID)
- .setIsMultipartKey(true)
- .enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
- .setConfig(clientConfig)
- .build();
+ KeyOutputStream keyOutputStream = createKeyOutputStream(openKey, requestId)
+ .setMultipartNumber(partNumber)
+ .setMultipartUploadID(uploadID)
+ .setIsMultipartKey(true)
+ .build();
keyOutputStream.addPreallocateBlocks(
openKey.getKeyInfo().getLatestVersionLocations(),
openKey.getOpenVersion());
@@ -1403,6 +1499,15 @@
public OzoneOutputStream createFile(String volumeName, String bucketName,
String keyName, long size, ReplicationConfig replicationConfig,
boolean overWrite, boolean recursive) throws IOException {
+ if (omVersion
+ .compareTo(OzoneManagerVersion.ERASURE_CODED_STORAGE_SUPPORT) < 0) {
+ if (replicationConfig.getReplicationType()
+ == HddsProtos.ReplicationType.EC) {
+ throw new IOException("Can not set the replication of the file to"
+ + " Erasure Coded replication, as OzoneManager does not support"
+ + " Erasure Coded replication.");
+ }
+ }
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
.setVolumeName(volumeName)
.setBucketName(bucketName)
@@ -1414,8 +1519,7 @@
.build();
OpenKeySession keySession =
ozoneManagerClient.createFile(keyArgs, overWrite, recursive);
- return createOutputStream(keySession, UUID.randomUUID().toString(),
- replicationConfig);
+ return createOutputStream(keySession, UUID.randomUUID().toString());
}
@Override
@@ -1494,7 +1598,8 @@
if (feInfo == null) {
LengthInputStream lengthInputStream = KeyInputStream
.getFromOmKeyInfo(keyInfo, xceiverClientManager,
- clientConfig.isChecksumVerify(), retryFunction);
+ clientConfig.isChecksumVerify(), retryFunction,
+ blockInputStreamFactory);
try {
Map< String, String > keyInfoMetadata = keyInfo.getMetadata();
if (Boolean.valueOf(keyInfoMetadata.get(OzoneConsts.GDPR_FLAG))) {
@@ -1514,7 +1619,8 @@
// Regular Key with FileEncryptionInfo
LengthInputStream lengthInputStream = KeyInputStream
.getFromOmKeyInfo(keyInfo, xceiverClientManager,
- clientConfig.isChecksumVerify(), retryFunction);
+ clientConfig.isChecksumVerify(), retryFunction,
+ blockInputStreamFactory);
final KeyProvider.KeyVersion decrypted = getDEK(feInfo);
final CryptoInputStream cryptoIn =
new CryptoInputStream(lengthInputStream.getWrappedStream(),
@@ -1525,7 +1631,8 @@
// Multipart Key with FileEncryptionInfo
List<LengthInputStream> lengthInputStreams = KeyInputStream
.getStreamsFromKeyInfo(keyInfo, xceiverClientManager,
- clientConfig.isChecksumVerify(), retryFunction);
+ clientConfig.isChecksumVerify(), retryFunction,
+ blockInputStreamFactory);
final KeyProvider.KeyVersion decrypted = getDEK(feInfo);
List<OzoneCryptoInputStream> cryptoInputStreams = new ArrayList<>();
@@ -1542,18 +1649,11 @@
}
private OzoneOutputStream createOutputStream(OpenKeySession openKey,
- String requestId, ReplicationConfig replicationConfig)
- throws IOException {
- KeyOutputStream keyOutputStream =
- new KeyOutputStream.Builder()
- .setHandler(openKey)
- .setXceiverClientManager(xceiverClientManager)
- .setOmClient(ozoneManagerClient)
- .setRequestID(requestId)
- .setReplicationConfig(replicationConfig)
- .enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
- .setConfig(clientConfig)
- .build();
+ String requestId) throws IOException {
+
+ KeyOutputStream keyOutputStream = createKeyOutputStream(openKey, requestId)
+ .build();
+
keyOutputStream
.addPreallocateBlocks(openKey.getKeyInfo().getLatestVersionLocations(),
openKey.getOpenVersion());
@@ -1588,6 +1688,30 @@
}
}
+ private KeyOutputStream.Builder createKeyOutputStream(OpenKeySession openKey,
+ String requestId) {
+ KeyOutputStream.Builder builder;
+
+ ReplicationConfig replicationConfig =
+ openKey.getKeyInfo().getReplicationConfig();
+ if (replicationConfig.getReplicationType() ==
+ HddsProtos.ReplicationType.EC) {
+ builder = new ECKeyOutputStream.Builder()
+ .setReplicationConfig((ECReplicationConfig) replicationConfig)
+ .setByteBufferPool(byteBufferPool);
+ } else {
+ builder = new KeyOutputStream.Builder()
+ .setReplicationConfig(replicationConfig);
+ }
+
+ return builder.setHandler(openKey)
+ .setXceiverClientManager(xceiverClientManager)
+ .setOmClient(ozoneManagerClient)
+ .setRequestID(requestId)
+ .enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
+ .setConfig(clientConfig);
+ }
+
@Override
public KeyProvider getKeyProvider() throws IOException {
URI kmsUri = getKeyProviderUri();
@@ -1680,4 +1804,27 @@
.setOwnerName(owner);
return ozoneManagerClient.setBucketOwner(builder.build());
}
+
+ public ExecutorService getECReconstructExecutor() {
+ // local ref to a volatile to ensure access
+ // to a completed initialized object
+ ExecutorService executor = ecReconstructExecutor;
+ if (executor == null) {
+ synchronized (this) {
+ executor = ecReconstructExecutor;
+ if (executor == null) {
+ ecReconstructExecutor = new ThreadPoolExecutor(
+ EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
+ clientConfig.getEcReconstructStripeReadPoolLimit(),
+ 60, TimeUnit.SECONDS, new SynchronousQueue<>(),
+ new ThreadFactoryBuilder()
+ .setNameFormat("ec-reconstruct-reader-TID-%d")
+ .build(),
+ new ThreadPoolExecutor.CallerRunsPolicy());
+ executor = ecReconstructExecutor;
+ }
+ }
+ }
+ return executor;
+ }
}
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockBlockAllocator.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockBlockAllocator.java
index 747b39b..0d5e1a2 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockBlockAllocator.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockBlockAllocator.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.ozone.client;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation;
@@ -27,7 +28,7 @@
*/
public interface MockBlockAllocator {
- Iterable<? extends KeyLocation> allocateBlock(
- KeyArgs createKeyRequest);
+ Iterable<? extends KeyLocation> allocateBlock(KeyArgs createKeyRequest,
+ ExcludeList excludeList);
}
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java
index cb9875b..1f29425 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockDatanodeStorage.java
@@ -18,11 +18,13 @@
package org.apache.hadoop.ozone.client;
import org.apache.commons.collections.map.HashedMap;
+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.DatanodeBlockID;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
@@ -32,11 +34,18 @@
public class MockDatanodeStorage {
private final Map<DatanodeBlockID, BlockData> blocks = new HashedMap();
+ private final Map<BlockID, String> fullBlockData = new HashMap<>();
private final Map<String, ChunkInfo> chunks = new HashMap<>();
private final Map<String, ByteString> data = new HashMap<>();
+ private boolean failed = false;
+
+ public void setStorageFailed() {
+ this.failed = true;
+ }
+
public void putBlock(DatanodeBlockID blockID, BlockData blockData) {
blocks.put(blockID, blockData);
}
@@ -47,9 +56,17 @@
public void writeChunk(
DatanodeBlockID blockID,
- ChunkInfo chunkInfo, ByteString bytes) {
- data.put(createKey(blockID, chunkInfo), bytes);
+ ChunkInfo chunkInfo, ByteString bytes) throws IOException {
+ if (failed) {
+ throw new IOException("This storage was marked as failed.");
+ }
+ data.put(createKey(blockID, chunkInfo),
+ ByteString.copyFrom(bytes.toByteArray()));
chunks.put(createKey(blockID, chunkInfo), chunkInfo);
+ fullBlockData
+ .put(new BlockID(blockID.getContainerID(), blockID.getLocalID()),
+ fullBlockData.getOrDefault(blockID, "")
+ .concat(bytes.toStringUtf8()));
}
public ChunkInfo readChunkInfo(
@@ -70,4 +87,12 @@
+ chunkInfo.getChunkName() + "_" + chunkInfo.getOffset();
}
+ public Map<String, ByteString> getAllBlockData() {
+ return this.data;
+ }
+
+ public String getFullBlockData(BlockID blockID) {
+ return this.fullBlockData.get(blockID);
+ }
+
}
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java
index 17a7f6b..90ed563 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockOmTransport.java
@@ -17,6 +17,14 @@
*/
package org.apache.hadoop.ozone.client;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ozone.om.protocolPB.OmTransport;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
@@ -50,6 +58,7 @@
import java.io.IOException;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.Map;
import java.util.function.Function;
@@ -75,7 +84,7 @@
}
public MockOmTransport() {
- this.blockAllocator = new SinglePipelineBlockAllocator();
+ this(new SinglePipelineBlockAllocator(new OzoneConfiguration()));
}
@Override
@@ -129,10 +138,18 @@
private OzoneManagerProtocolProtos.AllocateBlockResponse allocateBlock(
OzoneManagerProtocolProtos.AllocateBlockRequest allocateBlockRequest) {
- return OzoneManagerProtocolProtos.AllocateBlockResponse.newBuilder()
- .setKeyLocation(
- blockAllocator.allocateBlock(allocateBlockRequest.getKeyArgs())
- .iterator().next()).build();
+ Iterator<? extends OzoneManagerProtocolProtos.KeyLocation> iterator =
+ blockAllocator.allocateBlock(allocateBlockRequest.getKeyArgs(),
+ ExcludeList.getFromProtoBuf(allocateBlockRequest.getExcludeList()))
+ .iterator();
+ OzoneManagerProtocolProtos.AllocateBlockResponse.Builder builder =
+ OzoneManagerProtocolProtos.AllocateBlockResponse.newBuilder()
+ .setKeyLocation(iterator.next());
+ while (iterator.hasNext()) {
+ builder.mergeKeyLocation(iterator.next());
+ }
+ return builder.build();
+
}
private DeleteVolumeResponse deleteVolume(
@@ -153,11 +170,28 @@
private CommitKeyResponse commitKey(CommitKeyRequest commitKeyRequest) {
final KeyArgs keyArgs = commitKeyRequest.getKeyArgs();
- final KeyInfo remove =
+ final KeyInfo openKey =
openKeys.get(keyArgs.getVolumeName()).get(keyArgs.getBucketName())
.remove(keyArgs.getKeyName());
+ final KeyInfo.Builder committedKeyInfoWithLocations =
+ KeyInfo.newBuilder().setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .setCreationTime(openKey.getCreationTime())
+ .setModificationTime(openKey.getModificationTime())
+ .setDataSize(keyArgs.getDataSize()).setLatestVersion(0L)
+ .addKeyLocationList(KeyLocationList.newBuilder()
+ .addAllKeyLocations(keyArgs.getKeyLocationsList()));
+ // Just inherit replication config details from open Key
+ if (openKey.hasEcReplicationConfig()) {
+ committedKeyInfoWithLocations
+ .setEcReplicationConfig(openKey.getEcReplicationConfig());
+ } else if (openKey.hasFactor()) {
+ committedKeyInfoWithLocations.setFactor(openKey.getFactor());
+ }
+ committedKeyInfoWithLocations.setType(openKey.getType());
keys.get(keyArgs.getVolumeName()).get(keyArgs.getBucketName())
- .put(keyArgs.getKeyName(), remove);
+ .put(keyArgs.getKeyName(), committedKeyInfoWithLocations.build());
return CommitKeyResponse.newBuilder()
.build();
}
@@ -165,33 +199,95 @@
private CreateKeyResponse createKey(CreateKeyRequest createKeyRequest) {
final KeyArgs keyArgs = createKeyRequest.getKeyArgs();
final long now = System.currentTimeMillis();
- final KeyInfo keyInfo = KeyInfo.newBuilder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .setCreationTime(now)
- .setModificationTime(now)
- .setType(keyArgs.getType())
- .setFactor(keyArgs.getFactor())
- .setDataSize(keyArgs.getDataSize())
- .setLatestVersion(0L)
- .addKeyLocationList(KeyLocationList.newBuilder()
- .addAllKeyLocations(
- blockAllocator.allocateBlock(createKeyRequest.getKeyArgs()))
- .build())
- .build();
+ final BucketInfo bucketInfo =
+ buckets.get(keyArgs.getVolumeName()).get(keyArgs.getBucketName());
+
+ final KeyInfo.Builder keyInfoBuilder =
+ KeyInfo.newBuilder().setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName()).setCreationTime(now)
+ .setModificationTime(now).setDataSize(keyArgs.getDataSize())
+ .setLatestVersion(0L).addKeyLocationList(
+ KeyLocationList.newBuilder().addAllKeyLocations(
+ blockAllocator.allocateBlock(createKeyRequest.getKeyArgs(),
+ new ExcludeList()))
+ .build());
+
+ if (keyArgs.getType() == HddsProtos.ReplicationType.NONE) {
+ // 1. Client did not pass replication config.
+ // Now lets try bucket defaults
+ if (bucketInfo.getDefaultReplicationConfig() != null) {
+ // Since Bucket defaults are available, let's inherit
+ final HddsProtos.ReplicationType type =
+ bucketInfo.getDefaultReplicationConfig().getType();
+ keyInfoBuilder
+ .setType(bucketInfo.getDefaultReplicationConfig().getType());
+ switch (type) {
+ case EC:
+ keyInfoBuilder.setEcReplicationConfig(
+ bucketInfo.getDefaultReplicationConfig()
+ .getEcReplicationConfig());
+ break;
+ case RATIS:
+ case STAND_ALONE:
+ keyInfoBuilder
+ .setFactor(bucketInfo.getDefaultReplicationConfig().getFactor());
+ break;
+ default:
+ throw new UnsupportedOperationException(
+ "Unknown replication type: " + type);
+ }
+ } else {
+ keyInfoBuilder.setType(HddsProtos.ReplicationType.RATIS);
+ keyInfoBuilder.setFactor(HddsProtos.ReplicationFactor.THREE);
+ }
+ } else {
+ // 1. Client passed the replication config.
+ // Let's use it.
+ final HddsProtos.ReplicationType type = keyArgs.getType();
+ keyInfoBuilder.setType(type);
+ switch (type) {
+ case EC:
+ keyInfoBuilder.setEcReplicationConfig(keyArgs.getEcReplicationConfig());
+ break;
+ case RATIS:
+ case STAND_ALONE:
+ keyInfoBuilder.setFactor(keyArgs.getFactor());
+ break;
+ default:
+ throw new UnsupportedOperationException(
+ "Unknown replication type: " + type);
+ }
+ }
+
+ final KeyInfo keyInfo = keyInfoBuilder.build();
openKeys.get(keyInfo.getVolumeName()).get(keyInfo.getBucketName())
.put(keyInfo.getKeyName(), keyInfo);
- return CreateKeyResponse.newBuilder()
- .setOpenVersion(0L)
- .setKeyInfo(keyInfo)
+ return CreateKeyResponse.newBuilder().setOpenVersion(0L).setKeyInfo(keyInfo)
.build();
}
private InfoBucketResponse infoBucket(InfoBucketRequest infoBucketRequest) {
+ BucketInfo bucketInfo = buckets.get(infoBucketRequest.getVolumeName())
+ .get(infoBucketRequest.getBucketName());
+ if (!bucketInfo.hasDefaultReplicationConfig()) {
+ final ReplicationConfig replicationConfig = ReplicationConfig
+ .getDefault(new OzoneConfiguration());
+
+ bucketInfo = bucketInfo.toBuilder().setDefaultReplicationConfig(
+ new DefaultReplicationConfig(
+ ReplicationType.fromProto(replicationConfig.getReplicationType()),
+ replicationConfig
+ .getReplicationType() != HddsProtos.ReplicationType.EC ?
+ ReplicationFactor
+ .valueOf(replicationConfig.getRequiredNodes()) :
+ null, replicationConfig
+ .getReplicationType() == HddsProtos.ReplicationType.EC ?
+ (ECReplicationConfig) replicationConfig :
+ null).toProto()).build();
+ }
return InfoBucketResponse.newBuilder()
- .setBucketInfo(buckets.get(infoBucketRequest.getVolumeName())
- .get(infoBucketRequest.getBucketName()))
+ .setBucketInfo(bucketInfo)
.build();
}
@@ -258,6 +354,10 @@
return CreateBucketResponse.newBuilder().build();
}
+ public Map<String, Map<String, Map<String, KeyInfo>>> getKeys() {
+ return this.keys;
+ }
+
@Override
public Text getDelegationTokenService() {
return null;
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java
index a5fa2bb..fbcc153 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java
@@ -23,7 +23,10 @@
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
import java.util.Map;
/**
@@ -34,6 +37,31 @@
private final Map<DatanodeDetails, MockDatanodeStorage> storage =
new HashMap<>();
+ private List<DatanodeDetails> pendingToFailNodes = new ArrayList<>();
+
+ public void setFailedStorages(List<DatanodeDetails> failedStorages) {
+ List<DatanodeDetails> remainingFailNodes = new ArrayList<>();
+ for (int i = 0; i < failedStorages.size(); i++) {
+ DatanodeDetails failedDN = failedStorages.get(i);
+ boolean isCurrentNodeMarked = false;
+ final Iterator<Map.Entry<DatanodeDetails, MockDatanodeStorage>> iterator =
+ storage.entrySet().iterator();
+ while (iterator.hasNext()) {
+ final Map.Entry<DatanodeDetails, MockDatanodeStorage> next =
+ iterator.next();
+ if (next.getKey().equals(failedDN)) {
+ final MockDatanodeStorage value = next.getValue();
+ value.setStorageFailed();
+ isCurrentNodeMarked = true;
+ }
+ }
+ if (!isCurrentNodeMarked) {
+ //This node does not initialized by client yet.
+ remainingFailNodes.add(failedDN);
+ }
+ }
+ this.pendingToFailNodes = remainingFailNodes;
+ }
@Override
public void close() throws IOException {
@@ -43,9 +71,13 @@
@Override
public XceiverClientSpi acquireClient(Pipeline pipeline)
throws IOException {
- return new MockXceiverClientSpi(pipeline, storage
- .computeIfAbsent(pipeline.getFirstNode(),
- r -> new MockDatanodeStorage()));
+ MockXceiverClientSpi mockXceiverClientSpi =
+ new MockXceiverClientSpi(pipeline, storage
+ .computeIfAbsent(pipeline.getFirstNode(),
+ r -> new MockDatanodeStorage()));
+ // Incase if this node already set to mark as failed.
+ setFailedStorages(this.pendingToFailNodes);
+ return mockXceiverClientSpi;
}
@Override
@@ -67,6 +99,13 @@
boolean b) {
}
-};
+
+ /**
+ * Returns data nodes details.
+ */
+ public Map<DatanodeDetails, MockDatanodeStorage> getStorages() {
+ return this.storage;
+ }
+}
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java
index 4d3db44..4dfe966 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientSpi.java
@@ -38,6 +38,7 @@
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import java.io.IOException;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
@@ -82,7 +83,13 @@
switch (request.getCmdType()) {
case WriteChunk:
return result(request,
- r -> r.setWriteChunk(writeChunk(request.getWriteChunk())));
+ r -> {
+ try {
+ return r.setWriteChunk(writeChunk(request.getWriteChunk()));
+ } catch (IOException e) {
+ return r.setResult(Result.IO_EXCEPTION);
+ }
+ });
case ReadChunk:
return result(request,
r -> r.setReadChunk(readChunk(request.getReadChunk())));
@@ -149,7 +156,7 @@
}
private WriteChunkResponseProto writeChunk(
- WriteChunkRequestProto writeChunk) {
+ WriteChunkRequestProto writeChunk) throws IOException {
datanodeStorage
.writeChunk(writeChunk.getBlockID(), writeChunk.getChunkData(),
writeChunk.getData());
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MultiNodePipelineBlockAllocator.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MultiNodePipelineBlockAllocator.java
new file mode 100644
index 0000000..97c5cfd
--- /dev/null
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MultiNodePipelineBlockAllocator.java
@@ -0,0 +1,143 @@
+/*
+ * 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.client;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Allocates the block with required number of nodes in the pipeline.
+ * The nodes are pre-created with port numbers starting from 0 to
+ * ( given cluster size -1).
+ */
+public class MultiNodePipelineBlockAllocator implements MockBlockAllocator {
+ private long blockId;
+ private int requiredNodes;
+ private final ConfigurationSource conf;
+ private List<HddsProtos.DatanodeDetailsProto> clusterDns = new ArrayList<>();
+ private int start = 0;
+
+ public MultiNodePipelineBlockAllocator(OzoneConfiguration conf,
+ int requiredNodes, int clusterSize) {
+ this.requiredNodes = requiredNodes;
+ this.conf = conf;
+ // Pre-initializing the datanodes. Later allocateBlock API will use this
+ // nodes to add the required number of nodes in the block pipelines.
+ for (int i = 0; i < clusterSize; i++) {
+ clusterDns.add(HddsProtos.DatanodeDetailsProto.newBuilder().setUuid128(
+ HddsProtos.UUID.newBuilder().setLeastSigBits(i).setMostSigBits(i)
+ .build()).setHostName("localhost").setIpAddress("1.2.3.4")
+ .addPorts(HddsProtos.Port.newBuilder().setName("RATIS").setValue(i)
+ .build()).build());
+ }
+ }
+
+ public List<HddsProtos.DatanodeDetailsProto> getClusterDns() {
+ return this.clusterDns;
+ }
+
+ /**
+ * This method selects the block pipeline nodes from the pre-created cluster
+ * nodes(clusterDns). It will use requiredNodes field to decide how many nodes
+ * to be chosen for the pipeline. To make the tests easy prediction of the
+ * node allocations, it will choose block pipeline nodes in a sliding window
+ * fashion starting from 0th index in clusterDns in incrementing order until
+ * given requireNodes number. Similarly for the next block pipeline, it will
+ * start from the index location of previous chosen pipeline's last node index
+ * + 1. Let's say cluster size was initialized with 10 and required nodes are
+ * 5, the first block pipeline will have nodes from 0 to 4 and the second
+ * block will be assigned with the index locations of 5th to 9th nodes. Once
+ * we finish round of allocations, then it will start from 0 again for next
+ * block. It will also support exclude list. If client passes exclude list, it
+ * will simply skip the node if it presents in exclude list, instead it will
+ * simply take the next node. If not enough nodes left due to the grown
+ * exclude list, it will throw IllegalStateException.
+ *
+ * @param keyArgs
+ * @param excludeList
+ * @return KeyLocation
+ */
+ @Override
+ public Iterable<? extends OzoneManagerProtocolProtos.KeyLocation>
+ allocateBlock(OzoneManagerProtocolProtos.KeyArgs keyArgs,
+ ExcludeList excludeList) {
+ HddsProtos.Pipeline.Builder builder =
+ HddsProtos.Pipeline.newBuilder().setFactor(keyArgs.getFactor())
+ .setType(keyArgs.getType()).setId(HddsProtos.PipelineID.newBuilder()
+ .setUuid128(HddsProtos.UUID.newBuilder().setLeastSigBits(1L)
+ .setMostSigBits(1L).build()).build());
+ addMembers(builder, requiredNodes, excludeList.getDatanodes(), keyArgs);
+ if (keyArgs.getType() == HddsProtos.ReplicationType.EC) {
+ builder.setEcReplicationConfig(keyArgs.getEcReplicationConfig());
+ }
+ final HddsProtos.Pipeline pipeline = builder.build();
+ List<OzoneManagerProtocolProtos.KeyLocation> results = new ArrayList<>();
+ long blockSize = (long) conf
+ .getStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE,
+ OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
+ results.add(OzoneManagerProtocolProtos.KeyLocation.newBuilder()
+ .setPipeline(pipeline).setBlockID(
+ HddsProtos.BlockID.newBuilder().setBlockCommitSequenceId(1L)
+ .setContainerBlockID(
+ HddsProtos.ContainerBlockID.newBuilder().setContainerID(1L)
+ .setLocalID(blockId++).build()).build()).setOffset(0L)
+ .setLength(blockSize).build());
+ return results;
+ }
+
+ private void addMembers(HddsProtos.Pipeline.Builder builder, int nodesNeeded,
+ Set<DatanodeDetails> excludedDataNodes,
+ OzoneManagerProtocolProtos.KeyArgs keyArgs) {
+ int clusterSize = clusterDns.size();
+ int counter = nodesNeeded;
+ int j = 0;
+ for (int i = 0; i < clusterDns.size(); i++) {
+ HddsProtos.DatanodeDetailsProto datanodeDetailsProto =
+ clusterDns.get(start % clusterSize);
+ start++;
+ if (excludedDataNodes
+ .contains(DatanodeDetails.getFromProtoBuf(datanodeDetailsProto))) {
+ continue;
+ } else {
+ builder.addMembers(datanodeDetailsProto);
+ if (keyArgs.getType() == HddsProtos.ReplicationType.EC) {
+ builder.addMemberReplicaIndexes(++j);
+ }
+ if (--counter == 0) {
+ break;
+ }
+ }
+ }
+ if (counter > 0) {
+ throw new IllegalStateException(
+ "MockedImpl: Could not find enough nodes.");
+ }
+ }
+
+}
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/SinglePipelineBlockAllocator.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/SinglePipelineBlockAllocator.java
index 71de5e8..20fbc03 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/SinglePipelineBlockAllocator.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/SinglePipelineBlockAllocator.java
@@ -17,6 +17,9 @@
*/
package org.apache.hadoop.ozone.client;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockID;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerBlockID;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
@@ -24,6 +27,8 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.PipelineID;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.Port;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.UUID;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation;
@@ -38,17 +43,18 @@
private long blockId;
private Pipeline pipeline;
+ private OzoneConfiguration conf;
- public SinglePipelineBlockAllocator() {
-
+ public SinglePipelineBlockAllocator(OzoneConfiguration conf) {
+ this.conf = conf;
}
@Override
- public Iterable<? extends KeyLocation> allocateBlock(
- KeyArgs keyArgs) {
+ public Iterable<? extends KeyLocation> allocateBlock(KeyArgs keyArgs,
+ ExcludeList excludeList) {
if (pipeline == null) {
- pipeline = Pipeline.newBuilder()
+ Pipeline.Builder bldr = Pipeline.newBuilder()
.setFactor(keyArgs.getFactor())
.setType(keyArgs.getType())
.setId(PipelineID.newBuilder()
@@ -68,10 +74,18 @@
.setName("RATIS")
.setValue(1234)
.build())
- .build())
- .build();
+ .build());
+ if (keyArgs.getType() == HddsProtos.ReplicationType.EC) {
+ bldr.setEcReplicationConfig(keyArgs.getEcReplicationConfig());
+ }
+ pipeline = bldr.build();
}
+ long blockSize = (long)conf.getStorageSize(
+ OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE,
+ OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT,
+ StorageUnit.BYTES);
+
List<KeyLocation> results = new ArrayList<>();
results.add(KeyLocation.newBuilder()
.setPipeline(pipeline)
@@ -83,7 +97,7 @@
.build())
.build())
.setOffset(0L)
- .setLength(keyArgs.getDataSize())
+ .setLength(blockSize)
.build());
return results;
}
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClient.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClient.java
index 2f4ebcf..23b181f 100644
--- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClient.java
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneClient.java
@@ -18,10 +18,13 @@
package org.apache.hadoop.ozone.client;
+import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
-import org.apache.hadoop.hdds.conf.InMemoryConfiguration;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
@@ -70,21 +73,24 @@
@Before
public void init() throws IOException {
- ConfigurationSource config = new InMemoryConfiguration();
+ OzoneConfiguration config = new OzoneConfiguration();
+ createNewClient(config, new SinglePipelineBlockAllocator(config));
+ }
+
+ private void createNewClient(ConfigurationSource config,
+ MockBlockAllocator blkAllocator) throws IOException {
client = new OzoneClient(config, new RpcClient(config, null) {
@Override
- protected OmTransport createOmTransport(
- String omServiceId)
+ protected OmTransport createOmTransport(String omServiceId)
throws IOException {
- return new MockOmTransport();
+ return new MockOmTransport(blkAllocator);
}
@NotNull
@Override
protected XceiverClientFactory createXceiverClientFactory(
- List<X509Certificate> x509Certificates)
- throws IOException {
+ List<X509Certificate> x509Certificates) throws IOException {
return new MockXceiverClientFactory();
}
});
@@ -183,6 +189,35 @@
}
}
+ @Test
+ public void testPutKeyWithECReplicationConfig() throws IOException {
+ close();
+ OzoneConfiguration config = new OzoneConfiguration();
+ config.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2,
+ StorageUnit.KB);
+ int data = 3;
+ int parity = 2;
+ int chunkSize = 1024;
+ createNewClient(config,
+ new MultiNodePipelineBlockAllocator(config, data + parity, 15));
+ String value = new String(new byte[chunkSize], UTF_8);
+ OzoneBucket bucket = getOzoneBucket();
+
+ for (int i = 0; i < 10; i++) {
+ String keyName = UUID.randomUUID().toString();
+ try (OzoneOutputStream out = bucket
+ .createKey(keyName, value.getBytes(UTF_8).length,
+ new ECReplicationConfig(data, parity,
+ ECReplicationConfig.EcCodec.RS, chunkSize),
+ new HashMap<>())) {
+ out.write(value.getBytes(UTF_8));
+ out.write(value.getBytes(UTF_8));
+ }
+ OzoneKey key = bucket.getKey(keyName);
+ Assert.assertEquals(keyName, key.getName());
+ }
+ }
+
private OzoneBucket getOzoneBucket() throws IOException {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneECClient.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneECClient.java
new file mode 100644
index 0000000..38b0940
--- /dev/null
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestOzoneECClient.java
@@ -0,0 +1,1066 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client;
+
+import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+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.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry;
+import org.apache.hadoop.ozone.client.io.BlockStreamAccessor;
+import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.protocolPB.OmTransport;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.ozone.erasurecode.rawcoder.RSRawErasureCoderFactory;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * Real unit test for OzoneECClient.
+ * <p>
+ * Used for testing Ozone client without external network calls.
+ */
+public class TestOzoneECClient {
+ private int chunkSize = 1024;
+ private int dataBlocks = 3;
+ private int parityBlocks = 2;
+ private int inputSize = chunkSize * dataBlocks;
+ private OzoneClient client;
+ private ObjectStore store;
+ private String keyName = UUID.randomUUID().toString();
+ private String volumeName = UUID.randomUUID().toString();
+ private String bucketName = UUID.randomUUID().toString();
+ private byte[][] inputChunks = new byte[dataBlocks][chunkSize];
+ private final XceiverClientFactory factoryStub =
+ new MockXceiverClientFactory();
+ private OzoneConfiguration conf = new OzoneConfiguration();
+ private MultiNodePipelineBlockAllocator allocator =
+ new MultiNodePipelineBlockAllocator(conf, dataBlocks + parityBlocks, 15);
+ private final MockOmTransport transportStub = new MockOmTransport(allocator);
+ private final RawErasureEncoder encoder =
+ new RSRawErasureCoderFactory().createEncoder(
+ new ECReplicationConfig(dataBlocks, parityBlocks));
+
+ @Before
+ public void init() throws IOException {
+ conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2,
+ StorageUnit.KB);
+ createNewClient(conf, transportStub);
+ }
+
+ private void createNewClient(ConfigurationSource config,
+ MockBlockAllocator blkAllocator) throws IOException {
+ createNewClient(config, new MockOmTransport(blkAllocator));
+ }
+
+ private void createNewClient(ConfigurationSource config,
+ final MockOmTransport transport) throws IOException {
+ client = new OzoneClient(config, new RpcClient(config, null) {
+
+ @Override
+ protected OmTransport createOmTransport(String omServiceId)
+ throws IOException {
+ return transport;
+ }
+
+ @Override
+ protected XceiverClientFactory createXceiverClientFactory(
+ List<X509Certificate> x509Certificates) throws IOException {
+ return factoryStub;
+ }
+ });
+
+ store = client.getObjectStore();
+ initInputChunks();
+ }
+
+ private void initInputChunks() {
+ for (int i = 0; i < dataBlocks; i++) {
+ inputChunks[i] = getBytesWith(i + 1, chunkSize);
+ }
+ }
+
+ private byte[] getBytesWith(int singleDigitNumber, int total) {
+ StringBuilder builder = new StringBuilder(singleDigitNumber);
+ for (int i = 1; i <= total; i++) {
+ builder.append(singleDigitNumber);
+ }
+ return builder.toString().getBytes(UTF_8);
+ }
+
+ @After
+ public void close() throws IOException {
+ client.close();
+ }
+
+ @Test
+ public void testPutECKeyAndCheckDNStoredData() throws IOException {
+ OzoneBucket bucket = writeIntoECKey(inputChunks, keyName, null);
+ OzoneKey key = bucket.getKey(keyName);
+ Assert.assertEquals(keyName, key.getName());
+ Map<DatanodeDetails, MockDatanodeStorage> storages =
+ ((MockXceiverClientFactory) factoryStub).getStorages();
+ DatanodeDetails[] dnDetails =
+ storages.keySet().toArray(new DatanodeDetails[storages.size()]);
+ Arrays.sort(dnDetails);
+ for (int i = 0; i < inputChunks.length; i++) {
+ MockDatanodeStorage datanodeStorage = storages.get(dnDetails[i]);
+ Assert.assertEquals(1, datanodeStorage.getAllBlockData().size());
+ ByteString content =
+ datanodeStorage.getAllBlockData().values().iterator().next();
+ Assert.assertEquals(new String(inputChunks[i], UTF_8),
+ content.toStringUtf8());
+ }
+ }
+
+ @Test
+ public void testPutECKeyAndCheckParityData() throws IOException {
+ OzoneBucket bucket = writeIntoECKey(inputChunks, keyName, null);
+ final ByteBuffer[] dataBuffers = new ByteBuffer[dataBlocks];
+ for (int i = 0; i < inputChunks.length; i++) {
+ dataBuffers[i] = ByteBuffer.wrap(inputChunks[i]);
+ }
+ final ByteBuffer[] parityBuffers = new ByteBuffer[parityBlocks];
+ for (int i = 0; i < parityBlocks; i++) {
+ parityBuffers[i] = ByteBuffer.allocate(chunkSize);
+ }
+ encoder.encode(dataBuffers, parityBuffers);
+ OzoneKey key = bucket.getKey(keyName);
+ Assert.assertEquals(keyName, key.getName());
+ Map<DatanodeDetails, MockDatanodeStorage> storages =
+ ((MockXceiverClientFactory) factoryStub).getStorages();
+ DatanodeDetails[] dnDetails =
+ storages.keySet().toArray(new DatanodeDetails[storages.size()]);
+ Arrays.sort(dnDetails);
+
+ for (int i = dataBlocks; i < parityBlocks + dataBlocks; i++) {
+ MockDatanodeStorage datanodeStorage = storages.get(dnDetails[i]);
+ Assert.assertEquals(1, datanodeStorage.getAllBlockData().size());
+ ByteString content =
+ datanodeStorage.getAllBlockData().values().iterator().next();
+ Assert.assertEquals(
+ new String(parityBuffers[i - dataBlocks].array(), UTF_8),
+ content.toStringUtf8());
+ }
+
+ }
+
+ @Test
+ public void testPutECKeyAndReadContent() throws IOException {
+ OzoneBucket bucket = writeIntoECKey(inputChunks, keyName, null);
+ OzoneKey key = bucket.getKey(keyName);
+ Assert.assertEquals(keyName, key.getName());
+ try (OzoneInputStream is = bucket.readKey(keyName)) {
+ byte[] fileContent = new byte[chunkSize];
+ for (int i = 0; i < dataBlocks; i++) {
+ Assert.assertEquals(inputChunks[i].length, is.read(fileContent));
+ Assert.assertTrue(Arrays.equals(inputChunks[i], fileContent));
+ }
+ // A further read should give EOF
+ Assert.assertEquals(-1, is.read(fileContent));
+ }
+ }
+
+ @Test
+ public void testCreateBucketWithDefaultReplicationConfig()
+ throws IOException {
+ final OzoneBucket bucket = writeIntoECKey(inputChunks, keyName,
+ new DefaultReplicationConfig(ReplicationType.EC,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS, chunkSize)));
+
+ // create key without mentioning replication config. Since we set EC
+ // replication in bucket, key should be EC key.
+ try (OzoneOutputStream out = bucket.createKey("mykey", inputSize)) {
+ Assert.assertTrue(out.getOutputStream() instanceof ECKeyOutputStream);
+ for (int i = 0; i < inputChunks.length; i++) {
+ out.write(inputChunks[i]);
+ }
+ }
+ }
+
+ @Test
+ public void test4ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(4);
+ }
+
+ // Test random number of chunks in single write op.
+ @Test
+ public void test5ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(5);
+ }
+
+ @Test
+ public void test6ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(6);
+ }
+
+ @Test
+ public void test7ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(7);
+ }
+
+ @Test
+ public void test9ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(9);
+ }
+
+ @Test
+ public void test10ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(10);
+ }
+
+ @Test
+ public void test12ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(12);
+ }
+
+ public void testMultipleChunksInSingleWriteOp(int numChunks)
+ throws IOException {
+ byte[] inputData = new byte[numChunks * chunkSize];
+ for (int i = 0; i < numChunks; i++) {
+ int start = (i * chunkSize);
+ Arrays.fill(inputData, start, start + chunkSize - 1,
+ String.valueOf(i % 9).getBytes(UTF_8)[0]);
+ }
+ final OzoneBucket bucket = writeIntoECKey(inputData, keyName,
+ new DefaultReplicationConfig(ReplicationType.EC,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS, chunkSize)));
+ OzoneKey key = bucket.getKey(keyName);
+ validateContent(inputData, bucket, key);
+ }
+
+ private void validateContent(byte[] inputData, OzoneBucket bucket,
+ OzoneKey key) throws IOException {
+ Assert.assertEquals(keyName, key.getName());
+ try (OzoneInputStream is = bucket.readKey(keyName)) {
+ byte[] fileContent = new byte[inputData.length];
+ Assert.assertEquals(inputData.length, is.read(fileContent));
+ Assert.assertEquals(new String(inputData, UTF_8),
+ new String(fileContent, UTF_8));
+ }
+ }
+
+ @Test
+ public void testSmallerThanChunkSize() throws IOException {
+ byte[] firstSmallChunk = new byte[chunkSize - 1];
+ Arrays.fill(firstSmallChunk, 0, firstSmallChunk.length - 1,
+ Byte.parseByte("1"));
+
+ writeIntoECKey(firstSmallChunk, keyName,
+ new DefaultReplicationConfig(ReplicationType.EC,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS, chunkSize)));
+ OzoneManagerProtocolProtos.KeyLocationList blockList =
+ transportStub.getKeys().get(volumeName).get(bucketName).get(keyName)
+ .getKeyLocationListList().get(0);
+
+ Map<DatanodeDetails, MockDatanodeStorage> storages =
+ ((MockXceiverClientFactory) factoryStub).getStorages();
+ OzoneManagerProtocolProtos.KeyLocation keyLocations =
+ blockList.getKeyLocations(0);
+
+ List<MockDatanodeStorage> dns = new ArrayList<>();
+ for (int i = 0; i < dataBlocks + parityBlocks; i++) {
+ HddsProtos.DatanodeDetailsProto member =
+ blockList.getKeyLocations(0).getPipeline().getMembers(i);
+ MockDatanodeStorage mockDatanodeStorage =
+ storages.get(getMatchingStorage(storages, member.getUuid()));
+ dns.add(mockDatanodeStorage);
+ }
+ String firstBlockData = dns.get(0).getFullBlockData(new BlockID(
+ keyLocations.getBlockID().getContainerBlockID().getContainerID(),
+ keyLocations.getBlockID().getContainerBlockID().getLocalID()));
+
+ Assert.assertArrayEquals(firstSmallChunk, firstBlockData.getBytes(UTF_8));
+
+ final ByteBuffer[] dataBuffers = new ByteBuffer[dataBlocks];
+ dataBuffers[0] = ByteBuffer.wrap(firstSmallChunk);
+ //Let's pad the remaining length equal to firstSmall chunk len
+ for (int i = 1; i < dataBlocks; i++) {
+ dataBuffers[i] = ByteBuffer.allocate(firstSmallChunk.length);
+ Arrays.fill(dataBuffers[i].array(), 0, firstSmallChunk.length, (byte) 0);
+ }
+
+ final ByteBuffer[] parityBuffers = new ByteBuffer[parityBlocks];
+ for (int i = 0; i < parityBlocks; i++) {
+ parityBuffers[i] = ByteBuffer.allocate(firstSmallChunk.length);
+ }
+ encoder.encode(dataBuffers, parityBuffers);
+
+ //Lets assert the parity data.
+ for (int i = dataBlocks; i < dataBlocks + parityBlocks; i++) {
+ String parityBlockData = dns.get(i).getFullBlockData(new BlockID(
+ keyLocations.getBlockID().getContainerBlockID().getContainerID(),
+ keyLocations.getBlockID().getContainerBlockID().getLocalID()));
+ String expected =
+ new String(parityBuffers[i - dataBlocks].array(), UTF_8);
+ Assert.assertEquals(expected, parityBlockData);
+ Assert.assertEquals(expected.length(), parityBlockData.length());
+
+ }
+ }
+
+ private static DatanodeDetails getMatchingStorage(
+ Map<DatanodeDetails, MockDatanodeStorage> storages, String uuid) {
+ Iterator<DatanodeDetails> iterator = storages.keySet().iterator();
+ while (iterator.hasNext()) {
+ DatanodeDetails dn = iterator.next();
+ if (dn.getUuid().toString().equals(uuid)) {
+ return dn;
+ }
+ }
+ return null;
+ }
+
+ @Test
+ public void testMultipleChunksWithPartialChunkInSingleWriteOp()
+ throws IOException {
+ final int partialChunkLen = 10;
+ final int numFullChunks = 9;
+ final int inputBuffLen = (numFullChunks * chunkSize) + partialChunkLen;
+ byte[] inputData = new byte[inputBuffLen];
+ for (int i = 0; i < numFullChunks; i++) {
+ int start = (i * chunkSize);
+ Arrays.fill(inputData, start, start + chunkSize - 1,
+ String.valueOf(i).getBytes(UTF_8)[0]);
+ }
+ //fill the last partial chunk as well.
+ Arrays.fill(inputData, (numFullChunks * chunkSize),
+ ((numFullChunks * chunkSize)) + partialChunkLen - 1, (byte) 1);
+ final OzoneBucket bucket = writeIntoECKey(inputData, keyName,
+ new DefaultReplicationConfig(ReplicationType.EC,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS, chunkSize)));
+ OzoneKey key = bucket.getKey(keyName);
+ validateContent(inputData, bucket, key);
+ }
+
+ @Test
+ public void testCommitKeyInfo()
+ throws IOException {
+ final OzoneBucket bucket = writeIntoECKey(inputChunks, keyName,
+ new DefaultReplicationConfig(ReplicationType.EC,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS, chunkSize)));
+
+ // create key without mentioning replication config. Since we set EC
+ // replication in bucket, key should be EC key.
+ try (OzoneOutputStream out = bucket.createKey("mykey", 6 * inputSize)) {
+ Assert.assertTrue(out.getOutputStream() instanceof ECKeyOutputStream);
+ // Block Size is 2kb, so to create 3 blocks we need 6 iterations here
+ for (int j = 0; j < 6; j++) {
+ for (int i = 0; i < inputChunks.length; i++) {
+ out.write(inputChunks[i]);
+ }
+ }
+ }
+ OzoneManagerProtocolProtos.KeyLocationList blockList =
+ transportStub.getKeys().get(volumeName).get(bucketName).get("mykey")
+ .getKeyLocationListList().get(0);
+
+ Assert.assertEquals(3, blockList.getKeyLocationsCount());
+ // As the mock allocator allocates block with id's increasing sequentially
+ // from 1. Therefore the block should be in the order with id starting 1, 2
+ // and then 3.
+ for (int i = 0; i < 3; i++) {
+ long localId = blockList.getKeyLocationsList().get(i).getBlockID()
+ .getContainerBlockID().getLocalID();
+ Assert.assertEquals(i + 1, localId);
+ }
+
+ Assert.assertEquals(1,
+ transportStub.getKeys().get(volumeName).get(bucketName).get("mykey")
+ .getKeyLocationListCount());
+ Assert.assertEquals(inputChunks[0].length * 3 * 6,
+ transportStub.getKeys().get(volumeName).get(bucketName).get("mykey")
+ .getDataSize());
+ }
+
+ @Test
+ public void testPartialStripeWithSingleChunkAndPadding() throws IOException {
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ try (OzoneOutputStream out = bucket.createKey(keyName, inputSize,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+ for (int i = 0; i < inputChunks[0].length; i++) {
+ out.write(inputChunks[0][i]);
+ }
+ }
+ OzoneKey key = bucket.getKey(keyName);
+ validateContent(inputChunks[0], bucket, key);
+ }
+
+ @Test
+ public void testPartialStripeLessThanSingleChunkWithPadding()
+ throws IOException {
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ try (OzoneOutputStream out = bucket.createKey(keyName, inputSize,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+ for (int i = 0; i < inputChunks[0].length - 1; i++) {
+ out.write(inputChunks[0][i]);
+ }
+ }
+ OzoneKey key = bucket.getKey(keyName);
+ validateContent(Arrays.copyOf(inputChunks[0], inputChunks[0].length - 1),
+ bucket, key);
+ }
+
+ @Test
+ public void testPartialStripeWithPartialLastChunk()
+ throws IOException {
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ // Last chunk is one byte short of the others.
+ byte[] lastChunk =
+ Arrays.copyOf(inputChunks[inputChunks.length - 1],
+ inputChunks[inputChunks.length - 1].length - 1);
+
+ int inSize = chunkSize * (inputChunks.length - 1) + lastChunk.length;
+ try (OzoneOutputStream out = bucket.createKey(keyName, inSize,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+ for (int i = 0; i < inputChunks.length - 1; i++) {
+ out.write(inputChunks[i]);
+ }
+
+ for (int i = 0; i < lastChunk.length; i++) {
+ out.write(lastChunk[i]);
+ }
+ }
+
+ try (OzoneInputStream is = bucket.readKey(keyName)) {
+ byte[] fileContent = new byte[chunkSize];
+ for (int i = 0; i < 2; i++) {
+ Assert.assertEquals(inputChunks[i].length, is.read(fileContent));
+ Assert.assertTrue(Arrays.equals(inputChunks[i], fileContent));
+ }
+ Assert.assertEquals(lastChunk.length, is.read(fileContent));
+ Assert.assertTrue(Arrays.equals(lastChunk,
+ Arrays.copyOf(fileContent, lastChunk.length)));
+ // A further read should give EOF
+ Assert.assertEquals(-1, is.read(fileContent));
+ }
+ }
+
+ @Test
+ public void test10D4PConfigWithPartialStripe()
+ throws IOException {
+ // A large block size try to trigger potential overflow
+ // refer to: HDDS-6295
+ conf.set(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE,
+ OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT);
+ int dataBlks = 10;
+ int parityBlks = 4;
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ // A partial chunk to trigger partialStripe check
+ // in ECKeyOutputStream.close()
+ int inSize = chunkSize - 1;
+ byte[] partialChunk = new byte[inSize];
+
+ try (OzoneOutputStream out = bucket.createKey(keyName, inSize,
+ new ECReplicationConfig(dataBlks, parityBlks,
+ ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+ out.write(partialChunk);
+ }
+
+ try (OzoneInputStream is = bucket.readKey(keyName)) {
+ byte[] fileContent = new byte[chunkSize];
+ Assert.assertEquals(inSize, is.read(fileContent));
+ Assert.assertTrue(Arrays.equals(partialChunk,
+ Arrays.copyOf(fileContent, inSize)));
+ }
+ }
+
+ @Test
+ public void testWriteShouldFailIfMoreThanParityNodesFail()
+ throws IOException {
+ testNodeFailuresWhileWriting(new int[] {0, 1, 2}, 3, 2);
+ }
+
+ @Test
+ public void testWriteShouldSuccessIfLessThanParityNodesFail()
+ throws IOException {
+ testNodeFailuresWhileWriting(new int[] {0}, 2, 2);
+ }
+
+ @Test
+ public void testWriteShouldSuccessIf4NodesFailed() throws IOException {
+ testNodeFailuresWhileWriting(new int[] {0, 1, 2, 3}, 1, 2);
+ }
+
+ @Test
+ public void testWriteShouldSuccessWithAdditional1BlockGroupAfterFailure()
+ throws IOException {
+ testNodeFailuresWhileWriting(new int[] {0, 1, 2, 3}, 10, 3);
+ }
+
+ @Test
+ public void testStripeWriteRetriesOn2Failures() throws IOException {
+ OzoneConfiguration con = new OzoneConfiguration();
+ con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2, StorageUnit.KB);
+ // Cluster has 15 nodes. So, first we will create 3 block groups with
+ // distinct nodes in each. Block Group 1: 0-4, Block Group 2: 5-9, Block
+ // Group 3: 10-14
+ // To mark the node failed in the second block group.
+ int[] nodesIndexesToMarkFailure = new int[2];
+ nodesIndexesToMarkFailure[0] = 0;
+ // To mark the node failed in the second block group also.
+ nodesIndexesToMarkFailure[1] = 5;
+ // Mocked MultiNodePipelineBlockAllocator#allocateBlock implementation
+ // should pick next good block group as we have 15 nodes.
+ int clusterSize = 15;
+ testStripeWriteRetriesOnFailures(con, clusterSize,
+ nodesIndexesToMarkFailure);
+ // It should have used 3rd block group also. So, total initialized nodes
+ // count should be clusterSize.
+ Assert.assertTrue(((MockXceiverClientFactory) factoryStub).getStorages()
+ .size() == clusterSize);
+ }
+
+ @Test
+ public void testStripeWriteRetriesOn3Failures() throws IOException {
+ OzoneConfiguration con = new OzoneConfiguration();
+ con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2, StorageUnit.KB);
+
+ int[] nodesIndexesToMarkFailure = new int[3];
+ nodesIndexesToMarkFailure[0] = 0;
+ // To mark the node failed in the second block group.
+ nodesIndexesToMarkFailure[1] = 5;
+ // To mark the node failed in the third block group.
+ nodesIndexesToMarkFailure[2] = 10;
+ // Mocked MultiNodePipelineBlockAllocator#allocateBlock implementation will
+ // pick the remaining goods for the next block group.
+ int clusterSize = 15;
+ testStripeWriteRetriesOnFailures(con, clusterSize,
+ nodesIndexesToMarkFailure);
+ // It should have used 3rd block group also. So, total initialized nodes
+ // count should be clusterSize.
+ Assert.assertTrue(((MockXceiverClientFactory) factoryStub).getStorages()
+ .size() == clusterSize);
+ }
+
+ // The mocked impl throws IllegalStateException when there are not enough
+ // nodes in allocateBlock request. But write() converts it to IOException.
+ @Test(expected = IOException.class)
+ public void testStripeWriteRetriesOnAllNodeFailures() throws IOException {
+ OzoneConfiguration con = new OzoneConfiguration();
+ con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2, StorageUnit.KB);
+
+ // After writing first stripe, we will mark all nodes as bad in the cluster.
+ int clusterSize = 5;
+ int[] nodesIndexesToMarkFailure = new int[clusterSize];
+ for (int i = 0; i < nodesIndexesToMarkFailure.length; i++) {
+ nodesIndexesToMarkFailure[i] = i;
+ }
+ // Mocked MultiNodePipelineBlockAllocator#allocateBlock implementation can
+ // not pick new block group as all nodes in cluster marked as bad.
+ testStripeWriteRetriesOnFailures(con, clusterSize,
+ nodesIndexesToMarkFailure);
+ }
+
+ @Test
+ public void testStripeWriteRetriesOn4FailuresWith3RetriesAllowed()
+ throws IOException {
+ OzoneConfiguration con = new OzoneConfiguration();
+ con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 2, StorageUnit.KB);
+ con.setInt(OzoneConfigKeys.OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES, 3);
+
+ int[] nodesIndexesToMarkFailure = new int[4];
+ nodesIndexesToMarkFailure[0] = 0;
+ //To mark node failed in second block group.
+ nodesIndexesToMarkFailure[1] = 5;
+ //To mark node failed in third block group.
+ nodesIndexesToMarkFailure[2] = 10;
+ //To mark node failed in fourth block group.
+ nodesIndexesToMarkFailure[3] = 15;
+ try {
+ // Mocked MultiNodePipelineBlockAllocator#allocateBlock implementation can
+ // pick good block group, but client retries should be limited
+ // OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES_ON_FAILURE(here it was
+ // configured as 3). So, it should fail as we have marked 3 nodes as bad.
+ testStripeWriteRetriesOnFailures(con, 20, nodesIndexesToMarkFailure);
+ Assert.fail(
+ "Expecting it to fail as retries should exceed the max allowed times:"
+ + " " + 3);
+ } catch (IOException e) {
+ Assert.assertEquals("Completed max allowed retries 3 on stripe failures.",
+ e.getMessage());
+ }
+ }
+
+ public void testStripeWriteRetriesOnFailures(OzoneConfiguration con,
+ int clusterSize, int[] nodesIndexesToMarkFailure) throws IOException {
+ close();
+ MultiNodePipelineBlockAllocator blkAllocator =
+ new MultiNodePipelineBlockAllocator(con, dataBlocks + parityBlocks,
+ clusterSize);
+ createNewClient(con, blkAllocator);
+ int numChunksToWriteAfterFailure = 3;
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ try (OzoneOutputStream out = bucket.createKey(keyName, 1024 * 3,
+ new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ chunkSize), new HashMap<>())) {
+ for (int i = 0; i < dataBlocks; i++) {
+ out.write(inputChunks[i]);
+ }
+ Assert.assertTrue(
+ ((MockXceiverClientFactory) factoryStub).getStorages().size() == 5);
+ List<DatanodeDetails> failedDNs = new ArrayList<>();
+ List<HddsProtos.DatanodeDetailsProto> dns = blkAllocator.getClusterDns();
+
+ for (int j = 0; j < nodesIndexesToMarkFailure.length; j++) {
+ failedDNs.add(DatanodeDetails
+ .getFromProtoBuf(dns.get(nodesIndexesToMarkFailure[j])));
+ }
+
+ // First let's set storage as bad
+ ((MockXceiverClientFactory) factoryStub).setFailedStorages(failedDNs);
+
+ // Writer should be able to write by using 3rd block group.
+ for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+ out.write(inputChunks[i]);
+ }
+ }
+ final OzoneKeyDetails key = bucket.getKey(keyName);
+ // Data supposed to store in single block group. Since we introduced the
+ // failures after first stripe, the second stripe data should have been
+ // written into new blockgroup. So, we should have 2 block groups. That
+ // means two keyLocations.
+ Assert.assertEquals(2, key.getOzoneKeyLocations().size());
+ try (OzoneInputStream is = bucket.readKey(keyName)) {
+ byte[] fileContent = new byte[chunkSize];
+ for (int i = 0; i < dataBlocks; i++) {
+ Assert.assertEquals(inputChunks[i].length, is.read(fileContent));
+ Assert.assertTrue("Expected: " + new String(inputChunks[i],
+ UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+ Arrays.equals(inputChunks[i], fileContent));
+ }
+ for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+ Assert.assertEquals(inputChunks[i].length, is.read(fileContent));
+ Assert.assertTrue("Expected: " + new String(inputChunks[i],
+ UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+ Arrays.equals(inputChunks[i], fileContent));
+ }
+ }
+ }
+
+ public void testNodeFailuresWhileWriting(int[] nodesIndexesToMarkFailure,
+ int numChunksToWriteAfterFailure, int numExpectedBlockGrps)
+ throws IOException {
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ try (OzoneOutputStream out = bucket.createKey(keyName, 1024 * 3,
+ new ECReplicationConfig(3, 2,
+ ECReplicationConfig.EcCodec.RS,
+ chunkSize), new HashMap<>())) {
+ for (int i = 0; i < dataBlocks; i++) {
+ out.write(inputChunks[i]);
+ }
+
+ List<DatanodeDetails> failedDNs = new ArrayList<>();
+ List<HddsProtos.DatanodeDetailsProto> dns = allocator.getClusterDns();
+ for (int j = 0; j < nodesIndexesToMarkFailure.length; j++) {
+ failedDNs.add(DatanodeDetails
+ .getFromProtoBuf(dns.get(nodesIndexesToMarkFailure[j])));
+ }
+
+ // First let's set storage as bad
+ ((MockXceiverClientFactory) factoryStub).setFailedStorages(failedDNs);
+
+ for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+ out.write(inputChunks[i % dataBlocks]);
+ }
+ }
+ final OzoneKeyDetails key = bucket.getKey(keyName);
+ // Data supposed to store in single block group. Since we introduced the
+ // failures after first stripe, the second stripe data should have been
+ // written into new block group. So, we should have numExpectedBlockGrps.
+ // That means two keyLocations.
+ Assert
+ .assertEquals(numExpectedBlockGrps, key.getOzoneKeyLocations().size());
+ try (OzoneInputStream is = bucket.readKey(keyName)) {
+ byte[] fileContent = new byte[chunkSize];
+ for (int i = 0; i < dataBlocks; i++) {
+ Assert.assertEquals(inputChunks[i].length, is.read(fileContent));
+ Assert.assertTrue("Expected: " + new String(inputChunks[i],
+ UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+ Arrays.equals(inputChunks[i], fileContent));
+ }
+ for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+ Assert.assertEquals(inputChunks[i % dataBlocks].length,
+ is.read(fileContent));
+ Assert.assertTrue("Expected: " + new String(inputChunks[i % dataBlocks],
+ UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+ Arrays.equals(inputChunks[i % dataBlocks], fileContent));
+ }
+ }
+ }
+
+ @Test
+ public void testLargeWriteOfMultipleStripesWithStripeFailure()
+ throws IOException {
+ close();
+ OzoneConfiguration con = new OzoneConfiguration();
+ // block size of 3KB could hold 3 full stripes
+ con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 3, StorageUnit.KB);
+ con.setInt(OzoneConfigKeys.OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES, 3);
+ MultiNodePipelineBlockAllocator blkAllocator =
+ new MultiNodePipelineBlockAllocator(con, dataBlocks + parityBlocks,
+ 15);
+ createNewClient(con, blkAllocator);
+
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ // should write > 1 full stripe to trigger potential issue
+ int numFullStripesBeforeFailure = 2;
+ int numChunksToWriteAfterFailure = dataBlocks;
+ int numExpectedBlockGrps = 2;
+ // fail the DNs for parity blocks
+ int[] nodesIndexesToMarkFailure = {3, 4};
+
+ try (OzoneOutputStream out = bucket.createKey(keyName,
+ 1024 * dataBlocks * numFullStripesBeforeFailure
+ + numChunksToWriteAfterFailure,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS,
+ chunkSize), new HashMap<>())) {
+ for (int j = 0; j < numFullStripesBeforeFailure; j++) {
+ for (int i = 0; i < dataBlocks; i++) {
+ out.write(inputChunks[i]);
+ }
+ }
+
+ List<DatanodeDetails> failedDNs = new ArrayList<>();
+ List<HddsProtos.DatanodeDetailsProto> dns = allocator.getClusterDns();
+ for (int j = 0; j < nodesIndexesToMarkFailure.length; j++) {
+ failedDNs.add(DatanodeDetails
+ .getFromProtoBuf(dns.get(nodesIndexesToMarkFailure[j])));
+ }
+
+ // First let's set storage as bad
+ ((MockXceiverClientFactory) factoryStub).setFailedStorages(failedDNs);
+
+ for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+ out.write(inputChunks[i % dataBlocks]);
+ }
+ }
+
+ final OzoneKeyDetails key = bucket.getKey(keyName);
+ // Data supposed to store in single block group. Since we introduced the
+ // failures after first stripe, the second stripe data should have been
+ // written into new block group. So, we should have numExpectedBlockGrps.
+ // That means two keyLocations.
+ Assert
+ .assertEquals(numExpectedBlockGrps, key.getOzoneKeyLocations().size());
+ try (OzoneInputStream is = bucket.readKey(keyName)) {
+ byte[] fileContent = new byte[chunkSize];
+ for (int i = 0; i < dataBlocks * numFullStripesBeforeFailure; i++) {
+ Assert.assertEquals(inputChunks[i % dataBlocks].length,
+ is.read(fileContent));
+ Assert.assertTrue(
+ "Expected: " + new String(inputChunks[i % dataBlocks], UTF_8)
+ + " \n " + "Actual: " + new String(fileContent, UTF_8),
+ Arrays.equals(inputChunks[i % dataBlocks], fileContent));
+ }
+ for (int i = 0; i < numChunksToWriteAfterFailure; i++) {
+ Assert.assertEquals(inputChunks[i % dataBlocks].length,
+ is.read(fileContent));
+ Assert.assertTrue(
+ "Expected: " + new String(inputChunks[i % dataBlocks],
+ UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+ Arrays.equals(inputChunks[i % dataBlocks], fileContent));
+ }
+ }
+ }
+
+ @Test
+ public void testPartialStripeWithPartialChunkRetry()
+ throws IOException {
+ close();
+ OzoneConfiguration con = new OzoneConfiguration();
+ // block size of 3KB could hold 3 full stripes
+ con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 3, StorageUnit.KB);
+ con.setInt(OzoneConfigKeys.OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES, 3);
+ MultiNodePipelineBlockAllocator blkAllocator =
+ new MultiNodePipelineBlockAllocator(con, dataBlocks + parityBlocks, 15);
+ createNewClient(con, blkAllocator);
+
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+ int numFullChunks = 7;
+ //Prepare additional partial chunk.
+ int partialChunkSize = 1020;
+ byte[] partialChunk = new byte[partialChunkSize];
+ Arrays.fill(partialChunk, 0, partialChunk.length, "1".getBytes(UTF_8)[0]);
+
+ // A partial chunk to trigger partialStripe check
+ // in ECKeyOutputStream.close()
+ int inSize = chunkSize;
+ try (OzoneOutputStream out = bucket.createKey(keyName, inSize,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+ for (int i = 0; i < numFullChunks; i++) {
+ out.write(inputChunks[i % dataBlocks]);
+ }
+
+ out.write(partialChunk);
+
+ int[] nodesIndexesToMarkFailure = new int[] {0, 4};
+ List<DatanodeDetails> failedDNs = new ArrayList<>();
+ List<HddsProtos.DatanodeDetailsProto> dns = blkAllocator.getClusterDns();
+ for (int j = 0; j < nodesIndexesToMarkFailure.length; j++) {
+ failedDNs.add(DatanodeDetails
+ .getFromProtoBuf(dns.get(nodesIndexesToMarkFailure[j])));
+ }
+
+ // First let's set storage as bad
+ ((MockXceiverClientFactory) factoryStub).setFailedStorages(failedDNs);
+
+ }
+
+ try (OzoneInputStream is = bucket.readKey(keyName)) {
+ byte[] fileContent = new byte[chunkSize];
+ for (int i = 0; i < numFullChunks; i++) {
+ Assert.assertEquals(inputChunks[i % dataBlocks].length,
+ is.read(fileContent));
+ Assert.assertTrue("Expected: " + new String(inputChunks[i % dataBlocks],
+ UTF_8) + " \n " + "Actual: " + new String(fileContent, UTF_8),
+ Arrays.equals(inputChunks[i % dataBlocks], fileContent));
+ }
+
+ byte[] partialChunkToRead = new byte[partialChunkSize];
+ Assert
+ .assertEquals(partialChunkToRead.length, is.read(partialChunkToRead));
+ Assert.assertTrue(Arrays.equals(partialChunk, partialChunkToRead));
+
+ Assert.assertEquals(-1, is.read(partialChunkToRead));
+ }
+ }
+
+ @Test
+ public void testDiscardPreAllocatedBlocksPreventRetryExceeds()
+ throws IOException {
+ close();
+ OzoneConfiguration con = new OzoneConfiguration();
+ int maxRetries = 3;
+ con.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE,
+ 2, StorageUnit.KB);
+ con.setInt(OzoneConfigKeys.OZONE_CLIENT_MAX_EC_STRIPE_WRITE_RETRIES,
+ maxRetries);
+ MultiNodePipelineBlockAllocator blkAllocator =
+ new MultiNodePipelineBlockAllocator(con, dataBlocks + parityBlocks,
+ 15);
+ createNewClient(con, blkAllocator);
+
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ int numStripesBeforeFailure = 1;
+ int numStripesAfterFailure = 1;
+ int numStripesTotal = numStripesBeforeFailure + numStripesAfterFailure;
+ int numExpectedBlockGrps = 2;
+ // fail any DNs to trigger retry
+ int[] nodesIndexesToMarkFailure = {0, 1};
+ long keySize = (long) chunkSize * dataBlocks * numStripesTotal;
+
+ try (OzoneOutputStream out = bucket.createKey(keyName, keySize,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS,
+ chunkSize), new HashMap<>())) {
+ Assert.assertTrue(out.getOutputStream() instanceof ECKeyOutputStream);
+ ECKeyOutputStream kos = (ECKeyOutputStream) out.getOutputStream();
+ List<OmKeyLocationInfo> blockInfos = getAllLocationInfoList(kos);
+ Assert.assertEquals(1, blockInfos.size());
+
+ // Mock some pre-allocated blocks to the key,
+ // should be > maxRetries
+ int numPreAllocatedBlocks = maxRetries + 1;
+ BlockID blockID = blockInfos.get(0).getBlockID();
+ Pipeline pipeline = blockInfos.get(0).getPipeline();
+ List<OmKeyLocationInfo> omKeyLocationInfos = new ArrayList<>();
+ for (int i = 0; i < numPreAllocatedBlocks; i++) {
+ BlockID nextBlockID = new BlockID(blockID.getContainerID(),
+ blockID.getLocalID() + i + 1);
+ omKeyLocationInfos.add(new OmKeyLocationInfo.Builder()
+ .setBlockID(nextBlockID)
+ .setPipeline(pipeline)
+ .build());
+ }
+ OmKeyLocationInfoGroup omKeyLocationInfoGroup =
+ new OmKeyLocationInfoGroup(0, omKeyLocationInfos);
+ kos.addPreallocateBlocks(omKeyLocationInfoGroup, 0);
+
+ // Good writes
+ for (int j = 0; j < numStripesBeforeFailure; j++) {
+ for (int i = 0; i < dataBlocks; i++) {
+ out.write(inputChunks[i]);
+ }
+ }
+
+ // Make the writes fail to trigger retry
+ List<DatanodeDetails> failedDNs = new ArrayList<>();
+ List<HddsProtos.DatanodeDetailsProto> dns = allocator.getClusterDns();
+ for (int j = 0; j < nodesIndexesToMarkFailure.length; j++) {
+ failedDNs.add(DatanodeDetails
+ .getFromProtoBuf(dns.get(nodesIndexesToMarkFailure[j])));
+ }
+ // First let's set storage as bad
+ ((MockXceiverClientFactory) factoryStub).setFailedStorages(failedDNs);
+
+ // Writes that will retry due to failed DNs
+ try {
+ for (int j = 0; j < numStripesAfterFailure; j++) {
+ for (int i = 0; i < dataBlocks; i++) {
+ out.write(inputChunks[i]);
+ }
+ }
+ } catch (IOException e) {
+ // If we don't discard pre-allocated blocks,
+ // retries should exceed the maxRetries and write will fail.
+ Assert.fail("Max retries exceeded");
+ }
+ }
+
+ final OzoneKeyDetails key = bucket.getKey(keyName);
+ Assert.assertEquals(numExpectedBlockGrps,
+ key.getOzoneKeyLocations().size());
+
+ try (OzoneInputStream is = bucket.readKey(keyName)) {
+ byte[] fileContent = new byte[chunkSize];
+ for (int i = 0; i < dataBlocks * numStripesTotal; i++) {
+ Assert.assertEquals(inputChunks[i % dataBlocks].length,
+ is.read(fileContent));
+ Assert.assertArrayEquals(
+ "Expected: " + new String(inputChunks[i % dataBlocks], UTF_8)
+ + " \n " + "Actual: " + new String(fileContent, UTF_8),
+ inputChunks[i % dataBlocks], fileContent);
+ }
+ }
+ }
+
+ private OzoneBucket writeIntoECKey(byte[] data, String key,
+ DefaultReplicationConfig defaultReplicationConfig) throws IOException {
+ return writeIntoECKey(new byte[][] {data}, key, defaultReplicationConfig);
+ }
+
+ private OzoneBucket writeIntoECKey(byte[][] chunks, String key,
+ DefaultReplicationConfig defaultReplicationConfig) throws IOException {
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ if (defaultReplicationConfig != null) {
+ final BucketArgs.Builder builder = BucketArgs.newBuilder();
+ builder.setDefaultReplicationConfig(defaultReplicationConfig);
+ volume.createBucket(bucketName, builder.build());
+ } else {
+ volume.createBucket(bucketName);
+ }
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ int size = (int) Arrays.stream(chunks).mapToLong(a -> a.length).sum();
+ try (OzoneOutputStream out = bucket.createKey(key, size,
+ new ECReplicationConfig(dataBlocks, parityBlocks,
+ ECReplicationConfig.EcCodec.RS, chunkSize), new HashMap<>())) {
+ for (int i = 0; i < chunks.length; i++) {
+ out.write(chunks[i]);
+ }
+ }
+ return bucket;
+ }
+
+ private List<OmKeyLocationInfo> getAllLocationInfoList(
+ ECKeyOutputStream kos) {
+ List<OmKeyLocationInfo> locationInfoList = new ArrayList<>();
+ for (BlockOutputStreamEntry streamEntry : kos.getStreamEntries()) {
+ BlockStreamAccessor streamAccessor =
+ new BlockStreamAccessor(streamEntry);
+ OmKeyLocationInfo info =
+ new OmKeyLocationInfo.Builder()
+ .setBlockID(streamAccessor.getStreamBlockID())
+ .setLength(streamAccessor.getStreamCurrentPosition())
+ .setOffset(0)
+ .setToken(streamAccessor.getStreamToken())
+ .setPipeline(streamAccessor.getStreamPipeline())
+ .build();
+ locationInfoList.add(info);
+ }
+ return locationInfoList;
+ }
+}
\ No newline at end of file
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/BlockStreamAccessor.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/BlockStreamAccessor.java
new file mode 100644
index 0000000..d9a7211
--- /dev/null
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/BlockStreamAccessor.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.io;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * An accessor helper class to BlockOutputStreamEntry for test.
+ */
+public class BlockStreamAccessor {
+
+ private final BlockOutputStreamEntry streamEntry;
+
+ public BlockStreamAccessor(BlockOutputStreamEntry entry) {
+ this.streamEntry = entry;
+ }
+
+ public BlockID getStreamBlockID() {
+ return streamEntry.getBlockID();
+ }
+
+ public Pipeline getStreamPipeline() {
+ return streamEntry.getPipeline();
+ }
+
+ public Token<OzoneBlockTokenIdentifier> getStreamToken() {
+ return streamEntry.getToken();
+ }
+
+ public long getStreamCurrentPosition() {
+ return streamEntry.getCurrentPosition();
+ }
+}
diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockOutputStreamEntry.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockOutputStreamEntry.java
new file mode 100644
index 0000000..4b87ef7
--- /dev/null
+++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/io/TestECBlockOutputStreamEntry.java
@@ -0,0 +1,128 @@
+/*
+ * 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.client.io;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientManager;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * {@link ECBlockOutputStreamEntry} tests.
+ */
+public class TestECBlockOutputStreamEntry {
+
+ @Test
+ public void
+ testAcquireDifferentClientForECBlocksOnTheSameHostButDifferentPort()
+ throws IOException {
+ PipelineID randomId = PipelineID.randomId();
+ ReplicationConfig ecReplicationConfig =
+ new ECReplicationConfig("RS-3-2-1024k");
+ DatanodeDetails node1 = aNode("127.0.0.1", "localhost", 2001);
+ DatanodeDetails node2 = aNode("127.0.0.1", "localhost", 2002);
+ DatanodeDetails node3 = aNode("127.0.0.1", "localhost", 2003);
+ DatanodeDetails node4 = aNode("127.0.0.1", "localhost", 2004);
+ DatanodeDetails node5 = aNode("127.0.0.1", "localhost", 2005);
+ List<DatanodeDetails> nodes =
+ Arrays.asList(node1, node2, node3, node4, node5);
+ Pipeline anECPipeline = Pipeline.newBuilder()
+ .setId(randomId)
+ .setReplicationConfig(ecReplicationConfig)
+ .setState(Pipeline.PipelineState.OPEN)
+ .setNodes(nodes)
+ .build();
+ XceiverClientManager manager =
+ new XceiverClientManager(new OzoneConfiguration());
+ HashSet<XceiverClientSpi> clients = new HashSet<>();
+ ECBlockOutputStreamEntry entry = new ECBlockOutputStreamEntry.Builder()
+ .setXceiverClientManager(manager)
+ .setPipeline(anECPipeline)
+ .build();
+ for (int i = 0; i < nodes.size(); i++) {
+ clients.add(
+ manager.acquireClient(
+ entry.createSingleECBlockPipeline(
+ anECPipeline, nodes.get(i), i
+ )));
+ }
+ assertEquals(5, clients.size());
+ }
+
+ @Test
+ public void
+ testAcquireDifferentClientForECBlocksOnTheSameHostWithSomeOnSamePortAlso()
+ throws IOException {
+ PipelineID randomId = PipelineID.randomId();
+ ReplicationConfig ecReplicationConfig =
+ new ECReplicationConfig("RS-3-2-1024k");
+ DatanodeDetails node1 = aNode("127.0.0.1", "localhost", 2001);
+ DatanodeDetails node2 = aNode("127.0.0.1", "localhost", 2001);
+ DatanodeDetails node3 = aNode("127.0.0.1", "localhost", 2003);
+ DatanodeDetails node4 = aNode("127.0.0.1", "localhost", 2001);
+ DatanodeDetails node5 = aNode("127.0.0.1", "localhost", 2005);
+ List<DatanodeDetails> nodes =
+ Arrays.asList(node1, node2, node3, node4, node5);
+ Pipeline anECPipeline = Pipeline.newBuilder()
+ .setId(randomId)
+ .setReplicationConfig(ecReplicationConfig)
+ .setState(Pipeline.PipelineState.OPEN)
+ .setNodes(nodes)
+ .build();
+ XceiverClientManager manager =
+ new XceiverClientManager(new OzoneConfiguration());
+ HashSet<XceiverClientSpi> clients = new HashSet<>();
+ ECBlockOutputStreamEntry entry = new ECBlockOutputStreamEntry.Builder()
+ .setXceiverClientManager(manager)
+ .setPipeline(anECPipeline)
+ .build();
+ for (int i = 0; i < nodes.size(); i++) {
+ clients.add(
+ manager.acquireClient(
+ entry.createSingleECBlockPipeline(
+ anECPipeline, nodes.get(i), i
+ )));
+ }
+ assertEquals(3, clients.size());
+ assertEquals(1, clients.stream().filter(c -> c.getRefcount() == 3).count());
+ assertEquals(2, clients.stream().filter(c -> c.getRefcount() == 1).count());
+ }
+
+ private DatanodeDetails aNode(String ip, String hostName, int port) {
+ return DatanodeDetails.newBuilder()
+ .setUuid(UUID.randomUUID())
+ .setIpAddress(ip)
+ .setHostName(hostName)
+ .addPort(
+ DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, port))
+ .build();
+ }
+}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
index c38dbc7..5717ca6 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
@@ -19,6 +19,8 @@
import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.ratis.util.TimeDuration;
@@ -257,6 +259,16 @@
public static final boolean OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT =
false;
+ public static final String OZONE_SERVER_DEFAULT_REPLICATION_KEY =
+ "ozone.server.default.replication";
+ public static final String OZONE_SERVER_DEFAULT_REPLICATION_DEFAULT =
+ ReplicationFactor.THREE.toString();
+
+ public static final String OZONE_SERVER_DEFAULT_REPLICATION_TYPE_KEY =
+ "ozone.server.default.replication.type";
+ public static final String OZONE_SERVER_DEFAULT_REPLICATION_TYPE_DEFAULT =
+ ReplicationType.RATIS.toString();
+
public static final String OZONE_OM_HA_PREFIX = "ozone.om.ha";
public static final String OZONE_FS_TRASH_INTERVAL_KEY =
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
index 953aa23..168b2c8 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
@@ -242,6 +242,7 @@
UPDATE_LAYOUT_VERSION_FAILED,
LAYOUT_FEATURE_FINALIZATION_FAILED,
PREPARE_FAILED,
- NOT_SUPPORTED_OPERATION_WHEN_PREPARED
+ NOT_SUPPORTED_OPERATION_WHEN_PREPARED,
+ NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION
}
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
index 9291d33..b23da41 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
@@ -20,6 +20,7 @@
import java.util.LinkedHashMap;
import java.util.Map;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.audit.Auditable;
@@ -51,6 +52,7 @@
private long quotaInBytes;
private long quotaInNamespace;
+ private DefaultReplicationConfig defaultReplicationConfig = null;
/**
* Bucket Owner Name.
*/
@@ -129,7 +131,24 @@
}
/**
+ * Returns Bucket default replication config.
+ * @return
+ */
+ public DefaultReplicationConfig getDefaultReplicationConfig() {
+ return defaultReplicationConfig;
+ }
+
+ /**
+ * Sets the Bucket default replication config.
+ */
+ private void setDefaultReplicationConfig(
+ DefaultReplicationConfig defaultReplicationConfig) {
+ this.defaultReplicationConfig = defaultReplicationConfig;
+ }
+
+ /**
* Returns Bucket Owner Name.
+ *
* @return ownerName.
*/
public String getOwnerName() {
@@ -173,6 +192,7 @@
private Map<String, String> metadata;
private long quotaInBytes;
private long quotaInNamespace;
+ private DefaultReplicationConfig defaultReplicationConfig;
private String ownerName;
/**
* Constructs a builder.
@@ -217,6 +237,12 @@
return this;
}
+ public Builder setDefaultReplicationConfig(
+ DefaultReplicationConfig defaultRepConfig) {
+ this.defaultReplicationConfig = defaultRepConfig;
+ return this;
+ }
+
public Builder setOwnerName(String owner) {
ownerName = owner;
return this;
@@ -229,8 +255,11 @@
public OmBucketArgs build() {
Preconditions.checkNotNull(volumeName);
Preconditions.checkNotNull(bucketName);
- return new OmBucketArgs(volumeName, bucketName, isVersionEnabled,
- storageType, metadata, quotaInBytes, quotaInNamespace, ownerName);
+ OmBucketArgs omBucketArgs =
+ new OmBucketArgs(volumeName, bucketName, isVersionEnabled,
+ storageType, metadata, quotaInBytes, quotaInNamespace, ownerName);
+ omBucketArgs.setDefaultReplicationConfig(defaultReplicationConfig);
+ return omBucketArgs;
}
}
@@ -253,6 +282,9 @@
if (quotaInNamespace > 0 || quotaInNamespace == OzoneConsts.QUOTA_RESET) {
builder.setQuotaInNamespace(quotaInNamespace);
}
+ if (defaultReplicationConfig != null) {
+ builder.setDefaultReplicationConfig(defaultReplicationConfig.toProto());
+ }
if (ownerName != null) {
builder.setOwnerName(ownerName);
}
@@ -265,16 +297,22 @@
* @return instance of OmBucketArgs
*/
public static OmBucketArgs getFromProtobuf(BucketArgs bucketArgs) {
- return new OmBucketArgs(bucketArgs.getVolumeName(),
- bucketArgs.getBucketName(),
- bucketArgs.hasIsVersionEnabled() ?
- bucketArgs.getIsVersionEnabled() : null,
- bucketArgs.hasStorageType() ? StorageType.valueOf(
- bucketArgs.getStorageType()) : null,
- KeyValueUtil.getFromProtobuf(bucketArgs.getMetadataList()),
- bucketArgs.getQuotaInBytes(),
- bucketArgs.getQuotaInNamespace(),
- bucketArgs.hasOwnerName() ?
- bucketArgs.getOwnerName() : null);
+ OmBucketArgs omBucketArgs =
+ new OmBucketArgs(bucketArgs.getVolumeName(),
+ bucketArgs.getBucketName(),
+ bucketArgs.hasIsVersionEnabled() ?
+ bucketArgs.getIsVersionEnabled() : null,
+ bucketArgs.hasStorageType() ? StorageType.valueOf(
+ bucketArgs.getStorageType()) : null,
+ KeyValueUtil.getFromProtobuf(bucketArgs.getMetadataList()),
+ bucketArgs.getQuotaInBytes(),
+ bucketArgs.getQuotaInNamespace(),
+ bucketArgs.hasOwnerName() ?
+ bucketArgs.getOwnerName() : null);
+ // OmBucketArgs ctor already has more arguments, so setting the default
+ // replication config separately.
+ omBucketArgs.setDefaultReplicationConfig(
+ new DefaultReplicationConfig(bucketArgs.getDefaultReplicationConfig()));
+ return omBucketArgs;
}
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
index ad81c8b..846b71c 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
@@ -27,7 +27,13 @@
import java.util.Objects;
import java.util.stream.Collectors;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.audit.Auditable;
@@ -75,6 +81,11 @@
*/
private BucketEncryptionKeyInfo bekInfo;
+ /**
+ * Optional default replication for bucket.
+ */
+ private DefaultReplicationConfig defaultReplicationConfig;
+
private final String sourceVolume;
private final String sourceBucket;
@@ -109,8 +120,10 @@
* @param usedBytes - Bucket Quota Usage in bytes.
* @param quotaInBytes Bucket quota in bytes.
* @param quotaInNamespace Bucket quota in counts.
- * @param bucketLayout Bucket Layout.
+ * @param bucketLayout bucket layout.
* @param owner owner of the bucket.
+ * @param defaultReplicationConfig default replication config.
+ * @param bucketLayout Bucket Layout.
*/
@SuppressWarnings("checkstyle:ParameterNumber")
private OmBucketInfo(String volumeName,
@@ -131,7 +144,8 @@
long quotaInBytes,
long quotaInNamespace,
BucketLayout bucketLayout,
- String owner) {
+ String owner,
+ DefaultReplicationConfig defaultReplicationConfig) {
this.volumeName = volumeName;
this.bucketName = bucketName;
this.acls = acls;
@@ -151,6 +165,7 @@
this.quotaInNamespace = quotaInNamespace;
this.bucketLayout = bucketLayout;
this.owner = owner;
+ this.defaultReplicationConfig = defaultReplicationConfig;
}
/**
@@ -250,12 +265,22 @@
/**
* Returns the Bucket Layout.
+ *
* @return BucketLayout.
*/
public BucketLayout getBucketLayout() {
return bucketLayout;
}
+ /**
+ * Returns bucket EC replication config.
+ *
+ * @return EC replication config.
+ */
+ public DefaultReplicationConfig getDefaultReplicationConfig() {
+ return defaultReplicationConfig;
+ }
+
public String getSourceVolume() {
return sourceVolume;
}
@@ -358,6 +383,10 @@
acl.getName(), (BitSet) acl.getAclBitSet().clone(),
acl.getAclScope())));
+ if (defaultReplicationConfig != null) {
+ builder.setDefaultReplicationConfig(defaultReplicationConfig.copy());
+ }
+
return builder.build();
}
@@ -381,7 +410,21 @@
.setQuotaInBytes(quotaInBytes)
.setQuotaInNamespace(quotaInNamespace)
.setBucketLayout(bucketLayout)
- .setOwner(owner);
+ .setOwner(owner)
+ .setDefaultReplicationConfig(defaultReplicationConfig);
+ }
+
+ public void setDefaultReplicationConfig(ReplicationConfig replicationConfig) {
+ this.defaultReplicationConfig = new DefaultReplicationConfig(
+ ReplicationType.fromProto(replicationConfig.getReplicationType()),
+ replicationConfig
+ .getReplicationType() == HddsProtos.ReplicationType.EC ?
+ null :
+ ReplicationFactor.valueOf(replicationConfig.getRequiredNodes()),
+ replicationConfig
+ .getReplicationType() == HddsProtos.ReplicationType.EC ?
+ ((ECReplicationConfig) replicationConfig) :
+ null);
}
/**
@@ -407,6 +450,7 @@
private long quotaInNamespace;
private BucketLayout bucketLayout;
private String owner;
+ private DefaultReplicationConfig defaultReplicationConfig;
public Builder() {
//Default values
@@ -535,6 +579,12 @@
return this;
}
+ public Builder setDefaultReplicationConfig(
+ DefaultReplicationConfig defaultReplConfig) {
+ this.defaultReplicationConfig = defaultReplConfig;
+ return this;
+ }
+
/**
* Constructs the OmBucketInfo.
* @return instance of OmBucketInfo.
@@ -545,11 +595,11 @@
Preconditions.checkNotNull(acls);
Preconditions.checkNotNull(isVersionEnabled);
Preconditions.checkNotNull(storageType);
-
return new OmBucketInfo(volumeName, bucketName, acls, isVersionEnabled,
storageType, creationTime, modificationTime, objectID, updateID,
metadata, bekInfo, sourceVolume, sourceBucket, usedBytes,
- usedNamespace, quotaInBytes, quotaInNamespace, bucketLayout, owner);
+ usedNamespace, quotaInBytes, quotaInNamespace, bucketLayout, owner,
+ defaultReplicationConfig);
}
}
@@ -578,6 +628,9 @@
if (bekInfo != null && bekInfo.getKeyName() != null) {
bib.setBeinfo(OMPBHelper.convert(bekInfo));
}
+ if (defaultReplicationConfig != null) {
+ bib.setDefaultReplicationConfig(defaultReplicationConfig.toProto());
+ }
if (sourceVolume != null) {
bib.setSourceVolume(sourceVolume);
}
@@ -626,6 +679,10 @@
obib.setBucketLayout(
BucketLayout.fromProto(bucketInfo.getBucketLayout()));
}
+ if (bucketInfo.hasDefaultReplicationConfig()) {
+ obib.setDefaultReplicationConfig(
+ OMPBHelper.convert(bucketInfo.getDefaultReplicationConfig()));
+ }
if (bucketInfo.hasObjectID()) {
obib.setObjectID(bucketInfo.getObjectID());
}
@@ -668,6 +725,7 @@
", quotaInBytes='" + quotaInBytes + "'" +
", quotaInNamespace='" + quotaInNamespace + "'" +
", bucketLayout='" + bucketLayout + '\'' +
+ ", defaultReplicationConfig='" + defaultReplicationConfig + '\'' +
sourceInfo +
'}';
}
@@ -696,7 +754,8 @@
Objects.equals(sourceBucket, that.sourceBucket) &&
Objects.equals(metadata, that.metadata) &&
Objects.equals(bekInfo, that.bekInfo) &&
- Objects.equals(owner, that.owner);
+ Objects.equals(owner, that.owner) &&
+ Objects.equals(defaultReplicationConfig, this.defaultReplicationConfig);
}
@Override
@@ -725,6 +784,7 @@
", quotaInNamespace=" + quotaInNamespace +
", bucketLayout=" + bucketLayout +
", owner=" + owner +
+ ", defaultReplicationConfig=" + defaultReplicationConfig +
'}';
}
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
index c3b8620..4b27276 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
@@ -29,6 +29,7 @@
import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.client.ContainerBlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConsts;
@@ -132,6 +133,10 @@
return dataSize;
}
+ public long getReplicatedSize() {
+ return QuotaUtil.getReplicatedSize(getDataSize(), replicationConfig);
+ }
+
public void setDataSize(long size) {
this.dataSize = size;
}
@@ -346,6 +351,10 @@
this.parentObjectID = parentObjectID;
}
+ public void setReplicationConfig(ReplicationConfig repConfig) {
+ this.replicationConfig = repConfig;
+ }
+
/**
* Builder of OmKeyInfo.
*/
@@ -552,9 +561,14 @@
.setVolumeName(volumeName)
.setBucketName(bucketName)
.setDataSize(dataSize)
- .setType(replicationConfig.getReplicationType())
- .setFactor(ReplicationConfig.getLegacyFactor(replicationConfig))
- .setLatestVersion(latestVersion)
+ .setType(replicationConfig.getReplicationType());
+ if (replicationConfig instanceof ECReplicationConfig) {
+ kb.setEcReplicationConfig(
+ ((ECReplicationConfig) replicationConfig).toProto());
+ } else {
+ kb.setFactor(ReplicationConfig.getLegacyFactor(replicationConfig));
+ }
+ kb.setLatestVersion(latestVersion)
.addAllKeyLocationList(keyLocations)
.setCreationTime(creationTime)
.setModificationTime(modificationTime)
@@ -594,7 +608,8 @@
.setCreationTime(keyInfo.getCreationTime())
.setModificationTime(keyInfo.getModificationTime())
.setReplicationConfig(ReplicationConfig
- .fromProtoTypeAndFactor(keyInfo.getType(), keyInfo.getFactor()))
+ .fromProto(keyInfo.getType(), keyInfo.getFactor(),
+ keyInfo.getEcReplicationConfig()))
.addAllMetadata(KeyValueUtil.getFromProtobuf(keyInfo.getMetadataList()))
.setFileEncryptionInfo(keyInfo.hasFileEncryptionInfo() ?
OMPBHelper.convert(keyInfo.getFileEncryptionInfo()) : null)
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java
index 41779ab..86da3eb 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java
@@ -16,6 +16,7 @@
*/
package org.apache.hadoop.ozone.om.helpers;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartKeyInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
@@ -200,10 +201,11 @@
multipartKeyInfo.getPartKeyInfoListList().forEach(partKeyInfo ->
list.put(partKeyInfo.getPartNumber(), partKeyInfo));
- final ReplicationConfig replicationConfig = ReplicationConfig
- .fromProtoTypeAndFactor(
- multipartKeyInfo.getType(),
- multipartKeyInfo.getFactor());
+ final ReplicationConfig replicationConfig = ReplicationConfig.fromProto(
+ multipartKeyInfo.getType(),
+ multipartKeyInfo.getFactor(),
+ multipartKeyInfo.getEcReplicationConfig()
+ );
return new OmMultipartKeyInfo(multipartKeyInfo.getUploadID(),
multipartKeyInfo.getCreationTime(), replicationConfig,
@@ -220,10 +222,17 @@
.setUploadID(uploadID)
.setCreationTime(creationTime)
.setType(replicationConfig.getReplicationType())
- .setFactor(ReplicationConfig.getLegacyFactor(replicationConfig))
.setObjectID(objectID)
.setUpdateID(updateID)
.setParentID(parentID);
+
+ if (replicationConfig instanceof ECReplicationConfig) {
+ ECReplicationConfig ecConf = (ECReplicationConfig) replicationConfig;
+ builder.setEcReplicationConfig(ecConf.toProto());
+ } else {
+ builder.setFactor(ReplicationConfig.getLegacyFactor(replicationConfig));
+ }
+
partKeyInfoList.forEach((key, value) -> builder.addPartKeyInfoList(value));
return builder.build();
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/QuotaUtil.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/QuotaUtil.java
new file mode 100644
index 0000000..09e0440
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/QuotaUtil.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.helpers;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
+
+/**
+ * Helper class to calculate quota related usage.
+ */
+public final class QuotaUtil {
+
+ private QuotaUtil() {
+ };
+
+ /**
+ * From the used space and replicationConfig, calculate the expected
+ * replicated size of the data.
+ * @param dataSize The number of bytes of data stored
+ * @param repConfig The replicationConfig used to store the data
+ * @return Number of bytes required to store the dataSize with replication
+ */
+ public static long getReplicatedSize(
+ long dataSize, ReplicationConfig repConfig) {
+ if (repConfig.getReplicationType() == RATIS) {
+ return dataSize * ((RatisReplicationConfig)repConfig)
+ .getReplicationFactor().getNumber();
+ } else if (repConfig.getReplicationType() == EC) {
+ ECReplicationConfig rc = (ECReplicationConfig)repConfig;
+ int dataStripeSize = rc.getData() * rc.getEcChunkSize();
+ long fullStripes = dataSize / dataStripeSize;
+ long partialFirstChunk =
+ Math.min(rc.getEcChunkSize(), dataSize % dataStripeSize);
+ long replicationOverhead =
+ fullStripes * rc.getParity() * rc.getEcChunkSize()
+ + partialFirstChunk * rc.getParity();
+ return dataSize + replicationOverhead;
+ } else {
+ return dataSize;
+ }
+ }
+
+}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
index 29d5a09..10ca155 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
@@ -25,6 +25,7 @@
import java.util.stream.Collectors;
import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos
.UpgradeFinalizationStatus;
@@ -220,7 +221,6 @@
* @param cmdType type of the request
*/
private OMRequest.Builder createOMRequest(Type cmdType) {
-
return OMRequest.newBuilder()
.setCmdType(cmdType)
.setVersion(ClientVersion.CURRENT_VERSION)
@@ -618,8 +618,13 @@
}
if (args.getReplicationConfig() != null) {
- keyArgs.setFactor(
- ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+ if (args.getReplicationConfig() instanceof ECReplicationConfig) {
+ keyArgs.setEcReplicationConfig(
+ ((ECReplicationConfig) args.getReplicationConfig()).toProto());
+ } else {
+ keyArgs.setFactor(
+ ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+ }
keyArgs.setType(args.getReplicationConfig().getReplicationType());
}
@@ -676,8 +681,13 @@
.setDataSize(args.getDataSize());
if (args.getReplicationConfig() != null) {
- keyArgs.setFactor(
- ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+ if (args.getReplicationConfig() instanceof ECReplicationConfig) {
+ keyArgs.setEcReplicationConfig(
+ ((ECReplicationConfig) args.getReplicationConfig()).toProto());
+ } else {
+ keyArgs.setFactor(
+ ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+ }
keyArgs.setType(args.getReplicationConfig().getReplicationType());
}
@@ -711,8 +721,13 @@
.collect(Collectors.toList()));
if (args.getReplicationConfig() != null) {
- keyArgsBuilder.setFactor(
- ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+ if (args.getReplicationConfig() instanceof ECReplicationConfig) {
+ keyArgsBuilder.setEcReplicationConfig(
+ ((ECReplicationConfig) args.getReplicationConfig()).toProto());
+ } else {
+ keyArgsBuilder.setFactor(
+ ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+ }
keyArgsBuilder.setType(args.getReplicationConfig().getReplicationType());
}
@@ -948,9 +963,13 @@
OzoneAcl.toProtobuf(a)).collect(Collectors.toList()));
if (omKeyArgs.getReplicationConfig() != null) {
- keyArgs.setFactor(
- ReplicationConfig
- .getLegacyFactor(omKeyArgs.getReplicationConfig()));
+ if (omKeyArgs.getReplicationConfig() instanceof ECReplicationConfig) {
+ keyArgs.setEcReplicationConfig(
+ ((ECReplicationConfig) omKeyArgs.getReplicationConfig()).toProto());
+ } else {
+ keyArgs.setFactor(ReplicationConfig
+ .getLegacyFactor(omKeyArgs.getReplicationConfig()));
+ }
keyArgs.setType(omKeyArgs.getReplicationConfig().getReplicationType());
}
@@ -1085,8 +1104,9 @@
OmMultipartUploadListParts omMultipartUploadListParts =
new OmMultipartUploadListParts(
- ReplicationConfig.fromProtoTypeAndFactor(response.getType(),
- response.getFactor()),
+ ReplicationConfig.fromProto(
+ response.getType(), response.getFactor(),
+ response.getEcReplicationConfig()),
response.getNextPartNumberMarker(), response.getIsTruncated());
omMultipartUploadListParts.addProtoPartList(response.getPartsListList());
@@ -1121,8 +1141,8 @@
proto.getKeyName(),
proto.getUploadId(),
Instant.ofEpochMilli(proto.getCreationTime()),
- ReplicationConfig.fromProtoTypeAndFactor(proto.getType(),
- proto.getFactor())
+ ReplicationConfig.fromProto(proto.getType(), proto.getFactor(),
+ proto.getEcReplicationConfig())
))
.collect(Collectors.toList());
@@ -1532,9 +1552,13 @@
.addAllAcls(args.getAcls().stream().map(a ->
OzoneAcl.toProtobuf(a)).collect(Collectors.toList()));
if (args.getReplicationConfig() != null) {
- keyArgsBuilder.setFactor(
- ReplicationConfig
- .getLegacyFactor(args.getReplicationConfig()));
+ if (args.getReplicationConfig() instanceof ECReplicationConfig) {
+ keyArgsBuilder.setEcReplicationConfig(
+ ((ECReplicationConfig) args.getReplicationConfig()).toProto());
+ } else {
+ keyArgsBuilder.setFactor(
+ ReplicationConfig.getLegacyFactor(args.getReplicationConfig()));
+ }
keyArgsBuilder.setType(args.getReplicationConfig().getReplicationType());
}
CreateFileRequest createFileRequest = CreateFileRequest.newBuilder()
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java
index 810111d..a7f1307 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java
@@ -21,6 +21,11 @@
import org.apache.hadoop.crypto.CipherSuite;
import org.apache.hadoop.crypto.CryptoProtocolVersion;
import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
@@ -142,6 +147,55 @@
ezKeyVersionName);
}
+ public static DefaultReplicationConfig convert(
+ HddsProtos.DefaultReplicationConfig defaultReplicationConfig) {
+ if (defaultReplicationConfig == null) {
+ throw new IllegalArgumentException(
+ "Invalid argument: default replication config" + " is null");
+ }
+
+ final ReplicationType type =
+ ReplicationType.fromProto(defaultReplicationConfig.getType());
+ DefaultReplicationConfig defaultReplicationConfigObj = null;
+ switch (type) {
+ case EC:
+ defaultReplicationConfigObj = new DefaultReplicationConfig(type,
+ new ECReplicationConfig(
+ defaultReplicationConfig.getEcReplicationConfig()));
+ break;
+ default:
+ final ReplicationFactor factor =
+ ReplicationFactor.fromProto(defaultReplicationConfig.getFactor());
+ defaultReplicationConfigObj = new DefaultReplicationConfig(type, factor);
+ }
+ return defaultReplicationConfigObj;
+ }
+
+ public static HddsProtos.DefaultReplicationConfig convert(
+ DefaultReplicationConfig defaultReplicationConfig) {
+ if (defaultReplicationConfig == null) {
+ throw new IllegalArgumentException(
+ "Invalid argument: default replication config" + " is null");
+ }
+
+ final HddsProtos.DefaultReplicationConfig.Builder builder =
+ HddsProtos.DefaultReplicationConfig.newBuilder();
+ builder.setType(
+ ReplicationType.toProto(defaultReplicationConfig.getType()));
+
+ if (defaultReplicationConfig.getFactor() != null) {
+ builder.setFactor(ReplicationFactor
+ .toProto(defaultReplicationConfig.getFactor()));
+ }
+
+ if (defaultReplicationConfig.getEcReplicationConfig() != null) {
+ builder.setEcReplicationConfig(
+ defaultReplicationConfig.getEcReplicationConfig().toProto());
+ }
+
+ return builder.build();
+ }
+
public static CipherSuite convert(CipherSuiteProto proto) {
switch (proto) {
case AES_CTR_NOPADDING:
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java
index 650fc91..aa64fd2 100644
--- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java
@@ -17,9 +17,13 @@
*/
package org.apache.hadoop.ozone.om.helpers;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.junit.Assert;
import org.junit.Test;
@@ -111,4 +115,56 @@
Assert.assertEquals((int) 1, cloneBucketInfo.getAcls().size());
}
+
+ @Test
+ public void getProtobufMessageEC() {
+ OmBucketInfo omBucketInfo =
+ OmBucketInfo.newBuilder().setBucketName("bucket").setVolumeName("vol1")
+ .setCreationTime(Time.now()).setIsVersionEnabled(false)
+ .setStorageType(StorageType.ARCHIVE).setAcls(Collections
+ .singletonList(new OzoneAcl(IAccessAuthorizer.ACLIdentityType.USER,
+ "defaultUser", IAccessAuthorizer.ACLType.WRITE_ACL,
+ OzoneAcl.AclScope.ACCESS))).build();
+ OzoneManagerProtocolProtos.BucketInfo protobuf = omBucketInfo.getProtobuf();
+ // No EC Config
+ Assert.assertFalse(protobuf.hasDefaultReplicationConfig());
+
+ // Reconstruct object from Proto
+ OmBucketInfo recovered = OmBucketInfo.getFromProtobuf(protobuf);
+ Assert.assertNull(recovered.getDefaultReplicationConfig());
+
+ // EC Config
+ omBucketInfo = OmBucketInfo.newBuilder()
+ .setBucketName("bucket")
+ .setVolumeName("vol1")
+ .setCreationTime(Time.now())
+ .setIsVersionEnabled(false)
+ .setStorageType(StorageType.ARCHIVE)
+ .setAcls(Collections.singletonList(new OzoneAcl(
+ IAccessAuthorizer.ACLIdentityType.USER,
+ "defaultUser", IAccessAuthorizer.ACLType.WRITE_ACL,
+ OzoneAcl.AclScope.ACCESS)))
+ .setDefaultReplicationConfig(
+ new DefaultReplicationConfig(ReplicationType.EC,
+ new ECReplicationConfig(3, 2))).build();
+ protobuf = omBucketInfo.getProtobuf();
+
+ Assert.assertTrue(protobuf.hasDefaultReplicationConfig());
+ Assert.assertEquals(3,
+ protobuf.getDefaultReplicationConfig().getEcReplicationConfig()
+ .getData());
+ Assert.assertEquals(2,
+ protobuf.getDefaultReplicationConfig().getEcReplicationConfig()
+ .getParity());
+
+ // Reconstruct object from Proto
+ recovered = OmBucketInfo.getFromProtobuf(protobuf);
+ Assert.assertEquals(ReplicationType.EC,
+ recovered.getDefaultReplicationConfig().getType());
+ ECReplicationConfig config =
+ recovered.getDefaultReplicationConfig().getEcReplicationConfig();
+ Assert.assertNotNull(config);
+ Assert.assertEquals(3, config.getData());
+ Assert.assertEquals(2, config.getParity());
+ }
}
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyInfo.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyInfo.java
index 3b04ed4..a6a0ec9 100644
--- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyInfo.java
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyInfo.java
@@ -18,7 +18,9 @@
package org.apache.hadoop.ozone.om.helpers;
import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -26,6 +28,7 @@
import org.apache.hadoop.ozone.ClientVersion;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo.Builder;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.util.Time;
import org.junit.Assert;
@@ -36,6 +39,9 @@
import java.util.Collections;
import java.util.List;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
import static org.apache.hadoop.ozone.OzoneAcl.AclScope.ACCESS;
/**
@@ -45,23 +51,67 @@
@Test
public void protobufConversion() {
- OmKeyInfo key = new Builder()
+ OmKeyInfo key = createOmKeyInfo(
+ RatisReplicationConfig.getInstance(ReplicationFactor.THREE));
+
+ OmKeyInfo keyAfterSerialization = OmKeyInfo.getFromProtobuf(
+ key.getProtobuf(ClientVersion.CURRENT_VERSION));
+
+ Assert.assertEquals(key, keyAfterSerialization);
+ }
+
+ @Test
+ public void getProtobufMessageEC() {
+ OmKeyInfo key = createOmKeyInfo(
+ RatisReplicationConfig.getInstance(ReplicationFactor.THREE));
+ OzoneManagerProtocolProtos.KeyInfo omKeyProto =
+ key.getProtobuf(ClientVersion.CURRENT_VERSION);
+
+ // No EC Config
+ Assert.assertFalse(omKeyProto.hasEcReplicationConfig());
+ Assert.assertEquals(THREE, omKeyProto.getFactor());
+ Assert.assertEquals(RATIS, omKeyProto.getType());
+
+ // Reconstruct object from Proto
+ OmKeyInfo recovered = OmKeyInfo.getFromProtobuf(omKeyProto);
+ Assert.assertEquals(RATIS,
+ recovered.getReplicationConfig().getReplicationType());
+ Assert.assertTrue(
+ recovered.getReplicationConfig() instanceof RatisReplicationConfig);
+
+ // EC Config
+ key = createOmKeyInfo(new ECReplicationConfig(3, 2));
+ omKeyProto = key.getProtobuf(ClientVersion.CURRENT_VERSION);
+
+ Assert.assertEquals(3, omKeyProto.getEcReplicationConfig().getData());
+ Assert.assertEquals(2, omKeyProto.getEcReplicationConfig().getParity());
+ Assert.assertFalse(omKeyProto.hasFactor());
+ Assert.assertEquals(EC, omKeyProto.getType());
+
+ // Reconstruct object from Proto
+ recovered = OmKeyInfo.getFromProtobuf(omKeyProto);
+ Assert.assertEquals(EC,
+ recovered.getReplicationConfig().getReplicationType());
+ Assert.assertTrue(
+ recovered.getReplicationConfig() instanceof ECReplicationConfig);
+ ECReplicationConfig config =
+ (ECReplicationConfig) recovered.getReplicationConfig();
+ Assert.assertEquals(3, config.getData());
+ Assert.assertEquals(2, config.getParity());
+ }
+
+ private OmKeyInfo createOmKeyInfo(ReplicationConfig replicationConfig) {
+ return new Builder()
.setKeyName("key1")
.setBucketName("bucket")
.setVolumeName("vol1")
.setCreationTime(123L)
.setModificationTime(123L)
.setDataSize(123L)
- .setReplicationConfig(
- RatisReplicationConfig.getInstance(ReplicationFactor.THREE))
+ .setReplicationConfig(replicationConfig)
.addMetadata("key1", "value1")
.addMetadata("key2", "value2")
.build();
-
- OmKeyInfo keyAfterSerialization = OmKeyInfo.getFromProtobuf(
- key.getProtobuf(ClientVersion.CURRENT_VERSION));
-
- Assert.assertEquals(key, keyAfterSerialization);
}
@Test
@@ -161,4 +211,4 @@
.setPipeline(pipeline)
.build();
}
-}
\ No newline at end of file
+}
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmMultipartKeyInfo.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmMultipartKeyInfo.java
index ee850bc..c6f8818 100644
--- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmMultipartKeyInfo.java
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmMultipartKeyInfo.java
@@ -18,15 +18,23 @@
package org.apache.hadoop.ozone.om.helpers;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
import org.apache.hadoop.util.Time;
-import org.junit.Assert;
import org.junit.Test;
import java.util.UUID;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
/**
* Class to test OmMultipartKeyInfo.
@@ -34,33 +42,97 @@
public class TestOmMultipartKeyInfo {
@Test
- public void testCopyObject() {
- OmMultipartKeyInfo omMultipartKeyInfo = new OmMultipartKeyInfo.Builder()
- .setUploadID(UUID.randomUUID().toString())
- .setCreationTime(Time.now())
- .setReplicationConfig(RatisReplicationConfig
- .getInstance(HddsProtos.ReplicationFactor.THREE))
+ public void copyObject() {
+ for (ReplicationConfig param : replicationConfigs().collect(toList())) {
+ testCopyObject(param);
+ }
+ }
+
+ //@ParameterizedTest
+ //@MethodSource("replicationConfigs")
+ private void testCopyObject(ReplicationConfig replicationConfig) {
+ // GIVEN
+ OmMultipartKeyInfo subject = createSubject()
+ .setReplicationConfig(replicationConfig)
.build();
- OmMultipartKeyInfo cloneMultipartKeyInfo = omMultipartKeyInfo.copyObject();
+ // WHEN
+ OmMultipartKeyInfo copy = subject.copyObject();
- Assert.assertEquals(cloneMultipartKeyInfo, omMultipartKeyInfo);
+ // THEN
+ assertNotSame(subject, copy);
+ assertEquals(subject, copy);
+ assertEquals(replicationConfig, copy.getReplicationConfig());
+ }
- // Just setting dummy values for this test.
- omMultipartKeyInfo.addPartKeyInfo(1,
- PartKeyInfo.newBuilder().setPartNumber(1).setPartName("/path")
- .setPartKeyInfo(KeyInfo.newBuilder()
+ @Test
+ public void protoConversion() {
+ for (ReplicationConfig param : replicationConfigs().collect(toList())) {
+ protoConversion(param);
+ }
+ }
+
+ //@ParameterizedTest
+ //@MethodSource("replicationConfigs")
+ private void protoConversion(ReplicationConfig replicationConfig) {
+ // GIVEN
+ OmMultipartKeyInfo subject = createSubject()
+ .setReplicationConfig(replicationConfig)
+ .build();
+
+ // WHEN
+ OzoneManagerProtocolProtos.MultipartKeyInfo proto = subject.getProto();
+ OmMultipartKeyInfo fromProto = OmMultipartKeyInfo.getFromProto(proto);
+
+ // THEN
+ assertEquals(subject, fromProto);
+ assertEquals(replicationConfig, fromProto.getReplicationConfig());
+ }
+
+ private static Stream<ReplicationConfig> replicationConfigs() {
+ return Stream.of(
+ StandaloneReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE),
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE),
+ new ECReplicationConfig(3, 2)
+ );
+ }
+
+ @Test
+ public void distinctListOfParts() {
+ // GIVEN
+ OmMultipartKeyInfo subject = createSubject().build();
+ OmMultipartKeyInfo copy = subject.copyObject();
+
+ // WHEN
+ subject.addPartKeyInfo(1, createPart(createKeyInfo()).build());
+
+ // THEN
+ assertEquals(0, copy.getPartKeyInfoMap().size());
+ assertEquals(1, subject.getPartKeyInfoMap().size());
+ }
+
+ private static OmMultipartKeyInfo.Builder createSubject() {
+ return new OmMultipartKeyInfo.Builder()
+ .setUploadID(UUID.randomUUID().toString())
+ .setCreationTime(Time.now());
+ }
+
+ private static PartKeyInfo.Builder createPart(KeyInfo.Builder partKeyInfo) {
+ return PartKeyInfo.newBuilder()
+ .setPartNumber(1)
+ .setPartName("/path")
+ .setPartKeyInfo(partKeyInfo);
+ }
+
+ private static KeyInfo.Builder createKeyInfo() {
+ return KeyInfo.newBuilder()
.setVolumeName(UUID.randomUUID().toString())
.setBucketName(UUID.randomUUID().toString())
.setKeyName(UUID.randomUUID().toString())
- .setDataSize(100L) // Just set dummy size for testing
+ .setDataSize(100L)
.setCreationTime(Time.now())
.setModificationTime(Time.now())
- .setType(HddsProtos.ReplicationType.RATIS)
- .setFactor(HddsProtos.ReplicationFactor.ONE).build()).build());
-
- Assert.assertEquals(0, cloneMultipartKeyInfo.getPartKeyInfoMap().size());
- Assert.assertEquals(1, omMultipartKeyInfo.getPartKeyInfoMap().size());
-
+ .setType(HddsProtos.ReplicationType.STAND_ALONE);
}
}
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestQuotaUtil.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestQuotaUtil.java
new file mode 100644
index 0000000..558676d
--- /dev/null
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestQuotaUtil.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.helpers;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec.RS;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+
+/**
+ * Tests for the QuotaUtil class.
+ */
+public class TestQuotaUtil {
+
+ private static final int ONE_MB = 1024 * 1024;
+
+ @Test
+ public void testRatisThreeReplication() {
+ ReplicationConfig repConfig = RatisReplicationConfig.getInstance(THREE);
+ long replicatedSize =
+ QuotaUtil.getReplicatedSize(123 * ONE_MB, repConfig);
+ Assert.assertEquals(123 * ONE_MB * 3, replicatedSize);
+ }
+
+ @Test
+ public void testRatisOneReplication() {
+ ReplicationConfig repConfig = RatisReplicationConfig.getInstance(ONE);
+ long replicatedSize =
+ QuotaUtil.getReplicatedSize(123 * ONE_MB, repConfig);
+ Assert.assertEquals(123 * ONE_MB, replicatedSize);
+ }
+
+ @Test
+ public void testECFullStripeReplication() {
+ ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS, ONE_MB);
+ long dataSize = ONE_MB * 3 * 123; // 123 full stripe
+ long replicatedSize = QuotaUtil.getReplicatedSize(dataSize, repConfig);
+ Assert.assertEquals(dataSize + 123 * ONE_MB * 2, replicatedSize);
+ }
+
+ @Test
+ public void testECPartialStripeIntoFirstChunk() {
+ ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS, ONE_MB);
+ long dataSize = ONE_MB * 3 * 123 + 10; // 123 full stripes, plus 10 bytes
+ long replicatedSize = QuotaUtil.getReplicatedSize(dataSize, repConfig);
+ // Expected is 123 parity stripes, plus another 10 bytes in each parity
+ Assert.assertEquals(dataSize + 123 * ONE_MB * 2 + 10 * 2, replicatedSize);
+ }
+
+ @Test
+ public void testECPartialStripeBeyondFirstChunk() {
+ ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS, ONE_MB);
+ // 123 full stripes, plus 1MB+10 bytes
+ long dataSize = ONE_MB * 3 * 123 + ONE_MB + 10;
+ long replicatedSize = QuotaUtil.getReplicatedSize(dataSize, repConfig);
+ // Expected is 123 parity stripes, plus another 1MB in each parity
+ Assert.assertEquals(
+ dataSize + 123 * ONE_MB * 2 + ONE_MB * 2, replicatedSize);
+ }
+
+ @Test
+ public void testECPartialSingleStripeFirstChunk() {
+ ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS, ONE_MB);
+ long dataSize = 10;
+ long replicatedSize = QuotaUtil.getReplicatedSize(dataSize, repConfig);
+ // Expected is 123 parity stripes, plus another 1MB in each parity
+ Assert.assertEquals(dataSize + 10 * 2, replicatedSize);
+ }
+
+ @Test
+ public void testECPartialSingleBeyondFirstChunk() {
+ ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS, ONE_MB);
+ long dataSize = 2 * ONE_MB + 10;
+ long replicatedSize = QuotaUtil.getReplicatedSize(dataSize, repConfig);
+ // Expected is 123 parity stripes, plus another 1MB in each parity
+ Assert.assertEquals(dataSize + ONE_MB * 2, replicatedSize);
+ }
+
+}
diff --git a/hadoop-ozone/dist/src/main/compose/ozone/test.sh b/hadoop-ozone/dist/src/main/compose/ozone/test.sh
index e494470..ac3b273 100755
--- a/hadoop-ozone/dist/src/main/compose/ozone/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/ozone/test.sh
@@ -39,7 +39,7 @@
execute_robot_test scm security/ozone-secure-token.robot
-for bucket in link generated; do
+for bucket in erasure link generated; do
execute_robot_test scm -v BUCKET:${bucket} -N s3-${bucket} s3
done
@@ -57,6 +57,8 @@
execute_robot_test scm -v SCHEME:ofs -v BUCKET_TYPE:link -N ozonefs-fso-ofs-link ozonefs/ozonefs.robot
execute_robot_test scm -v SCHEME:o3fs -v BUCKET_TYPE:bucket -N ozonefs-fso-o3fs-bucket ozonefs/ozonefs.robot
+execute_robot_test scm ec/basic.robot
+
stop_docker_env
generate_report
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/docker-compose.yaml
index 24ff9d7..7f52d73 100644
--- a/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/docker-compose.yaml
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/docker-compose.yaml
@@ -124,6 +124,24 @@
- ${OZONE_VOLUME}/dn3:/data
- *ozone-dir
- *transformation
+ dn4:
+ <<: *datanode
+ networks:
+ net:
+ ipv4_address: 10.9.0.18
+ volumes:
+ - ${OZONE_VOLUME}/dn4:/data
+ - *ozone-dir
+ - *transformation
+ dn5:
+ <<: *datanode
+ networks:
+ net:
+ ipv4_address: 10.9.0.19
+ volumes:
+ - ${OZONE_VOLUME}/dn5:/data
+ - *ozone-dir
+ - *transformation
recon:
command: ["ozone","recon"]
<<: *common-config
@@ -131,7 +149,7 @@
<<: *replication
networks:
net:
- ipv4_address: 10.9.0.18
+ ipv4_address: 10.9.0.20
ports:
- 9888:9888
volumes:
@@ -145,7 +163,7 @@
<<: *replication
networks:
net:
- ipv4_address: 10.9.0.19
+ ipv4_address: 10.9.0.21
ports:
- 9878:9878
volumes:
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/load.sh b/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/load.sh
index 902eb12..43c6688 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/load.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/compose/ha/load.sh
@@ -25,4 +25,4 @@
export COMPOSE_FILE="$TEST_DIR/compose/ha/docker-compose.yaml"
export OM_SERVICE_ID=omservice
-create_data_dirs "${OZONE_VOLUME}"/{om1,om2,om3,dn1,dn2,dn3,recon,s3g,scm}
+create_data_dirs "${OZONE_VOLUME}"/{om1,om2,om3,dn1,dn2,dn3,dn4,dn5,recon,s3g,scm}
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/docker-compose.yaml
index 4b1d64e..14617cc 100644
--- a/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/docker-compose.yaml
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/docker-compose.yaml
@@ -101,6 +101,24 @@
- ${OZONE_VOLUME}/dn3:/data
- *ozone-dir
- *transformation
+ dn4:
+ <<: *datanode
+ networks:
+ net:
+ ipv4_address: 10.9.0.16
+ volumes:
+ - ${OZONE_VOLUME}/dn4:/data
+ - *ozone-dir
+ - *transformation
+ dn5:
+ <<: *datanode
+ networks:
+ net:
+ ipv4_address: 10.9.0.17
+ volumes:
+ - ${OZONE_VOLUME}/dn5:/data
+ - *ozone-dir
+ - *transformation
recon:
command: ["ozone","recon"]
<<: *common-config
@@ -108,7 +126,7 @@
<<: *replication
networks:
net:
- ipv4_address: 10.9.0.16
+ ipv4_address: 10.9.0.18
ports:
- 9888:9888
volumes:
@@ -122,7 +140,7 @@
<<: *replication
networks:
net:
- ipv4_address: 10.9.0.17
+ ipv4_address: 10.9.0.19
ports:
- 9878:9878
volumes:
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/load.sh b/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/load.sh
index a8980e7..0e8fc05 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/load.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/compose/non-ha/load.sh
@@ -24,4 +24,4 @@
source "$TEST_DIR/testlib.sh"
export COMPOSE_FILE="$TEST_DIR/compose/non-ha/docker-compose.yaml"
-create_data_dirs "${OZONE_VOLUME}"/{om,dn1,dn2,dn3,recon,s3g,scm}
+create_data_dirs "${OZONE_VOLUME}"/{om,dn1,dn2,dn3,dn4,dn5,recon,s3g,scm}
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/test.sh b/hadoop-ozone/dist/src/main/compose/upgrade/test.sh
index 687f9bd..35358e3 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/test.sh
@@ -16,7 +16,7 @@
# limitations under the License.
# Version that will be run using the local build.
-: "${OZONE_CURRENT_VERSION:=1.2.0}"
+: "${OZONE_CURRENT_VERSION:=1.3.0}"
export OZONE_CURRENT_VERSION
TEST_DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null && pwd )
@@ -31,8 +31,10 @@
RESULT_DIR="$ALL_RESULT_DIR" create_results_dir
-# Upgrade tests to be run.
-run_test non-rolling-upgrade 1.1.0 1.2.0
+# Upgrade tests to be run. In CI we want to run just one set, but for a release
+# we might advise the release manager to run the full matrix.
+#run_test non-rolling-upgrade 1.1.0 1.3.0
+run_test non-rolling-upgrade 1.2.1 1.3.0
generate_report "upgrade" "$ALL_RESULT_DIR"
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/testlib.sh b/hadoop-ozone/dist/src/main/compose/upgrade/testlib.sh
index 1b6b05d..b3fa1df 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/testlib.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/testlib.sh
@@ -150,3 +150,11 @@
check_scm_mlv() {
check_mlv "$1" /data/metadata/scm/current/VERSION "$2"
}
+
+check_ec_is_disabled() {
+ execute_robot_test scm --include pre-finalized-ec-tests ec/upgrade-ec-check.robot
+}
+
+check_ec_is_enabled() {
+ execute_robot_test scm --include post-finalized-ec-tests ec/upgrade-ec-check.robot
+}
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh
new file mode 100755
index 0000000..f414ddf
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh
@@ -0,0 +1,84 @@
+#!/usr/bin/env bash
+# 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.
+
+source "$TEST_DIR"/testlib.sh
+
+# Helper function, not a callback.
+_check_hdds_mlvs() {
+ mlv="$1"
+ check_scm_mlv scm "$mlv"
+ check_dn_mlv dn1 "$mlv"
+ check_dn_mlv dn2 "$mlv"
+ check_dn_mlv dn3 "$mlv"
+}
+
+# Helper function, not a callback.
+_check_om_mlvs() {
+ mlv="$1"
+ check_om_mlv om1 "$mlv"
+ check_om_mlv om2 "$mlv"
+ check_om_mlv om3 "$mlv"
+}
+
+setup() {
+ export OZONE_OM_PREPARE='true'
+}
+
+with_old_version() {
+ generate old1
+ validate old1
+}
+
+with_new_version_pre_finalized() {
+ _check_hdds_mlvs 2
+ _check_om_mlvs 0
+
+ validate old1
+# HDDS-6261: overwrite the same keys intentionally
+ generate old1 --exclude create-volume-and-bucket
+
+ generate new1
+ validate new1
+
+ check_ec_is_disabled
+}
+
+with_old_version_downgraded() {
+ validate old1
+ validate new1
+
+ generate old2
+ validate old2
+
+ # HDDS-6261: overwrite the same keys again to trigger the precondition check
+ # that exists <= 1.1.0 OM
+ generate old1 --exclude create-volume-and-bucket
+}
+
+with_new_version_finalized() {
+ _check_hdds_mlvs 3
+ _check_om_mlvs 1
+
+ validate old1
+ validate new1
+ validate old2
+
+ generate new2
+ validate new2
+
+ check_ec_is_enabled
+}
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/driver.sh b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/driver.sh
index e59bec5..7b95238 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/driver.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/driver.sh
@@ -72,7 +72,6 @@
OUTPUT_NAME="$OZONE_UPGRADE_TO"-pre-finalized
OZONE_KEEP_RESULTS=true start_docker_env
callback with_new_version_pre_finalized
-
prepare_oms
stop_docker_env
prepare_for_image "$OZONE_UPGRADE_FROM"
diff --git a/hadoop-ozone/dist/src/main/compose/xcompat/docker-config b/hadoop-ozone/dist/src/main/compose/xcompat/docker-config
index 8bbb4d1..29b3696 100644
--- a/hadoop-ozone/dist/src/main/compose/xcompat/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/xcompat/docker-config
@@ -14,6 +14,8 @@
# See the License for the specific language governing permissions and
# limitations under the License.
+CORE-SITE.XML_fs.ofs.impl=org.apache.hadoop.fs.ozone.RootedOzoneFileSystem
+
OZONE-SITE.XML_hdds.datanode.dir=/data/hdds
OZONE-SITE.XML_hdds.scm.safemode.min.datanode=3
OZONE-SITE.XML_ozone.metadata.dirs=/data/metadata
diff --git a/hadoop-ozone/dist/src/main/compose/xcompat/test.sh b/hadoop-ozone/dist/src/main/compose/xcompat/test.sh
index a46590f..f583a2b 100755
--- a/hadoop-ozone/dist/src/main/compose/xcompat/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/xcompat/test.sh
@@ -73,6 +73,45 @@
KEEP_RUNNING=false stop_docker_env
}
+test_ec_cross_compatibility() {
+ echo "Running Erasure Coded storage backward compatibility tests."
+ local cluster_versions_with_ec="1.3.0"
+ local non_ec_client_versions="1.0.0 1.1.0 1.2.1"
+
+ for cluster_version in ${cluster_versions_with_ec}; do
+ export COMPOSE_FILE=new-cluster.yaml:clients.yaml cluster_version=${cluster_version}
+ OZONE_KEEP_RESULTS=true start_docker_env 5
+
+ echo -n "Generating data locally... "
+ dd if=/dev/urandom of=/tmp/1mb bs=1048576 count=1 >/dev/null 2>&1
+ dd if=/dev/urandom of=/tmp/2mb bs=1048576 count=2 >/dev/null 2>&1
+ dd if=/dev/urandom of=/tmp/3mb bs=1048576 count=3 >/dev/null 2>&1
+ echo "done"
+ echo -n "Copy data into client containers... "
+ for container in $(docker ps --format '{{.Names}}' | grep client); do
+ docker cp /tmp/1mb ${container}:/tmp/1mb
+ docker cp /tmp/2mb ${container}:/tmp/2mb
+ docker cp /tmp/3mb ${container}:/tmp/3mb
+ done
+ echo "done"
+ rm -f /tmp/1mb /tmp/2mb /tmp/3mb
+
+
+ local prefix=$(LC_CTYPE=C tr -dc '[:alnum:]' < /dev/urandom | head -c 5 | tr '[:upper:]' '[:lower:]')
+ OZONE_DIR=/opt/hadoop
+ execute_robot_test new_client --include setup-ec-data -N "xcompat-cluster-${cluster_version}-setup-data" -v prefix:"${prefix}" ec/backward-compat.robot
+ OZONE_DIR=/opt/ozone
+
+ for client_version in ${non_ec_client_versions}; do
+ client="old_client_${client_version//./_}"
+ unset OUTPUT_PATH
+ execute_robot_test "${client}" --include test-ec-compat -N "xcompat-cluster-${cluster_version}-client-${client_version}-read-${cluster_version}" -v prefix:"${prefix}" ec/backward-compat.robot
+ done
+
+ KEEP_RUNNING=false stop_docker_env
+ done
+}
+
create_results_dir
# current cluster with various clients
@@ -84,4 +123,6 @@
COMPOSE_FILE=old-cluster.yaml:clients.yaml test_cross_compatibility ${cluster_version}
done
+test_ec_cross_compatibility
+
generate_report
diff --git a/hadoop-ozone/dist/src/main/license/jar-report.txt b/hadoop-ozone/dist/src/main/license/jar-report.txt
index da8350d..3911c4d 100644
--- a/hadoop-ozone/dist/src/main/license/jar-report.txt
+++ b/hadoop-ozone/dist/src/main/license/jar-report.txt
@@ -64,6 +64,7 @@
share/ozone/lib/hdds-config.jar
share/ozone/lib/hdds-container-service.jar
share/ozone/lib/hdds-docs.jar
+share/ozone/lib/hdds-erasurecode.jar
share/ozone/lib/hdds-hadoop-dependency-client.jar
share/ozone/lib/hdds-hadoop-dependency-server.jar
share/ozone/lib/hdds-interface-admin.jar
diff --git a/hadoop-ozone/dist/src/main/org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory b/hadoop-ozone/dist/src/main/org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory
new file mode 100644
index 0000000..23b1a14
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory
@@ -0,0 +1,18 @@
+#
+# Licensed 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.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory
+org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawErasureCoderFactory
+org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory
+org.apache.hadoop.io.erasurecode.rawcoder.RSLegacyRawErasureCoderFactory
+org.apache.hadoop.io.erasurecode.rawcoder.XORRawErasureCoderFactory
diff --git a/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot b/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
index 5752eed..f0d8fa8 100644
--- a/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
@@ -54,11 +54,11 @@
Should Not contain ${output} OPEN
List containers with replication factor ONE
- ${output} = Execute ozone admin container list --factor=ONE
+ ${output} = Execute ozone admin container list -t RATIS -r ONE
Should Not contain ${output} THREE
List containers with replication factor THREE
- ${output} = Execute ozone admin container list --factor=THREE
+ ${output} = Execute ozone admin container list -t RATIS -r THREE
Should Not contain ${output} ONE
Container info
diff --git a/hadoop-ozone/dist/src/main/smoketest/ec/backward-compat.robot b/hadoop-ozone/dist/src/main/smoketest/ec/backward-compat.robot
new file mode 100644
index 0000000..74ab547
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/ec/backward-compat.robot
@@ -0,0 +1,103 @@
+# 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 Test EC backward compatibility
+Library OperatingSystem
+Resource lib.resource
+
+*** Test Cases ***
+Setup Cluster Data
+ [Tags] setup-ec-data
+ Prepare Data For Xcompat Tests
+
+Test Read Key Compat
+ [Tags] test-ec-compat
+ Key Should Match Local File /${prefix}vol1/${prefix}ratis/${prefix}3mb /tmp/3mb
+ Key Should Match Local File /${prefix}vol1/${prefix}default/${prefix}3mb /tmp/3mb
+
+ ${result} = Execute and checkrc ozone sh key get /${prefix}vol1/${prefix}ec/${prefix}3mb /tmp/${prefix}3mb 255
+ Should Contain ${result} NOT_SUPPORTED_OPERATION
+
+Test Listing Compat
+ [Tags] test-ec-compat
+ ${result} = Execute ozone sh volume list | jq -r '.name'
+ Should contain ${result} ${prefix}vol1
+ ${result} = Execute ozone sh bucket list /${prefix}vol1/ | jq -r '.name'
+ Should contain ${result} ${prefix}default
+ Should contain ${result} ${prefix}ratis
+ Should contain ${result} ${prefix}ec
+ ${result} = Execute ozone sh key list /${prefix}vol1/${prefix}default/ | jq -r '[.name, .replicationType, (.replicationFactor | tostring)] | join (" ")'
+ Should contain ${result} ${prefix}3mb RATIS 3
+ ${result} = Execute ozone sh key list /${prefix}vol1/${prefix}ratis/ | jq -r '[.name, .replicationType, (.replicationFactor | tostring)] | join (" ")'
+ Should contain ${result} ${prefix}3mb RATIS 3
+
+ ${result} = Execute and checkrc ozone sh key list /${prefix}vol1/${prefix}ec/ 255
+ Should contain ${result} NOT_SUPPORTED_OPERATION
+
+Test Info Compat
+ [Tags] test-ec-compat
+ ${result} = Execute ozone sh volume info ${prefix}vol1 | jq -r '.name'
+ Should contain ${result} ${prefix}vol1
+ ${result} = Execute ozone sh bucket info /${prefix}vol1/${prefix}default | jq -r '[.name, .replicationType, .replicationFactor] | join (" ")'
+ Should contain ${result} ${prefix}default # there is no replication config in the old client for bucket info
+ ${result} = Execute ozone sh bucket info /${prefix}vol1/${prefix}ratis | jq -r '[.name, .replicationType, .replicationFactor] | join (" ")'
+ Should contain ${result} ${prefix}ratis # there is no replication config in the old client for bucket info
+ ${result} = Execute ozone sh bucket info /${prefix}vol1/${prefix}ec | jq -r '[.name, .replicationType, .replicationFactor] | join (" ")'
+ Should contain ${result} ${prefix}ec # there is no replication config in the old client for bucket info
+
+Test FS Compat
+ [Tags] test-ec-compat
+ ${result} = Execute ozone fs -ls ofs://om/
+ Should contain ${result} /${prefix}vol1
+ ${result} = Execute ozone fs -ls ofs://om/${prefix}vol1/
+ Should contain ${result} /${prefix}vol1/${prefix}default
+ Should contain ${result} /${prefix}vol1/${prefix}ratis
+ Should contain ${result} /${prefix}vol1/${prefix}ec
+ ${result} = Execute ozone fs -ls ofs://om/${prefix}vol1/${prefix}default/${prefix}3mb
+ Should contain ${result} /${prefix}vol1/${prefix}default/${prefix}3mb
+ ${result} = Execute ozone fs -ls ofs://om/${prefix}vol1/${prefix}ratis/${prefix}3mb
+ Should contain ${result} /${prefix}vol1/${prefix}ratis/${prefix}3mb
+
+ ${result} = Execute and checkrc ozone fs -ls ofs://om/${prefix}vol1/${prefix}ec/ 1
+ Should contain ${result} ls: The list of keys contains keys with Erasure Coded replication set
+ ${result} = Execute and checkrc ozone fs -ls ofs://om/${prefix}vol1/${prefix}ec/${prefix}3mb 1
+ Should contain ${result} : No such file or directory
+ ${result} = Execute and checkrc ozone fs -get ofs://om/${prefix}vol1/${prefix}ec/${prefix}3mb 1
+ Should contain ${result} : No such file or directory
+
+Test FS Client Can Read Own Writes
+ [Tags] test-ec-compat
+ Execute ozone fs -put /tmp/1mb ofs://om/${prefix}vol1/${prefix}default/${prefix}1mb
+ Execute ozone fs -put /tmp/1mb ofs://om/${prefix}vol1/${prefix}ratis/${prefix}1mb
+ Execute ozone fs -put /tmp/1mb ofs://om/${prefix}vol1/${prefix}ec/${prefix}1mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ratis/${prefix}1mb /tmp/1mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ratis/${prefix}1mb /tmp/1mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ratis/${prefix}1mb /tmp/1mb
+ Execute ozone fs -rm -skipTrash ofs://om/${prefix}vol1/${prefix}default/${prefix}1mb
+ Execute ozone fs -rm -skipTrash ofs://om/${prefix}vol1/${prefix}ratis/${prefix}1mb
+ Execute ozone fs -rm -skipTrash ofs://om/${prefix}vol1/${prefix}ec/${prefix}1mb
+
+Test Client Can Read Own Writes
+ [Tags] test-ec-compat
+ Execute ozone sh key put /${prefix}vol1/${prefix}default/${prefix}2mb /tmp/2mb
+ Execute ozone sh key put /${prefix}vol1/${prefix}ratis/${prefix}2mb /tmp/2mb
+ Execute ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}2mb /tmp/2mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ratis/${prefix}2mb /tmp/2mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ratis/${prefix}2mb /tmp/2mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ratis/${prefix}2mb /tmp/2mb
+ Execute ozone sh key delete /${prefix}vol1/${prefix}default/${prefix}2mb
+ Execute ozone sh key delete /${prefix}vol1/${prefix}ratis/${prefix}2mb
+ Execute ozone sh key delete /${prefix}vol1/${prefix}ec/${prefix}2mb
diff --git a/hadoop-ozone/dist/src/main/smoketest/ec/basic.robot b/hadoop-ozone/dist/src/main/smoketest/ec/basic.robot
new file mode 100644
index 0000000..751211d
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/ec/basic.robot
@@ -0,0 +1,64 @@
+# 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 Test EC shell commands
+Library OperatingSystem
+Resource ../commonlib.robot
+Resource ../ozone-lib/shell.robot
+Resource lib.resource
+Suite Setup Prepare For Tests
+
+*** Test Cases ***
+Test Bucket Creation
+ ${result} = Execute ozone sh volume create /${prefix}vol1
+ Should not contain ${result} Failed
+ ${result} = Execute ozone sh bucket create /${prefix}vol1/${prefix}default
+ Should not contain ${result} Failed
+ Verify Bucket Empty Replication Config /${prefix}vol1/${prefix}default
+ ${result} = Execute ozone sh bucket create --replication 3 --type RATIS /${prefix}vol1/${prefix}ratis
+ Should not contain ${result} Failed
+ Verify Bucket Replica Replication Config /${prefix}vol1/${prefix}ratis RATIS THREE
+ ${result} = Execute ozone sh bucket create --replication rs-3-2-1024k --type EC /${prefix}vol1/${prefix}ec
+ Should not contain ${result} Failed
+ Verify Bucket EC Replication Config /${prefix}vol1/${prefix}ec RS 3 2 1048576
+
+Test Key Creation EC Bucket
+ Execute ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}1mb /tmp/1mb
+ Execute ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}2mb /tmp/2mb
+ Execute ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}3mb /tmp/3mb
+ Execute ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}100mb /tmp/100mb
+
+ Key Should Match Local File /${prefix}vol1/${prefix}ec/${prefix}1mb /tmp/1mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ec/${prefix}2mb /tmp/2mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ec/${prefix}3mb /tmp/3mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ec/${prefix}100mb /tmp/100mb
+
+ Verify Key EC Replication Config /${prefix}vol1/${prefix}ec/${prefix}1mb RS 3 2 1048576
+
+Test Key Creation Default Bucket
+ Execute ozone sh key put /${prefix}vol1/${prefix}default/${prefix}1mb /tmp/1mb
+ Key Should Match Local File /${prefix}vol1/${prefix}default/${prefix}1mb /tmp/1mb
+ Verify Key Replica Replication Config /${prefix}vol1/${prefix}default/${prefix}1mb RATIS THREE
+
+Test Ratis Key EC Bucket
+ Execute ozone sh key put --replication=THREE --type=RATIS /${prefix}vol1/${prefix}ec/${prefix}1mbRatis /tmp/1mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ec/${prefix}1mbRatis /tmp/1mb
+ Verify Key Replica Replication Config /${prefix}vol1/${prefix}ec/${prefix}1mbRatis RATIS THREE
+
+Test EC Key Ratis Bucket
+ Execute ozone sh key put --replication=rs-3-2-1024k --type=EC /${prefix}vol1/${prefix}ratis/${prefix}1mbEC /tmp/1mb
+ Key Should Match Local File /${prefix}vol1/${prefix}ratis/${prefix}1mbEC /tmp/1mb
+ Verify Key EC Replication Config /${prefix}vol1/${prefix}ratis/${prefix}1mbEC RS 3 2 1048576
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/smoketest/ec/lib.resource b/hadoop-ozone/dist/src/main/smoketest/ec/lib.resource
new file mode 100644
index 0000000..7c774a2
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/ec/lib.resource
@@ -0,0 +1,42 @@
+# 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 Keywords and other resources used by EC tests
+Library OperatingSystem
+Resource ../commonlib.robot
+Resource ../ozone-lib/shell.robot
+
+*** Variables ***
+${SCM} scm
+
+*** Keywords ***
+Prepare For Tests
+ ${random} Generate Random String 5 [NUMBERS]
+ Set Suite Variable ${prefix} ${random}
+ Execute dd if=/dev/urandom of=/tmp/1mb bs=1048576 count=1
+ Execute dd if=/dev/urandom of=/tmp/2mb bs=1048576 count=2
+ Execute dd if=/dev/urandom of=/tmp/3mb bs=1048576 count=3
+ Execute dd if=/dev/urandom of=/tmp/100mb bs=1048576 count=100
+
+# xcompat/test.sh creates unified test data files in /tmp for client containers
+Prepare Data For Xcompat Tests
+ Execute ozone sh volume create /${prefix}vol1
+ Execute ozone sh bucket create /${prefix}vol1/${prefix}default
+ Execute ozone sh bucket create --replication 3 --type RATIS /${prefix}vol1/${prefix}ratis
+ Execute ozone sh bucket create --replication rs-3-2-1024k --type EC /${prefix}vol1/${prefix}ec
+ Execute ozone sh key put /${prefix}vol1/${prefix}default/${prefix}3mb /tmp/3mb
+ Execute ozone sh key put /${prefix}vol1/${prefix}ratis/${prefix}3mb /tmp/3mb
+ Execute ozone sh key put /${prefix}vol1/${prefix}ec/${prefix}3mb /tmp/3mb
diff --git a/hadoop-ozone/dist/src/main/smoketest/ec/upgrade-ec-check.robot b/hadoop-ozone/dist/src/main/smoketest/ec/upgrade-ec-check.robot
new file mode 100644
index 0000000..dbfd9e8
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/ec/upgrade-ec-check.robot
@@ -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.
+
+*** Settings ***
+Documentation Test EC during upgrade
+Library OperatingSystem
+Resource lib.resource
+
+*** Test Cases ***
+Test EC Prior To Finalization
+ [Tags] pre-finalized-ec-tests
+ Execute ozone sh volume create /ectest
+ ${result} = Execute and checkrc ozone sh bucket create --replication rs-3-2-1024k --type EC /ectest/ectest 255
+ Should contain ${result} NOT_SUPPORTED_OPERATION
+ Execute ozone sh bucket create /ectest/testpropchange
+ ${result} = Execute and checkrc ozone sh bucket set-replication-config -r rs-3-2-1024k -t EC /ectest/testpropchange 255
+ Should contain ${result} NOT_SUPPORTED_OPERATION
+ ${result} = Execute and checkrc ozone sh key put -r rs-3-2-1024k -t EC /ectest/testpropchange/core-site.xml /etc/hadoop/core-site.xml 255
+ Should contain ${result} NOT_SUPPORTED_OPERATION
+
+
+
+Test EC After Finalization
+ [Tags] post-finalized-ec-tests
+ Execute ozone sh volume create /ectest-new
+ Execute ozone sh bucket create --replication rs-3-2-1024k --type EC /ectest-new/ectest
+ Verify Bucket EC Replication Config /ectest-new/ectest RS 3 2 1048576
+ Execute ozone sh bucket create /ectest-new/testpropchange
+ Execute ozone sh bucket set-replication-config -r rs-3-2-1024k -t EC /ectest-new/testpropchange
+ Verify Bucket EC Replication Config /ectest-new/testpropchange RS 3 2 1048576
+ Execute ozone sh key put -r rs-3-2-1024k -t EC /ectest-new/ectest/core-site.xml /etc/hadoop/core-site.xml
+ Key Should Match Local File /ectest-new/ectest/core-site.xml /etc/hadoop/core-site.xml
+ Verify Key EC Replication Config /ectest-new/ectest/core-site.xml RS 3 2 1048576
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot b/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot
index 23d305e..07753a3 100644
--- a/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot
@@ -61,3 +61,41 @@
${output} = Execute ozone sh key put ${key} ${file}
Should not contain ${output} Failed
Log Uploaded ${file} to ${key}
+
+Verify Bucket Empty Replication Config
+ [arguments] ${bucket}
+ ${result} = Execute ozone sh bucket info ${bucket} | jq -r '.replicationConfig'
+ Should Be Equal ${result} null
+
+Verify Bucket Replica Replication Config
+ [arguments] ${bucket} ${type} ${factor}
+ ${result} = Execute ozone sh bucket info ${bucket} | jq -r '.replicationConfig.replicationType, .replicationConfig.replicationFactor'
+ Verify Replica Replication Config ${result} ${type} ${factor}
+
+Verify Key Replica Replication Config
+ [arguments] ${key} ${type} ${factor}
+ ${result} = Execute ozone sh key info ${key} | jq -r '.replicationConfig.replicationType, .replicationConfig.replicationFactor'
+ Verify Replica Replication Config ${result} ${type} ${factor}
+
+Verify Replica Replication Config
+ [arguments] ${result} ${type} ${factor}
+ Should Match Regexp ${result} ^(?m)${type}$
+ Should Match Regexp ${result} ^(?m)${factor}$
+
+Verify Bucket EC Replication Config
+ [arguments] ${bucket} ${encoding} ${data} ${parity} ${chunksize}
+ ${result} = Execute ozone sh bucket info ${bucket} | jq -r '.replicationConfig.replicationType, .replicationConfig.codec, .replicationConfig.data, .replicationConfig.parity, .replicationConfig.ecChunkSize'
+ Verify EC Replication Config ${result} ${encoding} ${data} ${parity} ${chunksize}
+
+Verify Key EC Replication Config
+ [arguments] ${key} ${encoding} ${data} ${parity} ${chunksize}
+ ${result} = Execute ozone sh key info ${key} | jq -r '.replicationConfig.replicationType, .replicationConfig.codec, .replicationConfig.data, .replicationConfig.parity, .replicationConfig.ecChunkSize'
+ Verify EC Replication Config ${result} ${encoding} ${data} ${parity} ${chunksize}
+
+Verify EC Replication Config
+ [arguments] ${result} ${encoding} ${data} ${parity} ${chunksize}
+ Should Match Regexp ${result} ^(?m)EC$
+ Should Match Regexp ${result} ^(?m)${encoding}$
+ Should Match Regexp ${result} ^(?m)${data}$
+ Should Match Regexp ${result} ^(?m)${parity}$
+ Should Match Regexp ${result} ^(?m)${chunksize}$
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot b/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot
index f888d22..5e03d0c 100644
--- a/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot
@@ -45,8 +45,8 @@
Execute ozone fs -copyFromLocal NOTICE.txt ${DEEP_URL}/
${result} = Execute ozone sh key list ${VOLUME}/${BUCKET} | jq -r '.[].name'
Should contain ${result} NOTICE.txt
- ${result} = Execute ozone sh key info ${VOLUME}/${BUCKET}/${DEEP_DIR}/NOTICE.txt | jq -r '.replicationFactor'
- Should Be Equal ${result} 3
+ ${result} = Execute ozone sh key info ${VOLUME}/${BUCKET}/${DEEP_DIR}/NOTICE.txt | jq -r '.replicationConfig.replicationFactor'
+ Should Be Equal ${result} THREE
Put
Execute ozone fs -put NOTICE.txt ${DEEP_URL}/PUTFILE.txt
diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot b/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot
index 96939c9..6f3bbef 100644
--- a/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot
@@ -105,6 +105,7 @@
Set Suite Variable ${BUCKET}
Run Keyword if '${BUCKET}' == 'link' Setup links for S3 tests
Run Keyword if '${BUCKET}' == 'encrypted' Create encrypted bucket
+ Run Keyword if '${BUCKET}' == 'erasure' Create EC bucket
Setup links for S3 tests
${exists} = Bucket Exists o3://${OM_SERVICE_ID}/s3v/link
@@ -124,6 +125,11 @@
Execute ozone sh bucket link o3://${OM_SERVICE_ID}/legacy/source-bucket o3://${OM_SERVICE_ID}/s3v/${bucket}
[return] ${bucket}
+Create EC bucket
+ ${exists} = Bucket Exists o3://${OM_SERVICE_ID}/s3v/erasure
+ Return From Keyword If ${exists}
+ Execute ozone sh bucket create --replication rs-3-2-1024k --type EC o3://${OM_SERVICE_ID}/s3v/erasure
+
Generate random prefix
${random} = Generate Ozone String
Set Suite Variable ${PREFIX} ${random}
diff --git a/hadoop-ozone/dist/src/main/smoketest/upgrade/prepare.robot b/hadoop-ozone/dist/src/main/smoketest/upgrade/prepare.robot
index 0f6d7a0..b1b4095 100644
--- a/hadoop-ozone/dist/src/main/smoketest/upgrade/prepare.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/upgrade/prepare.robot
@@ -19,7 +19,7 @@
Test Timeout 5 minutes
Test Setup Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit test user testuser testuser.keytab
-** Test Cases ***
+*** Test Cases ***
Prepare Ozone Manager
${result} = Execute ozone admin om prepare -id %{OM_SERVICE_ID}
Wait Until Keyword Succeeds 3min 10sec Should contain ${result} OM Preparation successful!
diff --git a/hadoop-ozone/integration-test/pom.xml b/hadoop-ozone/integration-test/pom.xml
index c3bce81..cea5163 100644
--- a/hadoop-ozone/integration-test/pom.xml
+++ b/hadoop-ozone/integration-test/pom.xml
@@ -110,6 +110,11 @@
</dependency>
<dependency>
<groupId>org.junit.jupiter</groupId>
+ <artifactId>junit-jupiter-params</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-migrationsupport</artifactId>
<scope>test</scope>
</dependency>
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
index bf24ae9..4c4bdc6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
@@ -21,6 +21,8 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.ZoneOffset;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -29,6 +31,7 @@
import java.util.Set;
import java.util.TreeSet;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -42,8 +45,12 @@
import org.apache.hadoop.fs.TrashPolicy;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConsts;
@@ -82,6 +89,7 @@
import static org.junit.Assert.fail;
import org.apache.ozone.test.LambdaTestUtils;
+import org.apache.ozone.test.TestClock;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -166,7 +174,7 @@
conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
bucketLayout.name());
cluster = MiniOzoneCluster.newBuilder(conf)
- .setNumDatanodes(3)
+ .setNumDatanodes(5)
.build();
cluster.waitForClusterToBeReady();
@@ -1188,6 +1196,66 @@
}
@Test
+ public void testCreateKeyShouldUseRefreshedBucketReplicationConfig()
+ throws IOException {
+ OzoneBucket bucket =
+ TestDataUtil.createVolumeAndBucket(cluster, bucketLayout);
+ final TestClock testClock = new TestClock(Instant.now(), ZoneOffset.UTC);
+
+ String rootPath = String
+ .format("%s://%s.%s/", OzoneConsts.OZONE_URI_SCHEME, bucket.getName(),
+ bucket.getVolumeName());
+
+ // Set the fs.defaultFS and start the filesystem
+ Configuration conf = new OzoneConfiguration(cluster.getConf());
+ conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath);
+ // Set the number of keys to be processed during batch operate.
+ OzoneFileSystem o3FS = (OzoneFileSystem) FileSystem.get(conf);
+
+ //Let's reset the clock to control the time.
+ ((BasicOzoneClientAdapterImpl) (o3FS.getAdapter())).setClock(testClock);
+
+ createKeyAndAssertKeyType(bucket, o3FS, new Path(rootPath, "key"),
+ ReplicationType.RATIS);
+
+ bucket.setReplicationConfig(new ECReplicationConfig("rs-3-2-1024k"));
+
+ //After changing the bucket policy, it should create ec key, but o3fs will
+ // refresh after some time. So, it will be sill old type.
+ createKeyAndAssertKeyType(bucket, o3FS, new Path(rootPath, "key1"),
+ ReplicationType.RATIS);
+
+ testClock.fastForward(300 * 1000 + 1);
+
+ //After client bucket refresh time, it should create new type what is
+ // available on bucket at that moment.
+ createKeyAndAssertKeyType(bucket, o3FS, new Path(rootPath, "key2"),
+ ReplicationType.EC);
+
+ // Rechecking the same steps with changing to Ratis again to check the
+ // behavior is consistent.
+ bucket.setReplicationConfig(
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
+
+ createKeyAndAssertKeyType(bucket, o3FS, new Path(rootPath, "key3"),
+ ReplicationType.EC);
+
+ testClock.fastForward(300 * 1000 + 1);
+
+ createKeyAndAssertKeyType(bucket, o3FS, new Path(rootPath, "key4"),
+ ReplicationType.RATIS);
+ }
+
+ private void createKeyAndAssertKeyType(OzoneBucket bucket,
+ OzoneFileSystem o3FS, Path keyPath, ReplicationType expectedType)
+ throws IOException {
+ o3FS.createFile(keyPath).build().close();
+ Assert.assertEquals(expectedType.name(),
+ bucket.getKey(o3FS.pathToKey(keyPath)).getReplicationConfig()
+ .getReplicationType().name());
+ }
+
+ @Test
public void testGetTrashRoots() throws IOException {
String username = UserGroupInformation.getCurrentUser().getShortUserName();
Path trashRoot = new Path(OZONE_URI_DELIMITER, TRASH_PREFIX);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java
index 5c5c0e1..b88da8a 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java
@@ -31,7 +31,12 @@
import org.apache.hadoop.fs.TrashPolicy;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OFSPath;
import org.apache.hadoop.ozone.OzoneAcl;
@@ -81,6 +86,7 @@
import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
+import java.util.UUID;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
@@ -214,7 +220,7 @@
conf.set(OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS,
OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS_NATIVE);
cluster = MiniOzoneCluster.newBuilder(conf)
- .setNumDatanodes(3)
+ .setNumDatanodes(5)
.build();
cluster.waitForClusterToBeReady();
objectStore = cluster.getClient().getObjectStore();
@@ -1575,6 +1581,66 @@
}
@Test
+ public void testBucketDefaultsShouldNotBeInheritedToFileForNonEC()
+ throws Exception {
+ BucketArgs.Builder builder = BucketArgs.newBuilder();
+ builder.setStorageType(StorageType.DISK);
+ builder.setBucketLayout(BucketLayout.LEGACY);
+ builder.setDefaultReplicationConfig(
+ new DefaultReplicationConfig(ReplicationType.STAND_ALONE,
+ ReplicationFactor.ONE));
+ BucketArgs omBucketArgs = builder.build();
+ String vol = UUID.randomUUID().toString();
+ String buck = UUID.randomUUID().toString();
+ final OzoneBucket bucket100 = TestDataUtil
+ .createVolumeAndBucket(cluster, vol, buck, BucketLayout.LEGACY,
+ omBucketArgs);
+ Assert.assertEquals(ReplicationType.STAND_ALONE.name(),
+ bucket100.getReplicationConfig().getReplicationType().name());
+
+ // Bucket has default STAND_ALONE and client has default RATIS.
+ // In this case, it should not inherit from bucket
+ try (OzoneFSOutputStream file = adapter
+ .createFile(vol + "/" + buck + "/test", (short) 3, true, false)) {
+ file.write(new byte[1024]);
+ }
+ OFSPath ofsPath = new OFSPath(vol + "/" + buck + "/test");
+ final OzoneBucket bucket = adapter.getBucket(ofsPath, false);
+ final OzoneKeyDetails key = bucket.getKey(ofsPath.getKeyName());
+ Assert.assertEquals(key.getReplicationConfig().getReplicationType().name(),
+ ReplicationType.RATIS.name());
+ }
+
+ @Test
+ public void testBucketDefaultsShouldBeInheritedToFileForEC()
+ throws Exception {
+ BucketArgs.Builder builder = BucketArgs.newBuilder();
+ builder.setStorageType(StorageType.DISK);
+ builder.setBucketLayout(BucketLayout.LEGACY);
+ builder.setDefaultReplicationConfig(
+ new DefaultReplicationConfig(ReplicationType.EC,
+ new ECReplicationConfig("RS-3-2-1024")));
+ BucketArgs omBucketArgs = builder.build();
+ String vol = UUID.randomUUID().toString();
+ String buck = UUID.randomUUID().toString();
+ final OzoneBucket bucket101 = TestDataUtil
+ .createVolumeAndBucket(cluster, vol, buck, BucketLayout.LEGACY,
+ omBucketArgs);
+ Assert.assertEquals(ReplicationType.EC.name(),
+ bucket101.getReplicationConfig().getReplicationType().name());
+ // Bucket has default EC and client has default RATIS.
+ // In this case, it should inherit from bucket
+ try (OzoneFSOutputStream file = adapter
+ .createFile(vol + "/" + buck + "/test", (short) 3, true, false)) {
+ file.write(new byte[1024]);
+ }
+ OFSPath ofsPath = new OFSPath(vol + "/" + buck + "/test");
+ final OzoneBucket bucket = adapter.getBucket(ofsPath, false);
+ final OzoneKeyDetails key = bucket.getKey(ofsPath.getKeyName());
+ Assert.assertEquals(ReplicationType.EC.name(),
+ key.getReplicationConfig().getReplicationType().name());
+ }
+
public void testNonPrivilegedUserMkdirCreateBucket() throws IOException {
// This test is only meaningful when ACL is enabled
Assume.assumeTrue("ACL is not enabled. Skipping this test as it requires " +
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
index ac0ad91..6920dd4 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
@@ -48,6 +48,8 @@
import org.apache.hadoop.hdds.scm.ha.SCMHANodeDetails;
import org.apache.hadoop.hdds.scm.ha.SCMHAUtils;
import org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdds.scm.proxy.SCMClientConfig;
@@ -376,16 +378,21 @@
startRecon();
}
- private void waitForHddsDatanodesStop() throws TimeoutException,
- InterruptedException {
+ private void waitForHddsDatanodeToStop(DatanodeDetails dn)
+ throws TimeoutException, InterruptedException {
GenericTestUtils.waitFor(() -> {
- final int healthy = scm.getNodeCount(HEALTHY);
- boolean isReady = healthy == hddsDatanodes.size();
- if (!isReady) {
- LOG.info("Waiting on {} datanodes out of {} to be marked unhealthy.",
- healthy, hddsDatanodes.size());
+ NodeStatus status;
+ try {
+ status = scm.getScmNodeManager().getNodeStatus(dn);
+ } catch (NodeNotFoundException e) {
+ return true;
}
- return isReady;
+ if (status.equals(NodeStatus.inServiceHealthy())) {
+ LOG.info("Waiting on datanode to be marked stale.");
+ return false;
+ } else {
+ return true;
+ }
}, 1000, waitForClusterToBeReadyTimeout);
}
@@ -410,7 +417,7 @@
hddsDatanodes.remove(i);
if (waitForDatanode) {
// wait for node to be removed from SCM healthy node list.
- waitForHddsDatanodesStop();
+ waitForHddsDatanodeToStop(datanodeService.getDatanodeDetails());
}
String[] args = new String[] {};
HddsDatanodeService service =
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestDataUtil.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestDataUtil.java
index 2983270..4b1e724 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestDataUtil.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestDataUtil.java
@@ -56,6 +56,19 @@
public static OzoneBucket createVolumeAndBucket(MiniOzoneCluster cluster,
String volumeName, String bucketName, BucketLayout bucketLayout)
throws IOException {
+ BucketArgs omBucketArgs;
+ BucketArgs.Builder builder = BucketArgs.newBuilder();
+ builder.setStorageType(StorageType.DISK);
+ builder.setBucketLayout(bucketLayout);
+ omBucketArgs = builder.build();
+
+ return createVolumeAndBucket(cluster, volumeName, bucketName, bucketLayout,
+ omBucketArgs);
+ }
+
+ public static OzoneBucket createVolumeAndBucket(MiniOzoneCluster cluster,
+ String volumeName, String bucketName, BucketLayout bucketLayout,
+ BucketArgs omBucketArgs) throws IOException {
String userName = "user" + RandomStringUtils.randomNumeric(5);
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
@@ -69,11 +82,6 @@
objectStore.createVolume(volumeName, volumeArgs);
OzoneVolume volume = objectStore.getVolume(volumeName);
- BucketArgs omBucketArgs;
- BucketArgs.Builder builder = BucketArgs.newBuilder();
- builder.setStorageType(StorageType.DISK);
- builder.setBucketLayout(bucketLayout);
- omBucketArgs = builder.build();
volume.createBucket(bucketName, omBucketArgs);
return volume.getBucket(bucketName);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java
index ef561e6..b239621 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java
@@ -62,11 +62,17 @@
xmlPropsToSkipCompare.add("ozone.om.decommissioned.nodes" +
".EXAMPLEOMSERVICEID");
xmlPropsToSkipCompare.add("ozone.scm.nodes.EXAMPLESCMSERVICEID");
+ xmlPropsToSkipCompare.add("ozone.scm.nodes.EXAMPLESCMSERVICEID");
xmlPrefixToSkipCompare.add("ipc.client.rpc-timeout.ms");
xmlPropsToSkipCompare.add("ozone.om.leader.election.minimum.timeout" +
".duration"); // Deprecated config
configurationPropsToSkipCompare
.add(ScmConfig.ConfigStrings.HDDS_SCM_INIT_DEFAULT_LAYOUT_VERSION);
+ // Currently replication and type configs moved to server side.
+ configurationPropsToSkipCompare
+ .add(OzoneConfigKeys.OZONE_REPLICATION);
+ configurationPropsToSkipCompare
+ .add(OzoneConfigKeys.OZONE_REPLICATION_TYPE);
configurationPropsToSkipCompare
.add(OzoneConfigKeys.OZONE_CLIENT_REQUIRED_OM_VERSION_MIN_KEY);
configurationPropsToSkipCompare
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestECKeyOutputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestECKeyOutputStream.java
new file mode 100644
index 0000000..948c481
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestECKeyOutputStream.java
@@ -0,0 +1,421 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.rpc;
+
+import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
+import org.apache.hadoop.ozone.client.io.KeyOutputStream;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.container.TestHelper;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
+
+/**
+ * Tests key output stream.
+ */
+public class TestECKeyOutputStream {
+ private static MiniOzoneCluster cluster;
+ private static OzoneConfiguration conf = new OzoneConfiguration();
+ private static OzoneClient client;
+ private static ObjectStore objectStore;
+ private static int chunkSize;
+ private static int flushSize;
+ private static int maxFlushSize;
+ private static int blockSize;
+ private static String volumeName;
+ private static String bucketName;
+ private static String keyString;
+ private static int dataBlocks = 3;
+ private static int inputSize = dataBlocks * chunkSize;
+ private static byte[][] inputChunks = new byte[dataBlocks][chunkSize];
+
+ /**
+ * Create a MiniDFSCluster for testing.
+ */
+ @BeforeClass
+ public static void init() throws Exception {
+ chunkSize = 1024;
+ flushSize = 2 * chunkSize;
+ maxFlushSize = 2 * flushSize;
+ blockSize = 2 * maxFlushSize;
+
+ OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class);
+ clientConfig.setChecksumType(ContainerProtos.ChecksumType.NONE);
+ clientConfig.setStreamBufferFlushDelay(false);
+ conf.setFromObject(clientConfig);
+
+ conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
+ // If SCM detects dead node too quickly, then container would be moved to
+ // closed state and all in progress writes will get exception. To avoid
+ // that, we are just keeping higher timeout and none of the tests depending
+ // on deadnode detection timeout currently.
+ conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 30, TimeUnit.SECONDS);
+ conf.setTimeDuration(OZONE_SCM_DEADNODE_INTERVAL, 60, TimeUnit.SECONDS);
+ conf.setTimeDuration("hdds.ratis.raft.server.rpc.slowness.timeout", 300,
+ TimeUnit.SECONDS);
+ conf.setTimeDuration(
+ "hdds.ratis.raft.server.notification.no-leader.timeout", 300,
+ TimeUnit.SECONDS);
+ conf.setQuietMode(false);
+ conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 4,
+ StorageUnit.MB);
+ conf.setTimeDuration(HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL, 500,
+ TimeUnit.MILLISECONDS);
+ conf.setTimeDuration(HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL, 1,
+ TimeUnit.SECONDS);
+ cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(10)
+ .setTotalPipelineNumLimit(10).setBlockSize(blockSize)
+ .setChunkSize(chunkSize).setStreamBufferFlushSize(flushSize)
+ .setStreamBufferMaxSize(maxFlushSize)
+ .setStreamBufferSizeUnit(StorageUnit.BYTES).build();
+ cluster.waitForClusterToBeReady();
+ client = OzoneClientFactory.getRpcClient(conf);
+ objectStore = client.getObjectStore();
+ keyString = UUID.randomUUID().toString();
+ volumeName = "testeckeyoutputstream";
+ bucketName = volumeName;
+ objectStore.createVolume(volumeName);
+ objectStore.getVolume(volumeName).createBucket(bucketName);
+ initInputChunks();
+ }
+
+ /**
+ * Shutdown MiniDFSCluster.
+ */
+ @AfterClass
+ public static void shutdown() {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ }
+
+ @Test
+ public void testCreateKeyWithECReplicationConfig() throws Exception {
+ try (OzoneOutputStream key = TestHelper
+ .createKey(keyString, new ECReplicationConfig(3, 2,
+ ECReplicationConfig.EcCodec.RS, chunkSize), inputSize,
+ objectStore, volumeName, bucketName)) {
+ Assert.assertTrue(key.getOutputStream() instanceof ECKeyOutputStream);
+ }
+ }
+
+ @Test
+ public void testCreateKeyWithOutBucketDefaults() throws Exception {
+ OzoneVolume volume = objectStore.getVolume(volumeName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+ try (OzoneOutputStream out = bucket.createKey("myKey", inputSize)) {
+ Assert.assertTrue(out.getOutputStream() instanceof KeyOutputStream);
+ for (int i = 0; i < inputChunks.length; i++) {
+ out.write(inputChunks[i]);
+ }
+ }
+ }
+
+ @Test
+ public void testCreateKeyWithBucketDefaults() throws Exception {
+ String myBucket = UUID.randomUUID().toString();
+ OzoneVolume volume = objectStore.getVolume(volumeName);
+ final BucketArgs.Builder bucketArgs = BucketArgs.newBuilder();
+ bucketArgs.setDefaultReplicationConfig(
+ new DefaultReplicationConfig(ReplicationType.EC,
+ new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ chunkSize)));
+
+ volume.createBucket(myBucket, bucketArgs.build());
+ OzoneBucket bucket = volume.getBucket(myBucket);
+
+ try (OzoneOutputStream out = bucket.createKey(keyString, inputSize)) {
+ Assert.assertTrue(out.getOutputStream() instanceof ECKeyOutputStream);
+ for (int i = 0; i < inputChunks.length; i++) {
+ out.write(inputChunks[i]);
+ }
+ }
+ byte[] buf = new byte[chunkSize];
+ try (OzoneInputStream in = bucket.readKey(keyString)) {
+ for (int i = 0; i < inputChunks.length; i++) {
+ int read = in.read(buf, 0, chunkSize);
+ Assert.assertEquals(chunkSize, read);
+ Assert.assertTrue(Arrays.equals(buf, inputChunks[i]));
+ }
+ }
+ }
+
+ @Test
+ public void testOverwriteECKeyWithRatisKey() throws Exception {
+ String myBucket = UUID.randomUUID().toString();
+ OzoneVolume volume = objectStore.getVolume(volumeName);
+ final BucketArgs.Builder bucketArgs = BucketArgs.newBuilder();
+ volume.createBucket(myBucket, bucketArgs.build());
+ OzoneBucket bucket = volume.getBucket(myBucket);
+ createKeyAndCheckReplicationConfig(keyString, bucket,
+ new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ chunkSize));
+
+ //Overwrite with RATIS/THREE
+ createKeyAndCheckReplicationConfig(keyString, bucket,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
+
+ //Overwrite with RATIS/ONE
+ createKeyAndCheckReplicationConfig(keyString, bucket,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE));
+ }
+
+ @Test
+ public void testOverwriteRatisKeyWithECKey() throws Exception {
+ String myBucket = UUID.randomUUID().toString();
+ OzoneVolume volume = objectStore.getVolume(volumeName);
+ final BucketArgs.Builder bucketArgs = BucketArgs.newBuilder();
+ volume.createBucket(myBucket, bucketArgs.build());
+ OzoneBucket bucket = volume.getBucket(myBucket);
+
+ createKeyAndCheckReplicationConfig(keyString, bucket,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
+ // Overwrite with EC key
+ createKeyAndCheckReplicationConfig(keyString, bucket,
+ new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ chunkSize));
+ }
+
+ private void createKeyAndCheckReplicationConfig(String keyName,
+ OzoneBucket bucket, ReplicationConfig replicationConfig)
+ throws IOException {
+ try (OzoneOutputStream out = bucket
+ .createKey(keyName, inputSize, replicationConfig, new HashMap<>())) {
+ for (int i = 0; i < inputChunks.length; i++) {
+ out.write(inputChunks[i]);
+ }
+ }
+ OzoneKeyDetails key = bucket.getKey(keyName);
+ Assert.assertEquals(replicationConfig, key.getReplicationConfig());
+ }
+
+ @Test
+ public void testCreateRatisKeyAndWithECBucketDefaults() throws Exception {
+ OzoneBucket bucket = getOzoneBucket();
+ try (OzoneOutputStream out = bucket.createKey(
+ "testCreateRatisKeyAndWithECBucketDefaults", 2000,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE),
+ new HashMap<>())) {
+ Assert.assertTrue(out.getOutputStream() instanceof KeyOutputStream);
+ for (int i = 0; i < inputChunks.length; i++) {
+ out.write(inputChunks[i]);
+ }
+ }
+ }
+
+ @Test
+ public void test13ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(13);
+ }
+
+ @Test
+ public void test15ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(15);
+ }
+
+ @Test
+ public void test20ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(20);
+ }
+
+ @Test
+ public void test21ChunksInSingleWriteOp() throws IOException {
+ testMultipleChunksInSingleWriteOp(21);
+ }
+
+ public void testMultipleChunksInSingleWriteOp(int numChunks)
+ throws IOException {
+ byte[] inputData = getInputBytes(numChunks);
+ final OzoneBucket bucket = getOzoneBucket();
+ String keyName = "testMultipleChunksInSingleWriteOp" + numChunks;
+ try (OzoneOutputStream out = bucket.createKey(keyName, 4096,
+ new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ chunkSize), new HashMap<>())) {
+ out.write(inputData);
+ }
+
+ validateContent(inputData, bucket, bucket.getKey(keyName));
+ }
+
+ @Test
+ public void testECContainerKeysCount()
+ throws IOException, InterruptedException, TimeoutException {
+ byte[] inputData = getInputBytes(1);
+ final OzoneBucket bucket = getOzoneBucket();
+ ContainerOperationClient containerOperationClient =
+ new ContainerOperationClient(conf);
+
+ ECReplicationConfig repConfig = new ECReplicationConfig(
+ 3, 2, ECReplicationConfig.EcCodec.RS, chunkSize);
+ // Close all EC pipelines so we must get a fresh pipeline and hence
+ // container for this test.
+ PipelineManager pm =
+ cluster.getStorageContainerManager().getPipelineManager();
+ for (Pipeline p : pm.getPipelines(repConfig)) {
+ pm.closePipeline(p, true);
+ }
+
+ String keyName = UUID.randomUUID().toString();
+ try (OzoneOutputStream out = bucket.createKey(keyName, 4096,
+ repConfig, new HashMap<>())) {
+ out.write(inputData);
+ }
+ OzoneKeyDetails key = bucket.getKey(keyName);
+ long currentKeyContainerID =
+ key.getOzoneKeyLocations().get(0).getContainerID();
+
+ GenericTestUtils.waitFor(() -> {
+ try {
+ return containerOperationClient.getContainer(currentKeyContainerID)
+ .getNumberOfKeys() == 1;
+ } catch (IOException exception) {
+ Assert.fail("Unexpected exception " + exception);
+ return false;
+ }
+ }, 100, 10000);
+ validateContent(inputData, bucket, key);
+ }
+
+ private void validateContent(byte[] inputData, OzoneBucket bucket,
+ OzoneKey key) throws IOException {
+ try (OzoneInputStream is = bucket.readKey(key.getName())) {
+ byte[] fileContent = new byte[inputData.length];
+ Assert.assertEquals(inputData.length, is.read(fileContent));
+ Assert.assertEquals(new String(inputData, UTF_8),
+ new String(fileContent, UTF_8));
+ }
+ }
+
+ private OzoneBucket getOzoneBucket() throws IOException {
+ String myBucket = UUID.randomUUID().toString();
+ OzoneVolume volume = objectStore.getVolume(volumeName);
+ final BucketArgs.Builder bucketArgs = BucketArgs.newBuilder();
+ bucketArgs.setDefaultReplicationConfig(
+ new DefaultReplicationConfig(ReplicationType.EC,
+ new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ chunkSize)));
+
+ volume.createBucket(myBucket, bucketArgs.build());
+ return volume.getBucket(myBucket);
+ }
+
+ private static void initInputChunks() {
+ for (int i = 0; i < dataBlocks; i++) {
+ inputChunks[i] = getBytesWith(i + 1, chunkSize);
+ }
+ }
+
+ private static byte[] getBytesWith(int singleDigitNumber, int total) {
+ StringBuilder builder = new StringBuilder(singleDigitNumber);
+ for (int i = 1; i <= total; i++) {
+ builder.append(singleDigitNumber);
+ }
+ return builder.toString().getBytes(UTF_8);
+ }
+
+ @Test
+ public void testWriteShouldSucceedWhenDNKilled() throws Exception {
+ int numChunks = 3;
+ byte[] inputData = getInputBytes(numChunks);
+ final OzoneBucket bucket = getOzoneBucket();
+ String keyName = "testWriteShouldSucceedWhenDNKilled" + numChunks;
+ DatanodeDetails nodeToKill = null;
+ try {
+ try (OzoneOutputStream out = bucket.createKey(keyName, 1024,
+ new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ chunkSize), new HashMap<>())) {
+ out.write(inputData);
+ // Kill a node from first pipeline
+ nodeToKill =
+ ((ECKeyOutputStream) out.getOutputStream()).getStreamEntries()
+ .get(0).getPipeline().getFirstNode();
+ cluster.shutdownHddsDatanode(nodeToKill);
+
+ out.write(inputData);
+ // Check the second blockGroup pipeline to make sure that the failed not
+ // is not selected.
+ Assert.assertFalse(
+ ((ECKeyOutputStream) out.getOutputStream()).getStreamEntries()
+ .get(1).getPipeline().getNodes().contains(nodeToKill));
+ }
+
+ try (OzoneInputStream is = bucket.readKey(keyName)) {
+ // We wrote "inputData" twice, so do two reads and ensure the correct
+ // data comes back.
+ for (int i = 0; i < 2; i++) {
+ byte[] fileContent = new byte[inputData.length];
+ Assert.assertEquals(inputData.length, is.read(fileContent));
+ Assert.assertEquals(new String(inputData, UTF_8),
+ new String(fileContent, UTF_8));
+ }
+ }
+ } finally {
+ cluster.restartHddsDatanode(nodeToKill, true);
+ }
+ }
+
+ private byte[] getInputBytes(int numChunks) {
+ byte[] inputData = new byte[numChunks * chunkSize];
+ for (int i = 0; i < numChunks; i++) {
+ int start = (i * chunkSize);
+ Arrays.fill(inputData, start, start + chunkSize - 1,
+ String.valueOf(i % 9).getBytes(UTF_8)[0]);
+ }
+ return inputData;
+ }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadWithFSO.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadWithFSO.java
index 9ea04d4..293725f 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadWithFSO.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientMultipartUploadWithFSO.java
@@ -18,9 +18,11 @@
package org.apache.hadoop.ozone.client.rpc;
import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.OzoneTestUtils;
@@ -620,8 +622,10 @@
OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
bucket.listParts(keyName, uploadID, 0, 3);
- Assert.assertEquals(RATIS,
- ozoneMultipartUploadPartListParts.getReplicationType());
+ Assert.assertEquals(
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE),
+ ozoneMultipartUploadPartListParts.getReplicationConfig());
+
Assert.assertEquals(3,
ozoneMultipartUploadPartListParts.getPartInfoList().size());
@@ -722,8 +726,9 @@
OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
bucket.listParts(keyName, uploadID, 0, 2);
- Assert.assertEquals(RATIS,
- ozoneMultipartUploadPartListParts.getReplicationType());
+ Assert.assertEquals(
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE),
+ ozoneMultipartUploadPartListParts.getReplicationConfig());
Assert.assertEquals(2,
ozoneMultipartUploadPartListParts.getPartInfoList().size());
@@ -821,8 +826,9 @@
Assert.assertEquals(0,
ozoneMultipartUploadPartListParts.getPartInfoList().size());
- Assert.assertEquals(RATIS,
- ozoneMultipartUploadPartListParts.getReplicationType());
+ Assert.assertEquals(
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE),
+ ozoneMultipartUploadPartListParts.getReplicationConfig());
// As we don't have any parts with greater than partNumberMarker and list
// is not truncated, so it should return false here.
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
index 5bdd344..527b3bb 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
@@ -19,7 +19,6 @@
package org.apache.hadoop.ozone.client.rpc;
import java.io.IOException;
-
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
index 6272afa..fb50b81 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
@@ -35,12 +35,17 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.StorageType;
@@ -98,6 +103,7 @@
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.QuotaUtil;
import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
@@ -146,6 +152,8 @@
import org.junit.jupiter.api.MethodOrderer;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestMethodOrder;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
/**
* This is an abstract class to test all the public facing APIs of Ozone
@@ -188,7 +196,7 @@
// for testZReadKeyWithUnhealthyContainerReplica.
conf.set("ozone.scm.stale.node.interval", "10s");
cluster = MiniOzoneCluster.newBuilder(conf)
- .setNumDatanodes(3)
+ .setNumDatanodes(5)
.setTotalPipelineNumLimit(10)
.setScmId(scmId)
.setClusterId(clusterId)
@@ -589,7 +597,7 @@
@Test
public void testCreateBucketWithVersioning()
- throws IOException, OzoneClientException {
+ throws IOException {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
store.createVolume(volumeName);
@@ -637,6 +645,23 @@
}
@Test
+ public void testCreateBucketWithReplicationConfig()
+ throws IOException {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ ReplicationConfig repConfig = new ECReplicationConfig(3, 2);
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ BucketArgs bucketArgs = BucketArgs.newBuilder()
+ .setDefaultReplicationConfig(new DefaultReplicationConfig(repConfig))
+ .build();
+ volume.createBucket(bucketName, bucketArgs);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+ Assert.assertEquals(bucketName, bucket.getName());
+ Assert.assertEquals(repConfig, bucket.getReplicationConfig());
+ }
+
+ @Test
public void testCreateBucketWithAllArgument()
throws IOException {
String volumeName = UUID.randomUUID().toString();
@@ -645,18 +670,21 @@
ACLType.ALL, ACCESS);
List<OzoneAcl> acls = new ArrayList<>();
acls.add(userAcl);
+ ReplicationConfig repConfig = new ECReplicationConfig(3, 2);
store.createVolume(volumeName);
OzoneVolume volume = store.getVolume(volumeName);
BucketArgs.Builder builder = BucketArgs.newBuilder();
builder.setVersioning(true)
.setStorageType(StorageType.SSD)
- .setAcls(acls);
+ .setAcls(acls)
+ .setDefaultReplicationConfig(new DefaultReplicationConfig(repConfig));
volume.createBucket(bucketName, builder.build());
OzoneBucket bucket = volume.getBucket(bucketName);
Assert.assertEquals(bucketName, bucket.getName());
Assert.assertEquals(true, bucket.getVersioning());
Assert.assertEquals(StorageType.SSD, bucket.getStorageType());
Assert.assertTrue(bucket.getAcls().contains(userAcl));
+ Assert.assertEquals(repConfig, bucket.getReplicationConfig());
}
@Test
@@ -670,7 +698,6 @@
"Bucket or Volume name has an unsupported" +
" character : #",
() -> volume.createBucket(bucketName));
-
}
@Test
@@ -799,7 +826,6 @@
Assert.assertEquals(StorageType.SSD, newBucket.getStorageType());
}
-
@Test
public void testDeleteBucket()
throws Exception {
@@ -818,8 +844,8 @@
);
}
- private boolean verifyRatisReplication(String volumeName, String bucketName,
- String keyName, ReplicationType type, ReplicationFactor factor)
+ private void verifyReplication(String volumeName, String bucketName,
+ String keyName, ReplicationConfig replication)
throws IOException {
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
.setVolumeName(volumeName)
@@ -827,22 +853,13 @@
.setKeyName(keyName)
.setRefreshPipeline(true)
.build();
- HddsProtos.ReplicationType replicationType =
- HddsProtos.ReplicationType.valueOf(type.toString());
- HddsProtos.ReplicationFactor replicationFactor =
- HddsProtos.ReplicationFactor.valueOf(factor.getValue());
OmKeyInfo keyInfo = ozoneManager.lookupKey(keyArgs);
for (OmKeyLocationInfo info:
keyInfo.getLatestVersionLocations().getLocationList()) {
ContainerInfo container =
storageContainerLocationClient.getContainer(info.getContainerID());
- if (!ReplicationConfig.getLegacyFactor(container.getReplicationConfig())
- .equals(replicationFactor) || (
- container.getReplicationType() != replicationType)) {
- return false;
- }
+ Assert.assertEquals(replication, container.getReplicationConfig());
}
- return true;
}
@Test
@@ -870,9 +887,9 @@
OzoneInputStream is = bucket.readKey(keyName);
byte[] fileContent = new byte[value.getBytes(UTF_8).length];
is.read(fileContent);
- Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
- keyName, RATIS,
- ONE));
+ verifyReplication(volumeName, bucketName, keyName,
+ RatisReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE));
Assert.assertEquals(value, new String(fileContent, UTF_8));
Assert.assertFalse(key.getCreationTime().isBefore(testStartTime));
Assert.assertFalse(key.getModificationTime().isBefore(testStartTime));
@@ -1012,6 +1029,64 @@
store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
}
+ // TODO: testBucketQuota overlaps with testBucketUsedBytes,
+ // do cleanup when EC branch gets merged into master.
+ @ParameterizedTest
+ @MethodSource("replicationConfigs")
+ void testBucketQuota(ReplicationConfig repConfig) throws IOException {
+ int blockSize = (int) ozoneManager.getConfiguration().getStorageSize(
+ OZONE_SCM_BLOCK_SIZE, OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
+
+ for (int i = 0; i <= repConfig.getRequiredNodes(); i++) {
+ bucketQuotaTestHelper(i * blockSize, repConfig);
+ bucketQuotaTestHelper(i * blockSize + 1, repConfig);
+ }
+ }
+
+ private void bucketQuotaTestHelper(int keyLength, ReplicationConfig repConfig)
+ throws IOException {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ String keyName = UUID.randomUUID().toString();
+ long blockSize = (long) ozoneManager.getConfiguration().getStorageSize(
+ OZONE_SCM_BLOCK_SIZE, OZONE_SCM_BLOCK_SIZE_DEFAULT, StorageUnit.BYTES);
+
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+
+ byte[] value = new byte[keyLength];
+ int dataGroupSize = repConfig instanceof ECReplicationConfig ?
+ ((ECReplicationConfig) repConfig).getData() : 1;
+ long preAllocatedBlocks = Math.min(ozoneManager.getPreallocateBlocksMax(),
+ (keyLength - 1) / (blockSize * dataGroupSize) + 1);
+ long preAllocatedSpace = preAllocatedBlocks * blockSize
+ * repConfig.getRequiredNodes();
+ long keyQuota = QuotaUtil.getReplicatedSize(keyLength, repConfig);
+
+ OzoneOutputStream out = bucket.createKey(keyName, keyLength,
+ repConfig, new HashMap<>());
+ Assert.assertEquals(preAllocatedSpace,
+ store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
+ out.write(value);
+ out.close();
+ Assert.assertEquals(keyQuota,
+ store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
+
+ out = bucket.createKey(keyName, keyLength, repConfig, new HashMap<>());
+ Assert.assertEquals(keyQuota + preAllocatedSpace,
+ store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
+ out.write(value);
+ out.close();
+ Assert.assertEquals(keyQuota,
+ store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
+
+ bucket.deleteKey(keyName);
+ Assert.assertEquals(0L,
+ store.getVolume(volumeName).getBucket(bucketName).getUsedBytes());
+ }
+
@Test
public void testVolumeUsedNamespace() throws IOException {
String volumeName = UUID.randomUUID().toString();
@@ -1144,8 +1219,10 @@
OzoneVolume volume = store.getVolume(volumeName);
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
+ ReplicationConfig replication = RatisReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE);
OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
- RATIS, ONE);
+ replication);
assertNotNull(multipartInfo);
String uploadID = multipartInfo.getUploadID();
@@ -1228,8 +1305,9 @@
byte[] fileContent = new byte[value.getBytes(UTF_8).length];
is.read(fileContent);
is.close();
- Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
- keyName, ReplicationType.RATIS, ONE));
+ verifyReplication(volumeName, bucketName, keyName,
+ RatisReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE));
Assert.assertEquals(value, new String(fileContent, UTF_8));
Assert.assertFalse(key.getCreationTime().isBefore(testStartTime));
Assert.assertFalse(key.getModificationTime().isBefore(testStartTime));
@@ -1262,9 +1340,9 @@
byte[] fileContent = new byte[value.getBytes(UTF_8).length];
is.read(fileContent);
is.close();
- Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
- keyName, ReplicationType.RATIS,
- THREE));
+ verifyReplication(volumeName, bucketName, keyName,
+ RatisReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.THREE));
Assert.assertEquals(value, new String(fileContent, UTF_8));
Assert.assertFalse(key.getCreationTime().isBefore(testStartTime));
Assert.assertFalse(key.getModificationTime().isBefore(testStartTime));
@@ -1303,9 +1381,9 @@
byte[] fileContent = new byte[data.getBytes(UTF_8).length];
is.read(fileContent);
is.close();
- Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
- keyName, ReplicationType.RATIS,
- THREE));
+ verifyReplication(volumeName, bucketName, keyName,
+ RatisReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.THREE));
Assert.assertEquals(data, new String(fileContent, UTF_8));
Assert.assertFalse(key.getCreationTime().isBefore(testStartTime));
Assert.assertFalse(key.getModificationTime().isBefore(testStartTime));
@@ -2034,7 +2112,6 @@
.startsWith(bucketBaseNameA + i + "-"));
}
Assert.assertFalse(volBBucketAIter.hasNext());
-
}
@Test
@@ -2050,6 +2127,43 @@
}
@Test
+ public void testListBucketsReplicationConfig()
+ throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ getStore().createVolume(volumeName);
+ OzoneVolume volume = getStore().getVolume(volumeName);
+
+ // bucket-level replication config: null (default)
+ String bucketName = UUID.randomUUID().toString();
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.listBuckets(bucketName).next();
+ Assert.assertNull(bucket.getReplicationConfig());
+
+ // bucket-level replication config: EC/rs-3-2-1024k
+ String ecBucketName = UUID.randomUUID().toString();
+ ReplicationConfig ecRepConfig = new ECReplicationConfig(3, 2);
+ BucketArgs ecBucketArgs = BucketArgs.newBuilder()
+ .setDefaultReplicationConfig(
+ new DefaultReplicationConfig(ecRepConfig))
+ .build();
+ volume.createBucket(ecBucketName, ecBucketArgs);
+ OzoneBucket ecBucket = volume.listBuckets(ecBucketName).next();
+ Assert.assertEquals(ecRepConfig, ecBucket.getReplicationConfig());
+
+ // bucket-level replication config: RATIS/THREE
+ String ratisBucketName = UUID.randomUUID().toString();
+ ReplicationConfig ratisRepConfig = ReplicationConfig
+ .fromTypeAndFactor(RATIS, THREE);
+ BucketArgs ratisBucketArgs = BucketArgs.newBuilder()
+ .setDefaultReplicationConfig(
+ new DefaultReplicationConfig(ratisRepConfig))
+ .build();
+ volume.createBucket(ratisBucketName, ratisBucketArgs);
+ OzoneBucket ratisBucket = volume.listBuckets(ratisBucketName).next();
+ Assert.assertEquals(ratisRepConfig, ratisBucket.getReplicationConfig());
+ }
+
+ @Test
public void testListKey()
throws IOException {
String volumeA = "vol-a-" + RandomStringUtils.randomNumeric(5);
@@ -2199,9 +2313,18 @@
}
}
- @Test
- public void testInitiateMultipartUploadWithReplicationInformationSet() throws
- IOException {
+ static Stream<ReplicationConfig> replicationConfigs() {
+ return Stream.of(
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE),
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE),
+ new ECReplicationConfig(3, 2)
+ );
+ }
+
+ @ParameterizedTest
+ @MethodSource("replicationConfigs")
+ void testInitiateMultipartUpload(ReplicationConfig replicationConfig)
+ throws IOException {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
String keyName = UUID.randomUUID().toString();
@@ -2211,7 +2334,7 @@
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
- RATIS, ONE);
+ replicationConfig);
assertNotNull(multipartInfo);
String uploadID = multipartInfo.getUploadID();
@@ -2223,7 +2346,7 @@
// Call initiate multipart upload for the same key again, this should
// generate a new uploadID.
multipartInfo = bucket.initiateMultipartUpload(keyName,
- RATIS, ONE);
+ replicationConfig);
assertNotNull(multipartInfo);
Assert.assertEquals(volumeName, multipartInfo.getVolumeName());
@@ -2266,8 +2389,10 @@
assertNotNull(multipartInfo.getUploadID());
}
- @Test
- public void testUploadPartWithNoOverride() throws IOException {
+ @ParameterizedTest
+ @MethodSource("replicationConfigs")
+ void testUploadPartWithNoOverride(ReplicationConfig replication)
+ throws IOException {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
String keyName = UUID.randomUUID().toString();
@@ -2278,7 +2403,7 @@
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
- RATIS, ONE);
+ replication);
assertNotNull(multipartInfo);
String uploadID = multipartInfo.getUploadID();
@@ -2296,13 +2421,13 @@
.getCommitUploadPartInfo();
assertNotNull(commitUploadPartInfo);
- String partName = commitUploadPartInfo.getPartName();
assertNotNull(commitUploadPartInfo.getPartName());
-
}
- @Test
- public void testUploadPartOverrideWithStandAlone() throws IOException {
+ @ParameterizedTest
+ @MethodSource("replicationConfigs")
+ void testUploadPartOverride(ReplicationConfig replication)
+ throws IOException {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
@@ -2315,7 +2440,7 @@
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
- RATIS, ONE);
+ replication);
assertNotNull(multipartInfo);
String uploadID = multipartInfo.getUploadID();
@@ -2362,66 +2487,6 @@
}
@Test
- public void testUploadPartOverrideWithRatis() throws IOException {
-
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- String keyName = UUID.randomUUID().toString();
- String sampleData = "sample Value";
-
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- OzoneBucket bucket = volume.getBucket(bucketName);
- OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
- ReplicationType.RATIS, THREE);
-
- assertNotNull(multipartInfo);
- String uploadID = multipartInfo.getUploadID();
- Assert.assertEquals(volumeName, multipartInfo.getVolumeName());
- Assert.assertEquals(bucketName, multipartInfo.getBucketName());
- Assert.assertEquals(keyName, multipartInfo.getKeyName());
- assertNotNull(multipartInfo.getUploadID());
-
- int partNumber = 1;
-
- OzoneOutputStream ozoneOutputStream = bucket.createMultipartKey(keyName,
- sampleData.length(), partNumber, uploadID);
- ozoneOutputStream.write(string2Bytes(sampleData), 0, sampleData.length());
- ozoneOutputStream.close();
-
- OmMultipartCommitUploadPartInfo commitUploadPartInfo = ozoneOutputStream
- .getCommitUploadPartInfo();
-
- assertNotNull(commitUploadPartInfo);
- String partName = commitUploadPartInfo.getPartName();
- assertNotNull(commitUploadPartInfo.getPartName());
-
- // Overwrite the part by creating part key with same part number
- // and same content.
- ozoneOutputStream = bucket.createMultipartKey(keyName,
- sampleData.length(), partNumber, uploadID);
- ozoneOutputStream.write(string2Bytes(sampleData), 0, "name".length());
- ozoneOutputStream.close();
-
- commitUploadPartInfo = ozoneOutputStream
- .getCommitUploadPartInfo();
-
- assertNotNull(commitUploadPartInfo);
- assertNotNull(commitUploadPartInfo.getPartName());
-
- // AWS S3 for same content generates same partName during upload part.
- // In AWS S3 ETag is generated from md5sum. In Ozone right now we
- // don't do this. For now to make things work for large file upload
- // through aws s3 cp, the partName are generated in a predictable fashion.
- // So, when a part is override partNames will still be same irrespective
- // of content in ozone s3. This will make S3 Mpu completeMPU pass when
- // comparing part names and large file uploads work using aws cp.
- assertEquals("Part names should be same", partName,
- commitUploadPartInfo.getPartName());
- }
-
- @Test
public void testNoSuchUploadError() throws Exception {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
@@ -2461,7 +2526,9 @@
bucket.addAcl(acl3);
bucket.addAcl(acl4);
- doMultipartUpload(bucket, keyName, (byte)98);
+ ReplicationConfig replication = RatisReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE);
+ doMultipartUpload(bucket, keyName, (byte)98, replication);
OzoneObj keyObj = OzoneObjInfo.Builder.newBuilder()
.setBucketName(bucketName)
.setVolumeName(volumeName).setKeyName(keyName)
@@ -2507,7 +2574,7 @@
OzoneBucket bucket2 = client.getObjectStore().getVolume(volumeName)
.getBucket(bucketName);
try {
- initiateMultipartUpload(bucket2, keyName2, ReplicationType.RATIS, THREE);
+ initiateMultipartUpload(bucket2, keyName2, anyReplication());
fail("User without permission should fail");
} catch (Exception e) {
assertTrue(e instanceof OMException);
@@ -2530,7 +2597,7 @@
store.addAcl(bucketObj, acl9);
store.addAcl(bucketObj, acl10);
String uploadId = initiateMultipartUpload(bucket2, keyName2,
- ReplicationType.RATIS, THREE);
+ anyReplication());
// Upload part
byte[] data = generateData(OzoneConsts.OM_MULTIPART_MIN_SIZE, (byte)1);
@@ -2552,8 +2619,10 @@
}
}
- @Test
- public void testMultipartUploadOverride() throws Exception {
+ @ParameterizedTest
+ @MethodSource("replicationConfigs")
+ void testMultipartUploadOverride(ReplicationConfig replication)
+ throws Exception {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
String keyName = UUID.randomUUID().toString();
@@ -2563,11 +2632,11 @@
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
- doMultipartUpload(bucket, keyName, (byte)96);
+ doMultipartUpload(bucket, keyName, (byte)96, replication);
// Initiate Multipart upload again, now we should read latest version, as
// read always reads latest blocks.
- doMultipartUpload(bucket, keyName, (byte)97);
+ doMultipartUpload(bucket, keyName, (byte)97, replication);
}
@@ -2584,8 +2653,8 @@
OzoneBucket bucket = volume.getBucket(bucketName);
// Initiate multipart upload
- String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
- ONE);
+ String uploadID = initiateMultipartUpload(bucket, keyName,
+ anyReplication());
// Upload Parts
Map<Integer, String> partsMap = new TreeMap<>();
@@ -2617,8 +2686,8 @@
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
- String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
- ONE);
+ String uploadID = initiateMultipartUpload(bucket, keyName,
+ anyReplication());
// We have not uploaded any parts, but passing some list it should throw
// error.
@@ -2642,8 +2711,9 @@
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
- String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
- ONE);
+ ReplicationConfig replication = RatisReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE);
+ String uploadID = initiateMultipartUpload(bucket, keyName, replication);
uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
// We have not uploaded any parts, but passing some list it should throw
@@ -2667,8 +2737,9 @@
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
- String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
- ONE);
+ ReplicationConfig replication = RatisReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE);
+ String uploadID = initiateMultipartUpload(bucket, keyName, replication);
uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
// We have not uploaded any parts, but passing some list it should throw
@@ -2696,7 +2767,7 @@
}
@Test
- public void testAbortUploadFailWithInProgressPartUpload() throws Exception {
+ void testAbortUploadFailWithInProgressPartUpload() throws Exception {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
String keyName = UUID.randomUUID().toString();
@@ -2707,7 +2778,7 @@
OzoneBucket bucket = volume.getBucket(bucketName);
OmMultipartInfo omMultipartInfo = bucket.initiateMultipartUpload(keyName,
- RATIS, ONE);
+ anyReplication());
Assert.assertNotNull(omMultipartInfo.getUploadID());
@@ -2731,7 +2802,7 @@
}
@Test
- public void testCommitPartAfterCompleteUpload() throws Exception {
+ void testCommitPartAfterCompleteUpload() throws Exception {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
String keyName = UUID.randomUUID().toString();
@@ -2742,7 +2813,7 @@
OzoneBucket bucket = volume.getBucket(bucketName);
OmMultipartInfo omMultipartInfo = bucket.initiateMultipartUpload(keyName,
- RATIS, ONE);
+ anyReplication());
Assert.assertNotNull(omMultipartInfo.getUploadID());
@@ -2804,8 +2875,8 @@
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
- String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
- ONE);
+ String uploadID = initiateMultipartUpload(bucket, keyName,
+ anyReplication());
bucket.abortMultipartUpload(keyName, uploadID);
}
@@ -2820,14 +2891,16 @@
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
- String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
- ONE);
+ String uploadID = initiateMultipartUpload(bucket, keyName,
+ anyReplication());
uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
bucket.abortMultipartUpload(keyName, uploadID);
}
- @Test
- public void testListMultipartUploadParts() throws Exception {
+ @ParameterizedTest
+ @MethodSource("replicationConfigs")
+ void testListMultipartUploadParts(ReplicationConfig replication)
+ throws Exception {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
String keyName = UUID.randomUUID().toString();
@@ -2838,8 +2911,7 @@
OzoneBucket bucket = volume.getBucket(bucketName);
Map<Integer, String> partsMap = new TreeMap<>();
- String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
- ONE);
+ String uploadID = initiateMultipartUpload(bucket, keyName, replication);
String partName1 = uploadPart(bucket, keyName, uploadID, 1,
generateData(OzoneConsts.OM_MULTIPART_MIN_SIZE, (byte)97));
partsMap.put(1, partName1);
@@ -2855,10 +2927,9 @@
OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
bucket.listParts(keyName, uploadID, 0, 3);
- Assert.assertEquals(RATIS,
- ozoneMultipartUploadPartListParts.getReplicationType());
- Assert.assertEquals(3,
- ozoneMultipartUploadPartListParts.getPartInfoList().size());
+ Assert.assertEquals(
+ replication,
+ ozoneMultipartUploadPartListParts.getReplicationConfig());
Assert.assertEquals(partsMap.get(ozoneMultipartUploadPartListParts
.getPartInfoList().get(0).getPartNumber()),
@@ -2876,9 +2947,11 @@
Assert.assertFalse(ozoneMultipartUploadPartListParts.isTruncated());
}
- @Test
- public void testListMultipartUploadPartsWithContinuation()
- throws Exception {
+ @ParameterizedTest
+ @MethodSource("replicationConfigs")
+ void testListMultipartUploadPartsWithContinuation(
+ ReplicationConfig replication) throws Exception {
+
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
String keyName = UUID.randomUUID().toString();
@@ -2889,8 +2962,7 @@
OzoneBucket bucket = volume.getBucket(bucketName);
Map<Integer, String> partsMap = new TreeMap<>();
- String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
- ONE);
+ String uploadID = initiateMultipartUpload(bucket, keyName, replication);
String partName1 = uploadPart(bucket, keyName, uploadID, 1,
generateData(OzoneConsts.OM_MULTIPART_MIN_SIZE, (byte)97));
partsMap.put(1, partName1);
@@ -2906,8 +2978,8 @@
OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
bucket.listParts(keyName, uploadID, 0, 2);
- Assert.assertEquals(RATIS,
- ozoneMultipartUploadPartListParts.getReplicationType());
+ Assert.assertEquals(replication,
+ ozoneMultipartUploadPartListParts.getReplicationConfig());
Assert.assertEquals(2,
ozoneMultipartUploadPartListParts.getPartInfoList().size());
@@ -2994,8 +3066,8 @@
OzoneBucket bucket = volume.getBucket(bucketName);
- String uploadID = initiateMultipartUpload(bucket, keyName, RATIS,
- ONE);
+ String uploadID = initiateMultipartUpload(bucket, keyName,
+ anyReplication());
uploadPart(bucket, keyName, uploadID, 1,
generateData(OzoneConsts.OM_MULTIPART_MIN_SIZE, (byte)97));
@@ -3007,8 +3079,6 @@
Assert.assertEquals(0,
ozoneMultipartUploadPartListParts.getPartInfoList().size());
- Assert.assertEquals(RATIS,
- ozoneMultipartUploadPartListParts.getReplicationType());
// As we don't have any parts with greater than partNumberMarker and list
// is not truncated, so it should return false here.
@@ -3365,11 +3435,11 @@
return chars;
}
- private void doMultipartUpload(OzoneBucket bucket, String keyName, byte val)
+ private void doMultipartUpload(OzoneBucket bucket, String keyName, byte val,
+ ReplicationConfig replication)
throws Exception {
// Initiate Multipart upload request
- String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType
- .RATIS, THREE);
+ String uploadID = initiateMultipartUpload(bucket, keyName, replication);
// Upload parts
Map<Integer, String> partsMap = new TreeMap<>();
@@ -3402,9 +3472,8 @@
OzoneInputStream inputStream = bucket.readKey(keyName);
inputStream.read(fileContent);
- Assert.assertTrue(verifyRatisReplication(bucket.getVolumeName(),
- bucket.getName(), keyName, ReplicationType.RATIS,
- THREE));
+ verifyReplication(bucket.getVolumeName(), bucket.getName(), keyName,
+ replication);
StringBuilder sb = new StringBuilder(length);
@@ -3431,10 +3500,9 @@
}
private String initiateMultipartUpload(OzoneBucket bucket, String keyName,
- ReplicationType replicationType, ReplicationFactor replicationFactor)
- throws Exception {
+ ReplicationConfig replicationConfig) throws Exception {
OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
- replicationType, replicationFactor);
+ replicationConfig);
String uploadID = multipartInfo.getUploadID();
Assert.assertNotNull(uploadID);
@@ -3551,9 +3619,8 @@
OzoneInputStream is = bucket.readKey(keyName);
byte[] fileContent = new byte[text.getBytes(UTF_8).length];
is.read(fileContent);
- Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
- keyName, RATIS,
- ONE));
+ verifyReplication(volumeName, bucketName, keyName,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE));
Assert.assertEquals(text, new String(fileContent, UTF_8));
//Step 4
@@ -3631,9 +3698,8 @@
OzoneInputStream is = bucket.readKey(keyName);
byte[] fileContent = new byte[text.getBytes(UTF_8).length];
is.read(fileContent);
- Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
- keyName, RATIS,
- ONE));
+ verifyReplication(volumeName, bucketName, keyName,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE));
Assert.assertEquals(text, new String(fileContent, UTF_8));
//Step 4
@@ -3798,4 +3864,53 @@
createRequiredForVersioningTest(volumeName, bucketName, keyName, true);
checkExceptedResultForVersioningTest(volumeName, bucketName, keyName, 2);
}
+
+ @Test
+ public void testSetECReplicationConfigOnBucket()
+ throws IOException {
+ String volumeName = UUID.randomUUID().toString();
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ OzoneBucket bucket = getBucket(volume);
+ ReplicationConfig currentReplicationConfig = bucket.getReplicationConfig();
+ Assert.assertEquals(
+ StandaloneReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE),
+ currentReplicationConfig);
+ ECReplicationConfig ecReplicationConfig =
+ new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS, 1024);
+ bucket.setReplicationConfig(ecReplicationConfig);
+
+ // Get the bucket and check the updated config.
+ bucket = volume.getBucket(bucket.getName());
+
+ Assert.assertEquals(ecReplicationConfig, bucket.getReplicationConfig());
+
+ RatisReplicationConfig ratisReplicationConfig =
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE);
+ bucket.setReplicationConfig(ratisReplicationConfig);
+
+ // Get the bucket and check the updated config.
+ bucket = volume.getBucket(bucket.getName());
+
+ Assert.assertEquals(ratisReplicationConfig, bucket.getReplicationConfig());
+
+ //Reset replication config back.
+ bucket.setReplicationConfig(currentReplicationConfig);
+ }
+
+ private OzoneBucket getBucket(OzoneVolume volume) throws IOException {
+ String bucketName = UUID.randomUUID().toString();
+ BucketArgs.Builder builder = BucketArgs.newBuilder();
+ builder.setVersioning(true).setDefaultReplicationConfig(
+ new DefaultReplicationConfig(
+ StandaloneReplicationConfig.getInstance(
+ HddsProtos.ReplicationFactor.ONE)));
+ volume.createBucket(bucketName, builder.build());
+ return volume.getBucket(bucketName);
+ }
+
+ private static ReplicationConfig anyReplication() {
+ return RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE);
+ }
}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/ECStreamTestUtil.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/ECStreamTestUtil.java
new file mode 100644
index 0000000..0503dbe
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/ECStreamTestUtil.java
@@ -0,0 +1,408 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ozone.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.ozone.erasurecode.rawcoder.util.CodecUtil;
+import org.apache.ratis.util.Preconditions;
+import org.junit.Assert;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SplittableRandom;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class providing methods useful in EC tests.
+ */
+public final class ECStreamTestUtil {
+
+ private ECStreamTestUtil() {
+ }
+
+ public static OmKeyLocationInfo createKeyInfo(ReplicationConfig repConf,
+ long blockLength, Map<DatanodeDetails, Integer> dnMap) {
+
+ Pipeline pipeline = Pipeline.newBuilder()
+ .setState(Pipeline.PipelineState.CLOSED)
+ .setId(PipelineID.randomId())
+ .setNodes(new ArrayList<>(dnMap.keySet()))
+ .setReplicaIndexes(dnMap)
+ .setReplicationConfig(repConf)
+ .build();
+
+ OmKeyLocationInfo keyInfo = new OmKeyLocationInfo.Builder()
+ .setBlockID(new BlockID(1, 1))
+ .setLength(blockLength)
+ .setOffset(0)
+ .setPipeline(pipeline)
+ .setPartNumber(0)
+ .build();
+ return keyInfo;
+ }
+
+ public static OmKeyLocationInfo createKeyInfo(ReplicationConfig repConf,
+ int nodeCount, long blockLength) {
+ Map<DatanodeDetails, Integer> datanodes = new HashMap<>();
+ for (int i = 0; i < nodeCount; i++) {
+ datanodes.put(MockDatanodeDetails.randomDatanodeDetails(), i + 1);
+ }
+ return createKeyInfo(repConf, blockLength, datanodes);
+ }
+
+ /**
+ * Fill / Pad the remaining space in a buffer with zeros.
+ * @param buf
+ */
+ public static void zeroFill(ByteBuffer buf) {
+ byte[] a = buf.array();
+ Arrays.fill(a, buf.position(), buf.limit(), (byte)0);
+ buf.position(buf.limit());
+ }
+
+ /**
+ * Given a List of ByteBuffers, write length of random bytes from the given
+ * Random generator to the byte buffers. The data is striped across the
+ * buffers in stripeSize chunks.
+ * When the length of data has been written, the buffer limits are set to
+ * their final positions.
+ *
+ * @param bufs The list of buffers to fill with random data
+ * @param stripeSize The stripe size to use
+ * @param rand The random generator to use
+ * @param length The length of data to write.
+ */
+ public static void randomFill(ByteBuffer[] bufs, int stripeSize,
+ SplittableRandom rand, int length) {
+ Preconditions.assertTrue(totalSpaceAvailable(bufs) >= length);
+ int remaining = length;
+ while (remaining > 0) {
+ for (ByteBuffer b : bufs) {
+ int toWrite = Math.min(stripeSize, remaining);
+ for (int i = 0; i < toWrite; i++) {
+ b.put((byte) rand.nextInt(255));
+ }
+ remaining -= toWrite;
+ }
+ }
+ // Set the buffer limits to the final position
+ for (ByteBuffer b : bufs) {
+ b.limit(b.position());
+ }
+ }
+
+ public static void randomFill(ByteBuffer buf, SplittableRandom rand) {
+ while (buf.remaining() > 0) {
+ buf.put((byte) rand.nextInt(255));
+ }
+ }
+
+ private static int totalSpaceAvailable(ByteBuffer[] bufs) {
+ int space = 0;
+ for (ByteBuffer b : bufs) {
+ space += b.remaining();
+ }
+ return space;
+ }
+
+ /**
+ * Given a buffer which has data loaded, flip the buffer and ensure it matches
+ * byte for byte the next series of bytes from the Random generator.
+ * @param b Byte Buffers containing data
+ * @param rand The random generator
+ */
+ public static void assertBufferMatches(ByteBuffer b, SplittableRandom rand) {
+ b.flip();
+ int i = 0;
+ while (b.hasRemaining()) {
+ i++;
+ Assert.assertEquals("Failed on iteration " + i,
+ (byte)rand.nextInt(255), b.get());
+ }
+ }
+
+ /**
+ * Given a List of ByteBuffers and the RepConfig, encode the parity buffers
+ * from the data buffers. The data buffers should be passed "as is" after
+ * reading data. That is, the position will be at the last data byte read in
+ * or the buffer limit.
+ * The data buffers and parity will be returned "ready to read" with the
+ * position reset to zero.
+ * @param data List of data buffers
+ * @param ecConfig The ECReplicationConfig.
+ * @return List of encoded parity buffers.
+ * @throws IOException
+ */
+ public static ByteBuffer[] generateParity(ByteBuffer[] data,
+ ECReplicationConfig ecConfig) throws IOException {
+ // First data buffer dictates the size
+ int cellSize = data[0].limit();
+ data[0].flip();
+ // Store the positions of the remaining data buffers so we can restore them
+ int[] dataLimits = new int[data.length];
+ for (int i = 1; i < data.length; i++) {
+ dataLimits[i] = data[i].limit();
+ data[i].limit(cellSize);
+ zeroFill(data[i]);
+ data[i].flip();
+ }
+ ByteBuffer[] parity = new ByteBuffer[ecConfig.getParity()];
+ for (int i = 0; i < ecConfig.getParity(); i++) {
+ parity[i] = ByteBuffer.allocate(cellSize);
+ }
+ RawErasureEncoder encoder =
+ CodecUtil.createRawEncoderWithFallback(ecConfig);
+ encoder.encode(data, parity);
+
+ data[0].flip();
+ for (int i = 1; i < data.length; i++) {
+ data[i].limit(dataLimits[i]);
+ data[i].position(0);
+ }
+ return parity;
+ }
+
+ /**
+ * Returns a new map containing a random DatanodeDetails for each index in
+ * inputs.
+ * @param idxs A list of indexes to add to the map
+ * @return A map of DatanodeDetails to index.
+ */
+ public static Map<DatanodeDetails, Integer> createIndexMap(int... idxs) {
+ Map<DatanodeDetails, Integer> map = new HashMap<>();
+ for (int i : idxs) {
+ map.put(MockDatanodeDetails.randomDatanodeDetails(), i);
+ }
+ return map;
+ }
+
+
+ /**
+ * A stream factory which can be used in tests to provide TestBlockStream
+ * instances.
+ */
+ public static class TestBlockInputStreamFactory implements
+ BlockInputStreamFactory {
+
+ private Map<Integer, TestBlockInputStream> blockStreams =
+ new LinkedHashMap<>();
+ private List<ByteBuffer> blockStreamData;
+ // List of EC indexes that should fail immediately on read
+ private List<Integer> failIndexes = new ArrayList<>();
+
+ private Pipeline currentPipeline;
+
+ public synchronized
+ List<ECStreamTestUtil.TestBlockInputStream> getBlockStreams() {
+ return blockStreams.values().stream().collect(Collectors.toList());
+ }
+
+ public synchronized Set<Integer> getStreamIndexes() {
+ return blockStreams.keySet();
+ }
+
+ public synchronized ECStreamTestUtil.TestBlockInputStream getBlockStream(
+ int ecIndex) {
+ return blockStreams.get(ecIndex);
+ }
+
+ public synchronized void setBlockStreamData(List<ByteBuffer> bufs) {
+ this.blockStreamData = bufs;
+ }
+
+ public synchronized void setCurrentPipeline(Pipeline pipeline) {
+ this.currentPipeline = pipeline;
+ }
+
+ public synchronized void setFailIndexes(List<Integer> fail) {
+ failIndexes.addAll(fail);
+ }
+
+ public synchronized BlockExtendedInputStream create(
+ ReplicationConfig repConfig,
+ OmKeyLocationInfo blockInfo, Pipeline pipeline,
+ Token<OzoneBlockTokenIdentifier> token, boolean verifyChecksum,
+ XceiverClientFactory xceiverFactory,
+ Function<BlockID, Pipeline> refreshFunction) {
+
+ int repInd = currentPipeline.getReplicaIndex(pipeline.getNodes().get(0));
+ TestBlockInputStream stream = new TestBlockInputStream(
+ blockInfo.getBlockID(), blockInfo.getLength(),
+ blockStreamData.get(repInd - 1), repInd);
+ if (failIndexes.contains(repInd)) {
+ stream.setShouldError(true);
+ }
+ blockStreams.put(repInd, stream);
+ return stream;
+ }
+ }
+
+ /**
+ * A block stream that returns data from the provided ByteBuffer. Intended to
+ * be used in tests, rather than reading from a real block stream.
+ */
+ public static class TestBlockInputStream extends BlockExtendedInputStream {
+
+ private ByteBuffer data;
+ private boolean closed = false;
+ private BlockID blockID;
+ private long length;
+ private boolean shouldError = false;
+ private int shouldErrorPosition = 0;
+ private boolean shouldErrorOnSeek = false;
+ private IOException errorToThrow = null;
+ private int ecReplicaIndex = 0;
+ private static final byte EOF = -1;
+
+ TestBlockInputStream(BlockID blockId, long blockLen, ByteBuffer data) {
+ this(blockId, blockLen, data, 0);
+ }
+
+ TestBlockInputStream(BlockID blockId, long blockLen, ByteBuffer data,
+ int replicaIndex) {
+ this.blockID = blockId;
+ this.length = blockLen;
+ this.data = data;
+ this.ecReplicaIndex = replicaIndex;
+ data.position(0);
+ }
+
+ public boolean isClosed() {
+ return closed;
+ }
+
+ public void setShouldErrorOnSeek(boolean val) {
+ this.shouldErrorOnSeek = val;
+ }
+
+ public void setShouldError(boolean val) {
+ shouldError = val;
+ shouldErrorPosition = 0;
+ }
+
+ public void setShouldError(boolean val, int position,
+ IOException errorThrowable) {
+ this.shouldError = val;
+ this.shouldErrorPosition = position;
+ this.errorToThrow = errorThrowable;
+ }
+
+ public int getEcReplicaIndex() {
+ return ecReplicaIndex;
+ }
+
+ @Override
+ public BlockID getBlockID() {
+ return blockID;
+ }
+
+ @Override
+ public long getLength() {
+ return length;
+ }
+
+ @Override
+ public long getRemaining() {
+ return getLength() - getPos();
+ }
+
+ @Override
+ public int read(byte[] b, int off, int len)
+ throws IOException {
+ return read(ByteBuffer.wrap(b, off, len));
+ }
+
+ @Override
+ public int read(ByteBuffer buf) throws IOException {
+ if (shouldError && data.position() >= shouldErrorPosition) {
+ throwError();
+ }
+ if (getRemaining() == 0) {
+ return EOF;
+ }
+ int toRead = (int)Math.min(buf.remaining(), getRemaining());
+ for (int i = 0; i < toRead; i++) {
+ if (shouldError && data.position() >= shouldErrorPosition) {
+ throwError();
+ }
+ buf.put(data.get());
+ }
+ return toRead;
+ };
+
+ private void throwError() throws IOException {
+ if (errorToThrow != null) {
+ throw errorToThrow;
+ } else {
+ throw new IOException("Simulated error reading block");
+ }
+ }
+
+ @Override
+ protected int readWithStrategy(ByteReaderStrategy strategy) throws
+ IOException {
+ throw new IOException("Should not be called");
+ }
+
+ @Override
+ public void close() {
+ closed = true;
+ }
+
+ @Override
+ public void unbuffer() {
+ }
+
+ @Override
+ public long getPos() {
+ return data.position();
+ }
+
+ @Override
+ public void seek(long pos) throws IOException {
+ if (shouldErrorOnSeek) {
+ throw new IOException("Simulated exception");
+ }
+ data.position((int)pos);
+ }
+
+ }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStreamFactoryImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStreamFactoryImpl.java
new file mode 100644
index 0000000..8532835
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestBlockInputStreamFactoryImpl.java
@@ -0,0 +1,111 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.junit.Test;
+import org.junit.Assert;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Tests for BlockInputStreamFactoryImpl.
+ */
+public class TestBlockInputStreamFactoryImpl {
+
+ @Test
+ public void testNonECGivesBlockInputStream() {
+ BlockInputStreamFactory factory = new BlockInputStreamFactoryImpl();
+ ReplicationConfig repConfig =
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE);
+
+ OmKeyLocationInfo blockInfo = createKeyLocationInfo(repConfig, 3,
+ 1024 * 1024 * 10);
+
+ BlockExtendedInputStream stream =
+ factory.create(repConfig, blockInfo, blockInfo.getPipeline(),
+ blockInfo.getToken(), true, null, null);
+ Assert.assertTrue(stream instanceof BlockInputStream);
+ Assert.assertEquals(stream.getBlockID(), blockInfo.getBlockID());
+ Assert.assertEquals(stream.getLength(), blockInfo.getLength());
+ }
+
+ @Test
+ public void testECGivesECBlockInputStream() {
+ BlockInputStreamFactory factory = new BlockInputStreamFactoryImpl();
+ ReplicationConfig repConfig =
+ new ECReplicationConfig(3, 2);
+
+ OmKeyLocationInfo blockInfo =
+ createKeyLocationInfo(repConfig, 5, 1024 * 1024 * 10);
+
+ BlockExtendedInputStream stream =
+ factory.create(repConfig, blockInfo, blockInfo.getPipeline(),
+ blockInfo.getToken(), true, null, null);
+ Assert.assertTrue(stream instanceof ECBlockInputStreamProxy);
+ Assert.assertEquals(stream.getBlockID(), blockInfo.getBlockID());
+ Assert.assertEquals(stream.getLength(), blockInfo.getLength());
+ }
+
+ private OmKeyLocationInfo createKeyLocationInfo(ReplicationConfig repConf,
+ long blockLength, Map<DatanodeDetails, Integer> dnMap) {
+
+ Pipeline pipeline = Pipeline.newBuilder()
+ .setState(Pipeline.PipelineState.CLOSED)
+ .setId(PipelineID.randomId())
+ .setNodes(new ArrayList<>(dnMap.keySet()))
+ .setReplicaIndexes(dnMap)
+ .setReplicationConfig(repConf)
+ .build();
+
+ OmKeyLocationInfo keyInfo = new OmKeyLocationInfo.Builder()
+ .setBlockID(new BlockID(1, 1))
+ .setLength(blockLength)
+ .setOffset(0)
+ .setPipeline(pipeline)
+ .setPartNumber(0)
+ .build();
+ return keyInfo;
+ }
+
+ private OmKeyLocationInfo createKeyLocationInfo(ReplicationConfig repConf,
+ int nodeCount, long blockLength) {
+ Map<DatanodeDetails, Integer> datanodes = new HashMap<>();
+ for (int i = 0; i < nodeCount; i++) {
+ datanodes.put(MockDatanodeDetails.randomDatanodeDetails(), i + 1);
+ }
+ return createKeyLocationInfo(repConf, blockLength, datanodes);
+ }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java
index c39598e..82f494f 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java
@@ -19,6 +19,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+
import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
import org.apache.hadoop.hdds.scm.storage.ChunkInputStream;
import org.apache.hadoop.ozone.client.io.KeyInputStream;
@@ -57,7 +58,8 @@
KeyInputStream keyInputStream = getKeyInputStream(keyName);
- BlockInputStream block0Stream = keyInputStream.getBlockStreams().get(0);
+ BlockInputStream block0Stream =
+ (BlockInputStream)keyInputStream.getBlockStreams().get(0);
block0Stream.initialize();
ChunkInputStream chunk0Stream = block0Stream.getChunkStreams().get(0);
@@ -121,7 +123,8 @@
try (KeyInputStream keyInputStream = getKeyInputStream(keyName)) {
- BlockInputStream block0Stream = keyInputStream.getBlockStreams().get(0);
+ BlockInputStream block0Stream =
+ (BlockInputStream)keyInputStream.getBlockStreams().get(0);
block0Stream.initialize();
ChunkInputStream chunk0Stream = block0Stream.getChunkStreams().get(0);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStream.java
new file mode 100644
index 0000000..17a0a6f
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStream.java
@@ -0,0 +1,522 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.hdds.scm.storage.ByteReaderStrategy;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.ozone.client.io.BadDataLocationException;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.security.token.Token;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+/**
+ * Tests for ECBlockInputStream.
+ */
+public class TestECBlockInputStream {
+
+ private static final int ONEMB = 1024 * 1024;
+
+ private ECReplicationConfig repConfig;
+ private TestBlockInputStreamFactory streamFactory;
+
+ @Before
+ public void setup() {
+ repConfig = new ECReplicationConfig(3, 2,
+ ECReplicationConfig.EcCodec.RS, ONEMB);
+ streamFactory = new TestBlockInputStreamFactory();
+ }
+
+ @Test
+ public void testSufficientLocations() {
+ // EC-3-2, 5MB block, so all 3 data locations are needed
+ OmKeyLocationInfo keyInfo = ECStreamTestUtil
+ .createKeyInfo(repConfig, 5, 5 * ONEMB);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+ Assert.assertTrue(ecb.hasSufficientLocations());
+ }
+
+ // EC-3-2, very large block, so all 3 data locations are needed
+ keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, 5, 5000 * ONEMB);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+ Assert.assertTrue(ecb.hasSufficientLocations());
+ }
+
+ Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+
+ // EC-3-2, 1 byte short of 1MB with 1 location
+ dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+ keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB - 1, dnMap);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+ Assert.assertTrue(ecb.hasSufficientLocations());
+ }
+
+ // EC-3-2, 5MB blocks, only 2 locations passed so we do not have sufficient
+ // locations.
+ dnMap.clear();
+ dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+ keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+ Assert.assertFalse(ecb.hasSufficientLocations());
+ }
+
+ // EC-3-2, 5MB blocks, only 1 data and 2 parity locations present. For now
+ // this will fail as we don't support reconstruction reads yet.
+ dnMap.clear();
+ dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 1);
+ dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 4);
+ dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), 5);
+ keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, 5 * ONEMB, dnMap);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, new TestBlockInputStreamFactory())) {
+ Assert.assertFalse(ecb.hasSufficientLocations());
+ }
+ }
+
+ @Test
+ public void testCorrectBlockSizePassedToBlockStreamLessThanCell()
+ throws IOException {
+ ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB - 100);
+
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ ecb.read(buf);
+ // We expect only 1 block stream and it should have a length passed of
+ // ONEMB - 100.
+ List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+ Assert.assertEquals(ONEMB - 100, streams.get(0).getLength());
+ }
+ }
+
+ @Test
+ public void testCorrectBlockSizePassedToBlockStreamTwoCells()
+ throws IOException {
+ ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB + 100);
+
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ ecb.read(buf);
+ List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+ Assert.assertEquals(ONEMB, streams.get(0).getLength());
+ Assert.assertEquals(100, streams.get(1).getLength());
+ }
+ }
+
+ @Test
+ public void testCorrectBlockSizePassedToBlockStreamThreeCells()
+ throws IOException {
+ ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 2 * ONEMB + 100);
+
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ ecb.read(buf);
+ List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+ Assert.assertEquals(ONEMB, streams.get(0).getLength());
+ Assert.assertEquals(ONEMB, streams.get(1).getLength());
+ Assert.assertEquals(100, streams.get(2).getLength());
+ }
+ }
+
+ @Test
+ public void testCorrectBlockSizePassedToBlockStreamThreeFullAndPartialStripe()
+ throws IOException {
+ ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 10 * ONEMB + 100);
+
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ ecb.read(buf);
+ List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+ Assert.assertEquals(4 * ONEMB, streams.get(0).getLength());
+ Assert.assertEquals(3 * ONEMB + 100, streams.get(1).getLength());
+ Assert.assertEquals(3 * ONEMB, streams.get(2).getLength());
+ }
+ }
+
+ @Test
+ public void testCorrectBlockSizePassedToBlockStreamSingleFullCell()
+ throws IOException {
+ ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, ONEMB);
+
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ ecb.read(buf);
+ List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+ Assert.assertEquals(ONEMB, streams.get(0).getLength());
+ }
+ }
+
+ @Test
+ public void testCorrectBlockSizePassedToBlockStreamSeveralFullCells()
+ throws IOException {
+ ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 9 * ONEMB);
+
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ ecb.read(buf);
+ List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+ Assert.assertEquals(3 * ONEMB, streams.get(0).getLength());
+ Assert.assertEquals(3 * ONEMB, streams.get(1).getLength());
+ Assert.assertEquals(3 * ONEMB, streams.get(2).getLength());
+ }
+ }
+
+ @Test
+ public void testSimpleRead() throws IOException {
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+
+ ByteBuffer buf = ByteBuffer.allocate(100);
+
+ int read = ecb.read(buf);
+ Assert.assertEquals(100, read);
+ validateBufferContents(buf, 0, 100, (byte) 0);
+ Assert.assertEquals(100, ecb.getPos());
+ }
+ for (TestBlockInputStream s : streamFactory.getBlockStreams()) {
+ Assert.assertTrue(s.isClosed());
+ }
+ }
+
+ /**
+ * This test is to ensure we can read a small key of 1 chunk or less when only
+ * the first replica index is available.
+ */
+ @Test
+ public void testSimpleReadUnderOneChunk() throws IOException {
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 1, ONEMB);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+
+ ByteBuffer buf = ByteBuffer.allocate(100);
+
+ int read = ecb.read(buf);
+ Assert.assertEquals(100, read);
+ validateBufferContents(buf, 0, 100, (byte) 0);
+ Assert.assertEquals(100, ecb.getPos());
+ }
+ for (TestBlockInputStream s : streamFactory.getBlockStreams()) {
+ Assert.assertTrue(s.isClosed());
+ }
+ }
+
+ @Test
+ public void testReadPastEOF() throws IOException {
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 50);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+
+ ByteBuffer buf = ByteBuffer.allocate(100);
+
+ int read = ecb.read(buf);
+ Assert.assertEquals(50, read);
+ read = ecb.read(buf);
+ Assert.assertEquals(read, -1);
+ }
+ }
+
+ @Test
+ public void testReadCrossingMultipleECChunkBounds() throws IOException {
+ // EC-3-2, 5MB block, so all 3 data locations are needed
+ repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ 100);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+
+ // EC Chunk size is 100 and 3-2. Create a byte buffer to read 3.5 chunks,
+ // so 350
+ ByteBuffer buf = ByteBuffer.allocate(350);
+ int read = ecb.read(buf);
+ Assert.assertEquals(350, read);
+
+ validateBufferContents(buf, 0, 100, (byte) 0);
+ validateBufferContents(buf, 100, 200, (byte) 1);
+ validateBufferContents(buf, 200, 300, (byte) 2);
+ validateBufferContents(buf, 300, 350, (byte) 0);
+
+ buf.clear();
+ read = ecb.read(buf);
+ Assert.assertEquals(350, read);
+
+ validateBufferContents(buf, 0, 50, (byte) 0);
+ validateBufferContents(buf, 50, 150, (byte) 1);
+ validateBufferContents(buf, 150, 250, (byte) 2);
+ validateBufferContents(buf, 250, 350, (byte) 0);
+
+ }
+ for (TestBlockInputStream s : streamFactory.getBlockStreams()) {
+ Assert.assertTrue(s.isClosed());
+ }
+ }
+
+ @Test(expected = EOFException.class)
+ public void testSeekPastBlockLength() throws IOException {
+ repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 100);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ ecb.seek(1000);
+ }
+ }
+
+ @Test(expected = EOFException.class)
+ public void testSeekToLength() throws IOException {
+ repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 100);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ ecb.seek(100);
+ }
+ }
+
+ @Test
+ public void testSeekToLengthZeroLengthBlock() throws IOException {
+ repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 0);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ ecb.seek(0);
+ Assert.assertEquals(0, ecb.getPos());
+ Assert.assertEquals(0, ecb.getRemaining());
+ }
+ }
+
+ @Test
+ public void testSeekToValidPosition() throws IOException {
+ repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ ecb.seek(ONEMB - 1);
+ Assert.assertEquals(ONEMB - 1, ecb.getPos());
+ Assert.assertEquals(ONEMB * 4 + 1, ecb.getRemaining());
+ // First read should read the last byte of the first chunk
+ Assert.assertEquals(0, ecb.read());
+ Assert.assertEquals(ONEMB,
+ streamFactory.getBlockStreams().get(0).position);
+ // Second read should be the first byte of the second chunk.
+ Assert.assertEquals(1, ecb.read());
+
+ // Seek to the end of the file minus one byte
+ ecb.seek(ONEMB * 5 - 1);
+ Assert.assertEquals(1, ecb.read());
+ Assert.assertEquals(ONEMB * 2,
+ streamFactory.getBlockStreams().get(1).position);
+ // Second read should be EOF as there is no data left
+ Assert.assertEquals(-1, ecb.read());
+ Assert.assertEquals(0, ecb.getRemaining());
+ }
+ }
+
+ @Test
+ public void testErrorReadingBlockReportsBadLocation() throws IOException {
+ repConfig = new ECReplicationConfig(3, 2, ECReplicationConfig.EcCodec.RS,
+ ONEMB);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 5, 5 * ONEMB);
+ try (ECBlockInputStream ecb = new ECBlockInputStream(repConfig,
+ keyInfo, true, null, null, streamFactory)) {
+ // Read a full stripe to ensure all streams are created in the stream
+ // factory
+ ByteBuffer buf = ByteBuffer.allocate(3 * ONEMB);
+ int read = ecb.read(buf);
+ Assert.assertEquals(3 * ONEMB, read);
+ // Now make replication index 2 error on the next read
+ streamFactory.getBlockStreams().get(1).setThrowException(true);
+ buf.clear();
+ try {
+ ecb.read(buf);
+ Assert.fail("Exception should be thrown");
+ } catch (IOException e) {
+ Assert.assertTrue(e instanceof BadDataLocationException);
+ Assert.assertEquals(2,
+ keyInfo.getPipeline().getReplicaIndex(
+ ((BadDataLocationException) e).getFailedLocation()));
+ }
+ }
+ }
+
+ private void validateBufferContents(ByteBuffer buf, int from, int to,
+ byte val) {
+ for (int i = from; i < to; i++) {
+ Assert.assertEquals(val, buf.get(i));
+ }
+ }
+
+ private static class TestBlockInputStreamFactory implements
+ BlockInputStreamFactory {
+
+ private List<TestBlockInputStream> blockStreams = new ArrayList<>();
+
+ public synchronized List<TestBlockInputStream> getBlockStreams() {
+ return blockStreams;
+ }
+
+ public synchronized BlockExtendedInputStream create(
+ ReplicationConfig repConfig, OmKeyLocationInfo blockInfo,
+ Pipeline pipeline, Token<OzoneBlockTokenIdentifier> token,
+ boolean verifyChecksum, XceiverClientFactory xceiverFactory,
+ Function<BlockID, Pipeline> refreshFunction) {
+ TestBlockInputStream stream = new TestBlockInputStream(
+ blockInfo.getBlockID(), blockInfo.getLength(),
+ (byte)blockStreams.size());
+ blockStreams.add(stream);
+ return stream;
+ }
+ }
+
+ private static class TestBlockInputStream extends BlockExtendedInputStream {
+
+ private long position = 0;
+ private boolean closed = false;
+ private byte dataVal = 1;
+ private BlockID blockID;
+ private long length;
+ private boolean throwException = false;
+ private static final byte EOF = -1;
+
+ @SuppressWarnings("checkstyle:parameternumber")
+ TestBlockInputStream(BlockID blockId, long blockLen, byte dataVal) {
+ this.dataVal = dataVal;
+ this.blockID = blockId;
+ this.length = blockLen;
+ }
+
+ public boolean isClosed() {
+ return closed;
+ }
+
+ public void setThrowException(boolean shouldThrow) {
+ this.throwException = shouldThrow;
+ }
+
+ @Override
+ public BlockID getBlockID() {
+ return blockID;
+ }
+
+ @Override
+ public long getLength() {
+ return length;
+ }
+
+ @Override
+ public long getRemaining() {
+ return getLength() - position;
+ }
+
+ @Override
+ public int read(byte[] b, int off, int len)
+ throws IOException {
+ return read(ByteBuffer.wrap(b, off, len));
+ }
+
+ @Override
+ public int read(ByteBuffer buf) throws IOException {
+ if (getRemaining() == 0) {
+ return EOF;
+ }
+
+ if (throwException) {
+ throw new IOException("Simulated exception");
+ }
+
+ int toRead = Math.min(buf.remaining(), (int)getRemaining());
+ for (int i = 0; i < toRead; i++) {
+ buf.put(dataVal);
+ }
+ position += toRead;
+ return toRead;
+ };
+
+ @Override
+ protected int readWithStrategy(ByteReaderStrategy strategy) throws
+ IOException {
+ throw new IOException("Should not be called");
+ }
+
+ @Override
+ public void close() {
+ closed = true;
+ }
+
+ @Override
+ public void unbuffer() {
+ }
+
+ @Override
+ public void seek(long pos) {
+ this.position = pos;
+ }
+
+ @Override
+ public long getPos() {
+ return position;
+ }
+ }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStreamProxy.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStreamProxy.java
new file mode 100644
index 0000000..fd4e8ad
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockInputStreamProxy.java
@@ -0,0 +1,399 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.ozone.client.io.BadDataLocationException;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SplittableRandom;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+
+/**
+ * Unit tests for the ECBlockInputStreamProxy class.
+ */
+public class TestECBlockInputStreamProxy {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(TestECBlockInputStreamProxy.class);
+
+ private static final int ONEMB = 1024 * 1024;
+ private ECReplicationConfig repConfig;
+ private TestECBlockInputStreamFactory streamFactory;
+
+ private long randomSeed;
+ private ThreadLocalRandom random = ThreadLocalRandom.current();
+ private SplittableRandom dataGenerator;
+
+ @Before
+ public void setup() {
+ repConfig = new ECReplicationConfig(3, 2);
+ streamFactory = new TestECBlockInputStreamFactory();
+ randomSeed = random.nextLong();
+ dataGenerator = new SplittableRandom(randomSeed);
+ }
+
+ @Test
+ public void testExpectedDataLocations() {
+ Assert.assertEquals(1,
+ ECBlockInputStreamProxy.expectedDataLocations(repConfig, 1));
+ Assert.assertEquals(2,
+ ECBlockInputStreamProxy.expectedDataLocations(repConfig, ONEMB + 1));
+ Assert.assertEquals(3,
+ ECBlockInputStreamProxy.expectedDataLocations(repConfig, 3 * ONEMB));
+ Assert.assertEquals(3,
+ ECBlockInputStreamProxy.expectedDataLocations(repConfig, 10 * ONEMB));
+
+ repConfig = new ECReplicationConfig(6, 3);
+ Assert.assertEquals(1,
+ ECBlockInputStreamProxy.expectedDataLocations(repConfig, 1));
+ Assert.assertEquals(2,
+ ECBlockInputStreamProxy.expectedDataLocations(repConfig, ONEMB + 1));
+ Assert.assertEquals(3,
+ ECBlockInputStreamProxy.expectedDataLocations(repConfig, 3 * ONEMB));
+ Assert.assertEquals(6,
+ ECBlockInputStreamProxy.expectedDataLocations(repConfig, 10 * ONEMB));
+ }
+
+ @Test
+ public void testAvailableDataLocations() {
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ OmKeyLocationInfo blockInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, 1024, dnMap);
+ Assert.assertEquals(1, ECBlockInputStreamProxy.availableDataLocations(
+ blockInfo.getPipeline(), 1));
+ Assert.assertEquals(2, ECBlockInputStreamProxy.availableDataLocations(
+ blockInfo.getPipeline(), 2));
+ Assert.assertEquals(3, ECBlockInputStreamProxy.availableDataLocations(
+ blockInfo.getPipeline(), 3));
+
+ dnMap = ECStreamTestUtil.createIndexMap(1, 4, 5);
+ blockInfo = ECStreamTestUtil.createKeyInfo(repConfig, 1024, dnMap);
+ Assert.assertEquals(1, ECBlockInputStreamProxy.availableDataLocations(
+ blockInfo.getPipeline(), 3));
+
+ dnMap = ECStreamTestUtil.createIndexMap(2, 3, 4, 5);
+ blockInfo = ECStreamTestUtil.createKeyInfo(repConfig, 1024, dnMap);
+ Assert.assertEquals(0, ECBlockInputStreamProxy.availableDataLocations(
+ blockInfo.getPipeline(), 1));
+ }
+
+ @Test
+ public void testBlockIDCanBeRetrieved() throws IOException {
+ int blockLength = 1234;
+ generateData(blockLength);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ OmKeyLocationInfo blockInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+ try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+ Assert.assertEquals(blockInfo.getBlockID(), bis.getBlockID());
+ }
+ }
+
+ @Test
+ public void testBlockLengthCanBeRetrieved() throws IOException {
+ int blockLength = 1234;
+ generateData(blockLength);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ OmKeyLocationInfo blockInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+ try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+ Assert.assertEquals(1234, bis.getLength());
+ }
+ }
+
+ @Test
+ public void testBlockRemainingCanBeRetrieved() throws IOException {
+ int blockLength = 12345;
+ generateData(blockLength);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ OmKeyLocationInfo blockInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+ dataGenerator = new SplittableRandom(randomSeed);
+ ByteBuffer readBuffer = ByteBuffer.allocate(100);
+ try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+ Assert.assertEquals(12345, bis.getRemaining());
+ Assert.assertEquals(0, bis.getPos());
+ bis.read(readBuffer);
+ Assert.assertEquals(12345 - 100, bis.getRemaining());
+ Assert.assertEquals(100, bis.getPos());
+ }
+ }
+
+ @Test
+ public void testCorrectStreamCreatedDependingOnDataLocations()
+ throws IOException {
+ int blockLength = 5 * ONEMB;
+ ByteBuffer data = generateData(blockLength);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ OmKeyLocationInfo blockInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+ try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+ // Not all locations present, so we expect on;y the "missing=true" stream
+ // to be present.
+ Assert.assertTrue(streamFactory.getStreams().containsKey(false));
+ Assert.assertFalse(streamFactory.getStreams().containsKey(true));
+ }
+
+ streamFactory = new TestECBlockInputStreamFactory();
+ streamFactory.setData(data);
+ dnMap = ECStreamTestUtil.createIndexMap(2, 3, 4, 5);
+ blockInfo = ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+ try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+ // Not all locations present, so we expect on;y the "missing=true" stream
+ // to be present.
+ Assert.assertFalse(streamFactory.getStreams().containsKey(false));
+ Assert.assertTrue(streamFactory.getStreams().containsKey(true));
+ }
+ }
+
+ @Test
+ public void testCanReadNonReconstructionToEOF()
+ throws IOException {
+ int blockLength = 5 * ONEMB;
+ generateData(blockLength);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ OmKeyLocationInfo blockInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+ ByteBuffer readBuffer = ByteBuffer.allocate(100);
+ dataGenerator = new SplittableRandom(randomSeed);
+ try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+ while (true) {
+ int read = bis.read(readBuffer);
+ ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+ readBuffer.clear();
+ if (read < 100) {
+ break;
+ }
+ }
+ readBuffer.clear();
+ int read = bis.read(readBuffer);
+ Assert.assertEquals(-1, read);
+ }
+ }
+
+ @Test
+ public void testCanReadReconstructionToEOF()
+ throws IOException {
+ int blockLength = 5 * ONEMB;
+ generateData(blockLength);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(2, 3, 4, 5);
+ OmKeyLocationInfo blockInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+ ByteBuffer readBuffer = ByteBuffer.allocate(100);
+ dataGenerator = new SplittableRandom(randomSeed);
+ try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+ while (true) {
+ int read = bis.read(readBuffer);
+ ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+ readBuffer.clear();
+ if (read < 100) {
+ break;
+ }
+ }
+ readBuffer.clear();
+ int read = bis.read(readBuffer);
+ Assert.assertEquals(-1, read);
+ }
+ }
+
+ @Test
+ public void testCanHandleErrorAndFailOverToReconstruction()
+ throws IOException {
+ int blockLength = 5 * ONEMB;
+ generateData(blockLength);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ OmKeyLocationInfo blockInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+ ByteBuffer readBuffer = ByteBuffer.allocate(100);
+ DatanodeDetails badDN = blockInfo.getPipeline().getFirstNode();
+
+ dataGenerator = new SplittableRandom(randomSeed);
+ try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+ // Perform one read to get the stream created
+ int read = bis.read(readBuffer);
+ Assert.assertEquals(100, read);
+ ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+ // Setup an error to be thrown part through a read, so the dataBuffer
+ // will have been advanced by 50 bytes before the error. This tests it
+ // correctly rewinds and the same data is loaded again from the other
+ // stream.
+ streamFactory.getStreams().get(false).setShouldError(true, 151,
+ new BadDataLocationException(badDN, "Simulated Error"));
+ while (true) {
+ readBuffer.clear();
+ read = bis.read(readBuffer);
+ ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+ if (read < 100) {
+ break;
+ }
+ }
+ readBuffer.clear();
+ read = bis.read(readBuffer);
+ Assert.assertEquals(-1, read);
+ // Ensure the bad location was passed into the factory to create the
+ // reconstruction reader
+ Assert.assertEquals(badDN, streamFactory.getFailedLocations().get(0));
+ }
+ }
+
+ @Test
+ public void testCanSeekToNewPosition() throws IOException {
+ int blockLength = 5 * ONEMB;
+ generateData(blockLength);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ OmKeyLocationInfo blockInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+
+ ByteBuffer readBuffer = ByteBuffer.allocate(100);
+ dataGenerator = new SplittableRandom(randomSeed);
+ try (ECBlockInputStreamProxy bis = createBISProxy(repConfig, blockInfo)) {
+ // Perform one read to get the stream created
+ int read = bis.read(readBuffer);
+ Assert.assertEquals(100, read);
+
+ bis.seek(1024);
+ readBuffer.clear();
+ resetAndAdvanceDataGenerator(1024);
+ bis.read(readBuffer);
+ ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+ Assert.assertEquals(1124, bis.getPos());
+
+ // Set the non-reconstruction reader to thrown an exception on seek
+ streamFactory.getStreams().get(false).setShouldErrorOnSeek(true);
+ bis.seek(2048);
+ readBuffer.clear();
+ resetAndAdvanceDataGenerator(2048);
+ bis.read(readBuffer);
+ ECStreamTestUtil.assertBufferMatches(readBuffer, dataGenerator);
+
+ // Finally, set the recon reader to fail on seek.
+ streamFactory.getStreams().get(true).setShouldErrorOnSeek(true);
+ try {
+ bis.seek(1024);
+ Assert.fail("Seek should have raised an exception");
+ } catch (IOException e) {
+ // expected
+ }
+ }
+ }
+
+ private ByteBuffer generateData(int length) {
+ ByteBuffer data = ByteBuffer.allocate(length);
+ ECStreamTestUtil.randomFill(data, dataGenerator);
+ streamFactory.setData(data);
+ return data;
+ }
+
+ private void resetAndAdvanceDataGenerator(long position) {
+ dataGenerator = new SplittableRandom(randomSeed);
+ for (long i = 0; i < position; i++) {
+ dataGenerator.nextInt(255);
+ }
+ }
+
+ private ECBlockInputStreamProxy createBISProxy(ECReplicationConfig rConfig,
+ OmKeyLocationInfo blockInfo) {
+ return new ECBlockInputStreamProxy(
+ rConfig, blockInfo, true, null, null, streamFactory);
+ }
+
+ private static class TestECBlockInputStreamFactory
+ implements ECBlockInputStreamFactory {
+
+ private ByteBuffer data;
+
+ private Map<Boolean, ECStreamTestUtil.TestBlockInputStream> streams
+ = new HashMap<>();
+
+ private List<DatanodeDetails> failedLocations;
+
+ public void setData(ByteBuffer data) {
+ this.data = data;
+ }
+
+ public Map<Boolean, ECStreamTestUtil.TestBlockInputStream> getStreams() {
+ return streams;
+ }
+
+ public List<DatanodeDetails> getFailedLocations() {
+ return failedLocations;
+ }
+
+ @Override
+ public BlockExtendedInputStream create(boolean missingLocations,
+ List<DatanodeDetails> failedDatanodes,
+ ReplicationConfig repConfig, OmKeyLocationInfo blockInfo,
+ boolean verifyChecksum, XceiverClientFactory xceiverFactory,
+ Function<BlockID, Pipeline> refreshFunction) {
+ this.failedLocations = failedDatanodes;
+ ByteBuffer wrappedBuffer =
+ ByteBuffer.wrap(data.array(), 0, data.capacity());
+ ECStreamTestUtil.TestBlockInputStream is =
+ new ECStreamTestUtil.TestBlockInputStream(blockInfo.getBlockID(),
+ blockInfo.getLength(), wrappedBuffer);
+ streams.put(missingLocations, is);
+ return is;
+ }
+ }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedInputStream.java
new file mode 100644
index 0000000..ee3f50a
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedInputStream.java
@@ -0,0 +1,379 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.SplittableRandom;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.hadoop.ozone.client.rpc.read.ECStreamTestUtil.generateParity;
+
+/**
+ * Test for the ECBlockReconstructedInputStream class.
+ */
+public class TestECBlockReconstructedInputStream {
+
+ private ECReplicationConfig repConfig;
+ private ECStreamTestUtil.TestBlockInputStreamFactory streamFactory;
+ private long randomSeed;
+ private ThreadLocalRandom random = ThreadLocalRandom.current();
+ private SplittableRandom dataGenerator;
+ private ByteBufferPool bufferPool = new ElasticByteBufferPool();
+ private ExecutorService ecReconstructExecutor =
+ Executors.newFixedThreadPool(3);
+
+ @Before
+ public void setup() throws IOException {
+ repConfig = new ECReplicationConfig(3, 2);
+ streamFactory = new ECStreamTestUtil.TestBlockInputStreamFactory();
+
+ randomSeed = random.nextLong();
+ dataGenerator = new SplittableRandom(randomSeed);
+ }
+
+ @After
+ public void teardown() {
+ ecReconstructExecutor.shutdownNow();
+ }
+
+ private ECBlockReconstructedStripeInputStream createStripeInputStream(
+ Map<DatanodeDetails, Integer> dnMap, long blockLength) {
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+ return new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+ null, null, streamFactory, bufferPool, ecReconstructExecutor);
+ }
+
+ @Test
+ public void testBlockLengthReturned() throws IOException {
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(4, 5);
+ try (ECBlockReconstructedStripeInputStream stripeStream
+ = createStripeInputStream(dnMap, 12345L)) {
+ try (ECBlockReconstructedInputStream stream =
+ new ECBlockReconstructedInputStream(repConfig, bufferPool,
+ stripeStream)) {
+ Assert.assertEquals(12345L, stream.getLength());
+ }
+ }
+ }
+
+ @Test
+ public void testBlockIDReturned() throws IOException {
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 4, 5);
+ try (ECBlockReconstructedStripeInputStream stripeStream
+ = createStripeInputStream(dnMap, 12345L)) {
+ try (ECBlockReconstructedInputStream stream =
+ new ECBlockReconstructedInputStream(repConfig, bufferPool,
+ stripeStream)) {
+ Assert.assertEquals(new BlockID(1, 1), stream.getBlockID());
+ }
+ }
+ }
+
+ @Test
+ public void testReadDataByteBufferMultipleStripes() throws IOException {
+ int readBufferSize = random.nextInt(4096);
+ // 3 stripes and a partial chunk
+ int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+ + repConfig.getEcChunkSize() - 1;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+ repConfig.getEcChunkSize() * 4);
+ ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+ dataGenerator, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+ addDataStreamsToFactory(dataBufs, parity);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+ try (ECBlockReconstructedStripeInputStream stripeStream
+ = createStripeInputStream(dnMap, blockLength)) {
+ try (ECBlockReconstructedInputStream stream =
+ new ECBlockReconstructedInputStream(repConfig, bufferPool,
+ stripeStream)) {
+ ByteBuffer b = ByteBuffer.allocate(readBufferSize);
+ int totalRead = 0;
+ dataGenerator = new SplittableRandom(randomSeed);
+ while (totalRead < blockLength) {
+ int expectedRead = Math.min(blockLength - totalRead, readBufferSize);
+ long read = stream.read(b);
+ totalRead += read;
+ Assert.assertEquals(expectedRead, read);
+ ECStreamTestUtil.assertBufferMatches(b, dataGenerator);
+ b.clear();
+ }
+ // Next read should be EOF
+ b.clear();
+ long read = stream.read(b);
+ Assert.assertEquals(-1, read);
+ // Seek back to zero and read again to ensure the buffers are
+ // re-allocated after being freed at the end of block.
+ stream.seek(0);
+ read = stream.read(b);
+ Assert.assertEquals(readBufferSize, read);
+ dataGenerator = new SplittableRandom(randomSeed);
+ ECStreamTestUtil.assertBufferMatches(b, dataGenerator);
+ }
+ }
+ }
+
+
+ @Test
+ public void testReadDataWithUnbuffer() throws IOException {
+ // Read buffer is 16kb + 5 bytes so it does not align with stripes exactly
+ int readBufferSize = random.nextInt(1024 * 16 + 5);
+ // 3 stripes and a partial chunk
+ int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+ + repConfig.getEcChunkSize() - 1;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+ repConfig.getEcChunkSize() * 4);
+ ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+ dataGenerator, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+ addDataStreamsToFactory(dataBufs, parity);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+ try (ECBlockReconstructedStripeInputStream stripeStream
+ = createStripeInputStream(dnMap, blockLength)) {
+ try (ECBlockReconstructedInputStream stream =
+ new ECBlockReconstructedInputStream(repConfig, bufferPool,
+ stripeStream)) {
+ ByteBuffer b = ByteBuffer.allocate(readBufferSize);
+ int totalRead = 0;
+ dataGenerator = new SplittableRandom(randomSeed);
+ while (totalRead < blockLength) {
+ int expectedRead = Math.min(blockLength - totalRead, readBufferSize);
+ long read = stream.read(b);
+ totalRead += read;
+ Assert.assertEquals(expectedRead, read);
+ ECStreamTestUtil.assertBufferMatches(b, dataGenerator);
+ b.clear();
+ stream.unbuffer();
+ }
+ // Next read should be EOF
+ b.clear();
+ long read = stream.read(b);
+ Assert.assertEquals(-1, read);
+ }
+ }
+ }
+
+ @Test
+ public void testReadDataByteBufferUnderBufferSize() throws IOException {
+ int readBufferSize = 4096;
+ // Small block with less data that the read size
+ int blockLength = 1024;
+
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 1024);
+ ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+ dataGenerator, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+ addDataStreamsToFactory(dataBufs, parity);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+ try (ECBlockReconstructedStripeInputStream stripeStream
+ = createStripeInputStream(dnMap, blockLength)) {
+ try (ECBlockReconstructedInputStream stream =
+ new ECBlockReconstructedInputStream(repConfig, bufferPool,
+ stripeStream)) {
+ ByteBuffer b = ByteBuffer.allocate(readBufferSize);
+ dataGenerator = new SplittableRandom(randomSeed);
+ long read = stream.read(b);
+ Assert.assertEquals(blockLength, read);
+ ECStreamTestUtil.assertBufferMatches(b, dataGenerator);
+ b.clear();
+ // Next read should be EOF
+ read = stream.read(b);
+ Assert.assertEquals(-1, read);
+ }
+ }
+ }
+
+ @Test
+ public void testReadByteAtATime() throws IOException {
+ // 3 stripes and a partial chunk
+ int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+ + repConfig.getEcChunkSize() - 1;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+ repConfig.getEcChunkSize() * 4);
+ ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+ dataGenerator, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+ addDataStreamsToFactory(dataBufs, parity);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+ try (ECBlockReconstructedStripeInputStream stripeStream
+ = createStripeInputStream(dnMap, blockLength)) {
+ try (ECBlockReconstructedInputStream stream =
+ new ECBlockReconstructedInputStream(repConfig, bufferPool,
+ stripeStream)) {
+
+ dataGenerator = new SplittableRandom(randomSeed);
+ int totalRead = 0;
+ while (true) {
+ int val = stream.read();
+ if (val == -1) {
+ break;
+ }
+ Assert.assertEquals(dataGenerator.nextInt(255), val);
+ totalRead += 1;
+ }
+ Assert.assertEquals(blockLength, totalRead);
+ }
+ }
+ }
+
+ @Test
+ public void testReadByteBuffer() throws IOException {
+ // 3 stripes and a partial chunk
+ int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+ + repConfig.getEcChunkSize() - 1;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+ repConfig.getEcChunkSize() * 4);
+ ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+ dataGenerator, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+ addDataStreamsToFactory(dataBufs, parity);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+ byte[] buf = new byte[1024];
+ try (ECBlockReconstructedStripeInputStream stripeStream
+ = createStripeInputStream(dnMap, blockLength)) {
+ try (ECBlockReconstructedInputStream stream =
+ new ECBlockReconstructedInputStream(repConfig, bufferPool,
+ stripeStream)) {
+ int totalRead = 0;
+ dataGenerator = new SplittableRandom(randomSeed);
+ while (totalRead < blockLength) {
+ int expectedRead = Math.min(blockLength - totalRead, 1024);
+ long read = stream.read(buf, 0, buf.length);
+ totalRead += read;
+ Assert.assertEquals(expectedRead, read);
+ ECStreamTestUtil.assertBufferMatches(
+ ByteBuffer.wrap(buf, 0, (int)read), dataGenerator);
+ }
+ // Next read should be EOF
+ long read = stream.read(buf, 0, buf.length);
+ Assert.assertEquals(-1, read);
+ }
+ }
+ }
+
+ @Test
+ public void testSeek() throws IOException {
+ int readBufferSize = repConfig.getEcChunkSize() + 1024;
+ // 3 stripes and a partial chunk
+ int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+ + repConfig.getEcChunkSize() - 1;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+ repConfig.getEcChunkSize() * 4);
+ ECStreamTestUtil.randomFill(dataBufs, repConfig.getEcChunkSize(),
+ dataGenerator, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+ addDataStreamsToFactory(dataBufs, parity);
+
+ Map<DatanodeDetails, Integer> dnMap
+ = ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+ try (ECBlockReconstructedStripeInputStream stripeStream
+ = createStripeInputStream(dnMap, blockLength)) {
+ try (ECBlockReconstructedInputStream stream =
+ new ECBlockReconstructedInputStream(repConfig, bufferPool,
+ stripeStream)) {
+ ByteBuffer b = ByteBuffer.allocate(readBufferSize);
+
+ int seekPosition = 0;
+ for (int i = 0; i < 100; i++) {
+ resetAndAdvanceDataGenerator(seekPosition);
+ long expectedRead = Math.min(stream.getRemaining(), readBufferSize);
+ long read = stream.read(b);
+ Assert.assertEquals(expectedRead, read);
+ ECStreamTestUtil.assertBufferMatches(b, dataGenerator);
+ seekPosition = random.nextInt(blockLength);
+ stream.seek(seekPosition);
+ b.clear();
+ }
+ // Seeking beyond EOF should give an error
+ try {
+ stream.seek(blockLength + 1);
+ Assert.fail("Seek beyond EOF should error");
+ } catch (IOException e) {
+ // expected
+ }
+ }
+ }
+ }
+
+ private void resetAndAdvanceDataGenerator(long position) {
+ dataGenerator = new SplittableRandom(randomSeed);
+ for (long i = 0; i < position; i++) {
+ dataGenerator.nextInt(255);
+ }
+ }
+
+
+
+ /**
+ * Return a list of num ByteBuffers of the given size.
+ * @param num Number of buffers to create
+ * @param size The size of each buffer
+ * @return
+ */
+ private ByteBuffer[] allocateBuffers(int num, int size) {
+ ByteBuffer[] bufs = new ByteBuffer[num];
+ for (int i = 0; i < num; i++) {
+ bufs[i] = ByteBuffer.allocate(size);
+ }
+ return bufs;
+ }
+
+ private void addDataStreamsToFactory(ByteBuffer[] data, ByteBuffer[] parity) {
+ List<ByteBuffer> dataStreams = new ArrayList<>();
+ for (ByteBuffer b : data) {
+ dataStreams.add(b);
+ }
+ for (ByteBuffer b : parity) {
+ dataStreams.add(b);
+ }
+ streamFactory.setBlockStreamData(dataStreams);
+ }
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
new file mode 100644
index 0000000..0eff8f9
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestECBlockReconstructedStripeInputStream.java
@@ -0,0 +1,724 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStream;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.client.io.InsufficientLocationsException;
+import org.apache.hadoop.ozone.client.rpc.read.ECStreamTestUtil.TestBlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.rpc.read.ECStreamTestUtil.TestBlockInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.SplittableRandom;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.hadoop.ozone.client.rpc.read.ECStreamTestUtil.generateParity;
+
+/**
+ * Test for the ECBlockReconstructedStripeInputStream.
+ */
+public class TestECBlockReconstructedStripeInputStream {
+
+ private static final int ONEMB = 1024 * 1024;
+
+ private ECReplicationConfig repConfig;
+ private ECStreamTestUtil.TestBlockInputStreamFactory streamFactory;
+ private long randomSeed;
+ private ThreadLocalRandom random = ThreadLocalRandom.current();
+ private SplittableRandom dataGen;
+ private ByteBufferPool bufferPool = new ElasticByteBufferPool();
+ private ExecutorService ecReconstructExecutor =
+ Executors.newFixedThreadPool(3);
+
+ @Before
+ public void setup() {
+ repConfig = new ECReplicationConfig(3, 2,
+ ECReplicationConfig.EcCodec.RS, ONEMB);
+ streamFactory = new ECStreamTestUtil.TestBlockInputStreamFactory();
+
+ randomSeed = random.nextLong();
+ dataGen = new SplittableRandom(randomSeed);
+ }
+
+ @After
+ public void teardown() {
+ ecReconstructExecutor.shutdownNow();
+ }
+
+ @Test
+ public void testSufficientLocations() throws IOException {
+ // One chunk, only 1 location.
+ OmKeyLocationInfo keyInfo = ECStreamTestUtil
+ .createKeyInfo(repConfig, 1, ONEMB);
+ try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+ Assert.assertTrue(ecb.hasSufficientLocations());
+ }
+ // Two Chunks, but missing data block 2.
+ Map<DatanodeDetails, Integer> dnMap
+ = ECStreamTestUtil.createIndexMap(1, 4, 5);
+ keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB * 2, dnMap);
+ try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+ Assert.assertTrue(ecb.hasSufficientLocations());
+ }
+
+ // Three Chunks, but missing data block 2 and 3.
+ dnMap = ECStreamTestUtil.createIndexMap(1, 4, 5);
+ keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB * 3, dnMap);
+ try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+ Assert.assertTrue(ecb.hasSufficientLocations());
+ // Set a failed location
+ List<DatanodeDetails> failed = new ArrayList<>();
+ failed.add(keyInfo.getPipeline().getFirstNode());
+ ((ECBlockReconstructedStripeInputStream)ecb).addFailedDatanodes(failed);
+ Assert.assertFalse(ecb.hasSufficientLocations());
+ }
+
+ // Three Chunks, but missing data block 2 and 3 and parity 1.
+ dnMap = ECStreamTestUtil.createIndexMap(1, 4);
+ keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB * 3, dnMap);
+ try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+ Assert.assertFalse(ecb.hasSufficientLocations());
+ }
+
+ // Three Chunks, all available but fail 3
+ dnMap = ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB * 3, dnMap);
+ try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+ Assert.assertTrue(ecb.hasSufficientLocations());
+ // Set a failed location
+ List<DatanodeDetails> failed = new ArrayList<>();
+ for (DatanodeDetails dn : dnMap.keySet()) {
+ failed.add(dn);
+ if (failed.size() == 3) {
+ break;
+ }
+ }
+ ((ECBlockReconstructedStripeInputStream)ecb).addFailedDatanodes(failed);
+ Assert.assertFalse(ecb.hasSufficientLocations());
+ }
+
+ // One chunk, indexes 2 and 3 are padding, but still reported in the
+ // container list. The other locations are missing so we should have
+ // insufficient locations.
+ dnMap = ECStreamTestUtil.createIndexMap(2, 3);
+ keyInfo = ECStreamTestUtil.createKeyInfo(repConfig, ONEMB, dnMap);
+ try (ECBlockInputStream ecb = createInputStream(keyInfo)) {
+ Assert.assertFalse(ecb.hasSufficientLocations());
+ }
+ }
+
+ @Test
+ public void testReadFullStripesWithPartial() throws IOException {
+ // Generate the input data for 3 full stripes and generate the parity.
+ int chunkSize = repConfig.getEcChunkSize();
+ int partialStripeSize = chunkSize * 2 - 1;
+ int blockLength = chunkSize * repConfig.getData() * 3 + partialStripeSize;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 4 * chunkSize);
+ ECStreamTestUtil.randomFill(dataBufs, chunkSize, dataGen, blockLength);
+
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+ List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+ // Two data missing
+ locations.add(ECStreamTestUtil.createIndexMap(1, 4, 5));
+ // One data missing
+ locations.add(ECStreamTestUtil.createIndexMap(1, 2, 4, 5));
+ // Two data missing including first
+ locations.add(ECStreamTestUtil.createIndexMap(2, 4, 5));
+ // One data and one parity missing
+ locations.add(ECStreamTestUtil.createIndexMap(2, 3, 4));
+ // No missing indexes
+ locations.add(ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5));
+
+ for (Map<DatanodeDetails, Integer> dnMap : locations) {
+ streamFactory = new TestBlockInputStreamFactory();
+ addDataStreamsToFactory(dataBufs, parity);
+
+ OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+ stripeSize() * 3 + partialStripeSize, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+ ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+ dataGen = new SplittableRandom(randomSeed);
+ try (ECBlockReconstructedStripeInputStream ecb =
+ createInputStream(keyInfo)) {
+ // Read 3 full stripes
+ for (int i = 0; i < 3; i++) {
+ int read = ecb.readStripe(bufs);
+ for (int j = 0; j < bufs.length; j++) {
+ ECStreamTestUtil.assertBufferMatches(bufs[j], dataGen);
+ }
+ Assert.assertEquals(stripeSize(), read);
+
+ // Check the underlying streams have read 1 chunk per read:
+ for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+ Assert.assertEquals(chunkSize * (i + 1),
+ bis.getPos());
+ }
+ Assert.assertEquals(stripeSize() * (i + 1), ecb.getPos());
+ clearBuffers(bufs);
+ }
+ // The next read is a partial stripe
+ int read = ecb.readStripe(bufs);
+ Assert.assertEquals(partialStripeSize, read);
+ ECStreamTestUtil.assertBufferMatches(bufs[0], dataGen);
+ ECStreamTestUtil.assertBufferMatches(bufs[1], dataGen);
+ Assert.assertEquals(0, bufs[2].remaining());
+ Assert.assertEquals(0, bufs[2].position());
+
+ // A further read should give EOF
+ clearBuffers(bufs);
+ read = ecb.readStripe(bufs);
+ Assert.assertEquals(-1, read);
+ }
+ }
+ }
+
+ @Test
+ public void testReadPartialStripe() throws IOException {
+ int blockLength = repConfig.getEcChunkSize() - 1;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+ ECStreamTestUtil
+ .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+ addDataStreamsToFactory(dataBufs, parity);
+
+ ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+ // We have a length that is less than a single chunk, so blocks 2 and 3
+ // are padding and will not be present. Block 1 is lost and needs recovered
+ // from the parity and padded blocks 2 and 3.
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(4, 5);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+ dataGen = new SplittableRandom(randomSeed);
+ try (ECBlockReconstructedStripeInputStream ecb =
+ createInputStream(keyInfo)) {
+ int read = ecb.readStripe(bufs);
+ Assert.assertEquals(blockLength, read);
+ ECStreamTestUtil.assertBufferMatches(bufs[0], dataGen);
+ Assert.assertEquals(0, bufs[1].remaining());
+ Assert.assertEquals(0, bufs[1].position());
+ Assert.assertEquals(0, bufs[2].remaining());
+ Assert.assertEquals(0, bufs[2].position());
+ // Check the underlying streams have been advanced by 1 blockLength:
+ for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+ Assert.assertEquals(blockLength, bis.getPos());
+ }
+ Assert.assertEquals(ecb.getPos(), blockLength);
+ clearBuffers(bufs);
+ // A further read should give EOF
+ read = ecb.readStripe(bufs);
+ Assert.assertEquals(-1, read);
+ }
+ }
+
+ @Test
+ public void testReadPartialStripeTwoChunks() throws IOException {
+ int chunkSize = repConfig.getEcChunkSize();
+ int blockLength = chunkSize * 2 - 1;
+
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+ ECStreamTestUtil
+ .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+ addDataStreamsToFactory(dataBufs, parity);
+
+ ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+ // We have a length that is less than a single chunk, so blocks 2 and 3
+ // are padding and will not be present. Block 1 is lost and needs recovered
+ // from the parity and padded blocks 2 and 3.
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(4, 5);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+ dataGen = new SplittableRandom(randomSeed);
+ try (ECBlockReconstructedStripeInputStream ecb =
+ createInputStream(keyInfo)) {
+ int read = ecb.readStripe(bufs);
+ Assert.assertEquals(blockLength, read);
+ ECStreamTestUtil.assertBufferMatches(bufs[0], dataGen);
+ ECStreamTestUtil.assertBufferMatches(bufs[1], dataGen);
+ Assert.assertEquals(0, bufs[2].remaining());
+ Assert.assertEquals(0, bufs[2].position());
+ // Check the underlying streams have been advanced by 1 chunk:
+ for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+ Assert.assertEquals(chunkSize, bis.getPos());
+ }
+ Assert.assertEquals(ecb.getPos(), blockLength);
+ clearBuffers(bufs);
+ // A further read should give EOF
+ read = ecb.readStripe(bufs);
+ Assert.assertEquals(-1, read);
+ }
+ }
+
+ @Test
+ public void testReadPartialStripeThreeChunks() throws IOException {
+ int chunkSize = repConfig.getEcChunkSize();
+ int blockLength = chunkSize * 3 - 1;
+
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+ ECStreamTestUtil
+ .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+ // We have a length that is less than a stripe, so chunks 1 and 2 are full.
+ // Block 1 is lost and needs recovered
+ // from the parity and padded blocks 2 and 3.
+
+ List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+ // Two data missing
+ locations.add(ECStreamTestUtil.createIndexMap(3, 4, 5));
+ // Two data missing
+ locations.add(ECStreamTestUtil.createIndexMap(1, 4, 5));
+ // One data missing - the last one
+ locations.add(ECStreamTestUtil.createIndexMap(1, 2, 5));
+ // One data and one parity missing
+ locations.add(ECStreamTestUtil.createIndexMap(2, 3, 4));
+ // One data and one parity missing
+ locations.add(ECStreamTestUtil.createIndexMap(1, 2, 4));
+ // No indexes missing
+ locations.add(ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5));
+
+ for (Map<DatanodeDetails, Integer> dnMap : locations) {
+ streamFactory = new TestBlockInputStreamFactory();
+ addDataStreamsToFactory(dataBufs, parity);
+ ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+ dataGen = new SplittableRandom(randomSeed);
+ try (ECBlockReconstructedStripeInputStream ecb =
+ createInputStream(keyInfo)) {
+ int read = ecb.readStripe(bufs);
+ Assert.assertEquals(blockLength, read);
+ ECStreamTestUtil.assertBufferMatches(bufs[0], dataGen);
+ ECStreamTestUtil.assertBufferMatches(bufs[1], dataGen);
+ ECStreamTestUtil.assertBufferMatches(bufs[2], dataGen);
+ // Check the underlying streams have been advanced by 1 chunk:
+ for (TestBlockInputStream bis : streamFactory.getBlockStreams()) {
+ Assert.assertEquals(0, bis.getRemaining());
+ }
+ Assert.assertEquals(ecb.getPos(), blockLength);
+ clearBuffers(bufs);
+ // A further read should give EOF
+ read = ecb.readStripe(bufs);
+ Assert.assertEquals(-1, read);
+ }
+ }
+ }
+
+ @Test
+ public void testErrorThrownIfBlockNotLongEnough() throws IOException {
+ int blockLength = repConfig.getEcChunkSize() - 1;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 3 * ONEMB);
+ ECStreamTestUtil
+ .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+ addDataStreamsToFactory(dataBufs, parity);
+
+ // Set the parity buffer limit to be less than the block length
+ parity[0].limit(blockLength - 1);
+ parity[1].limit(blockLength - 1);
+
+ ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+ // We have a length that is less than a single chunk, so blocks 2 and 3
+ // are padding and will not be present. Block 1 is lost and needs recovered
+ // from the parity and padded blocks 2 and 3.
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(4, 5);
+ OmKeyLocationInfo keyInfo =
+ ECStreamTestUtil.createKeyInfo(repConfig, blockLength, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+ try (ECBlockReconstructedStripeInputStream ecb =
+ createInputStream(keyInfo)) {
+ try {
+ ecb.readStripe(bufs);
+ Assert.fail("Read should have thrown an exception");
+ } catch (InsufficientLocationsException e) {
+ // expected
+ }
+ }
+ }
+
+ @Test
+ public void testSeek() throws IOException {
+ // Generate the input data for 3 full stripes and generate the parity
+ // and a partial stripe
+ int chunkSize = repConfig.getEcChunkSize();
+ int partialStripeSize = chunkSize * 2 - 1;
+ int dataLength = stripeSize() * 3 + partialStripeSize;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 4 * chunkSize);
+ ECStreamTestUtil
+ .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, dataLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+ List<Map<DatanodeDetails, Integer>> locations = new ArrayList<>();
+ // Two data missing
+ locations.add(ECStreamTestUtil.createIndexMap(1, 4, 5));
+ // One data missing
+ locations.add(ECStreamTestUtil.createIndexMap(1, 2, 4, 5));
+ // Two data missing including first
+ locations.add(ECStreamTestUtil.createIndexMap(2, 4, 5));
+ // One data and one parity missing
+ locations.add(ECStreamTestUtil.createIndexMap(2, 3, 4));
+ // No locations missing
+ locations.add(ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5));
+
+ for (Map<DatanodeDetails, Integer> dnMap : locations) {
+ streamFactory = new TestBlockInputStreamFactory();
+ addDataStreamsToFactory(dataBufs, parity);
+
+ OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+ stripeSize() * 3 + partialStripeSize, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+ ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+ try (ECBlockReconstructedStripeInputStream ecb =
+ createInputStream(keyInfo)) {
+
+ // Read Stripe 1
+ int read = ecb.readStripe(bufs);
+ for (int j = 0; j < bufs.length; j++) {
+ validateContents(dataBufs[j], bufs[j], 0, chunkSize);
+ }
+ Assert.assertEquals(stripeSize(), read);
+ Assert.assertEquals(dataLength - stripeSize(), ecb.getRemaining());
+
+ // Seek to 0 and read again
+ clearBuffers(bufs);
+ ecb.seek(0);
+ ecb.readStripe(bufs);
+ for (int j = 0; j < bufs.length; j++) {
+ validateContents(dataBufs[j], bufs[j], 0, chunkSize);
+ }
+ Assert.assertEquals(stripeSize(), read);
+ Assert.assertEquals(dataLength - stripeSize(), ecb.getRemaining());
+
+ // Seek to the last stripe
+ // Seek to the last stripe
+ clearBuffers(bufs);
+ ecb.seek(stripeSize() * 3);
+ read = ecb.readStripe(bufs);
+ validateContents(dataBufs[0], bufs[0], 3 * chunkSize, chunkSize);
+ validateContents(dataBufs[1], bufs[1], 3 * chunkSize, chunkSize - 1);
+ Assert.assertEquals(0, bufs[2].remaining());
+ Assert.assertEquals(partialStripeSize, read);
+ Assert.assertEquals(0, ecb.getRemaining());
+
+ // seek to the start of stripe 3
+ clearBuffers(bufs);
+ ecb.seek(stripeSize() * (long)2);
+ read = ecb.readStripe(bufs);
+ for (int j = 0; j < bufs.length; j++) {
+ validateContents(dataBufs[j], bufs[j], 2 * chunkSize, chunkSize);
+ }
+ Assert.assertEquals(stripeSize(), read);
+ Assert.assertEquals(partialStripeSize, ecb.getRemaining());
+ }
+ }
+ }
+
+ @Test
+ public void testSeekToPartialOffsetFails() {
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 4, 5);
+ OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+ stripeSize() * 3, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+ try (ECBlockReconstructedStripeInputStream ecb =
+ createInputStream(keyInfo)) {
+ try {
+ ecb.seek(10);
+ Assert.fail("Seek should have thrown an exception");
+ } catch (IOException e) {
+ Assert.assertEquals("Requested position 10 does not align " +
+ "with a stripe offset", e.getMessage());
+ }
+ }
+ }
+
+ private Integer getRandomStreamIndex(Set<Integer> set) {
+ return set.stream().skip(new Random().nextInt(set.size()))
+ .findFirst().orElse(null);
+ }
+
+ @Test
+ public void testErrorReadingBlockContinuesReading() throws IOException {
+ // Generate the input data for 3 full stripes and generate the parity.
+ int chunkSize = repConfig.getEcChunkSize();
+ int partialStripeSize = chunkSize * 2 - 1;
+ int blockLength = repConfig.getEcChunkSize() * repConfig.getData() * 3
+ + partialStripeSize;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(),
+ 4 * chunkSize);
+ ECStreamTestUtil
+ .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+ for (int k = 0; k < 5; k++) {
+ Set<Integer> failed = new HashSet<>();
+ streamFactory = new TestBlockInputStreamFactory();
+ addDataStreamsToFactory(dataBufs, parity);
+
+ // Data block index 3 is missing and needs recovered initially.
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 4, 5);
+ OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+ stripeSize() * 3 + partialStripeSize, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+ ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+ try (ECBlockReconstructedStripeInputStream ecb =
+ createInputStream(keyInfo)) {
+ // After reading the first stripe, make one of the streams error
+ for (int i = 0; i < 3; i++) {
+ int read = ecb.readStripe(bufs);
+ for (int j = 0; j < bufs.length; j++) {
+ validateContents(dataBufs[j], bufs[j], i * chunkSize, chunkSize);
+ }
+ Assert.assertEquals(stripeSize() * (i + 1), ecb.getPos());
+ Assert.assertEquals(stripeSize(), read);
+ clearBuffers(bufs);
+ if (i == 0) {
+ Integer failStream =
+ getRandomStreamIndex(streamFactory.getStreamIndexes());
+ streamFactory.getBlockStream(failStream)
+ .setShouldError(true);
+ failed.add(failStream);
+ }
+ }
+ // The next read is a partial stripe
+ int read = ecb.readStripe(bufs);
+ Assert.assertEquals(partialStripeSize, read);
+ validateContents(dataBufs[0], bufs[0], 3 * chunkSize, chunkSize);
+ validateContents(dataBufs[1], bufs[1], 3 * chunkSize, chunkSize - 1);
+ Assert.assertEquals(0, bufs[2].remaining());
+ Assert.assertEquals(0, bufs[2].position());
+
+ // seek back to zero and read a stripe to re-open the streams
+ ecb.seek(0);
+ clearBuffers(bufs);
+ ecb.readStripe(bufs);
+ // Now fail another random stream and the read should fail with
+ // insufficient locations
+ Set<Integer> currentStreams =
+ new HashSet<>(streamFactory.getStreamIndexes());
+ currentStreams.removeAll(failed);
+ Integer failStream = getRandomStreamIndex(currentStreams);
+ streamFactory.getBlockStream(failStream)
+ .setShouldError(true);
+ try {
+ clearBuffers(bufs);
+ ecb.readStripe(bufs);
+ Assert.fail("InsufficientLocationsException expected");
+ } catch (InsufficientLocationsException e) {
+ // expected
+ }
+ }
+ }
+ }
+
+ @Test(expected = InsufficientLocationsException.class)
+ public void testAllLocationsFailOnFirstRead() throws IOException {
+ // This test simulates stale nodes. When the nodes are stale, but not yet
+ // dead, the locations will still be given to the client and it will try to
+ // read them, but the read will always fail.
+ // Additionally, if the key is small (less than 2 EC chunks), the locations
+ // for the indexes which are all padding will be returned to the client and
+ // this can confuse the "sufficient locations" check, resulting in a strange
+ // error when selecting parity indexes (HDDS-6258)
+ int chunkSize = repConfig.getEcChunkSize();
+ int partialStripeSize = chunkSize;
+ int blockLength = partialStripeSize;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), chunkSize);
+ ECStreamTestUtil
+ .randomFill(dataBufs, repConfig.getEcChunkSize(), dataGen, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+ streamFactory = new TestBlockInputStreamFactory();
+ addDataStreamsToFactory(dataBufs, parity);
+ // Fail all the indexes containing data on their first read.
+ streamFactory.setFailIndexes(indexesToList(1, 4, 5));
+ // The locations contain the padded indexes, as will often be the case
+ // when containers are reported by SCM.
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+ blockLength, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+ ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+ try (ECBlockReconstructedStripeInputStream ecb =
+ createInputStream(keyInfo)) {
+ ecb.readStripe(bufs);
+ }
+ }
+
+ @Test
+ public void testFailedLocationsAreNotRead() throws IOException {
+ // Generate the input data for 3 full stripes and generate the parity.
+ int chunkSize = repConfig.getEcChunkSize();
+ int partialStripeSize = chunkSize * 2 - 1;
+ int blockLength = chunkSize * repConfig.getData() * 3 + partialStripeSize;
+ ByteBuffer[] dataBufs = allocateBuffers(repConfig.getData(), 4 * chunkSize);
+ ECStreamTestUtil.randomFill(dataBufs, chunkSize, dataGen, blockLength);
+ ByteBuffer[] parity = generateParity(dataBufs, repConfig);
+
+ streamFactory = new TestBlockInputStreamFactory();
+ addDataStreamsToFactory(dataBufs, parity);
+
+ Map<DatanodeDetails, Integer> dnMap =
+ ECStreamTestUtil.createIndexMap(1, 2, 3, 4, 5);
+ OmKeyLocationInfo keyInfo = ECStreamTestUtil.createKeyInfo(repConfig,
+ stripeSize() * 3 + partialStripeSize, dnMap);
+ streamFactory.setCurrentPipeline(keyInfo.getPipeline());
+
+ ByteBuffer[] bufs = allocateByteBuffers(repConfig);
+ dataGen = new SplittableRandom(randomSeed);
+ try (ECBlockReconstructedStripeInputStream ecb =
+ createInputStream(keyInfo)) {
+ List<DatanodeDetails> failed = new ArrayList<>();
+ // Set the first 3 DNs as failed
+ for (Map.Entry<DatanodeDetails, Integer> e : dnMap.entrySet()) {
+ if (e.getValue() <= 2) {
+ failed.add(e.getKey());
+ }
+ }
+ ecb.addFailedDatanodes(failed);
+
+ // Read full stripe
+ int read = ecb.readStripe(bufs);
+ for (int j = 0; j < bufs.length; j++) {
+ ECStreamTestUtil.assertBufferMatches(bufs[j], dataGen);
+ }
+ Assert.assertEquals(stripeSize(), read);
+
+ // Now ensure that streams with repIndexes 1, 2 and 3 have not been
+ // created in the stream factory, indicating we did not read them.
+ List<TestBlockInputStream> streams = streamFactory.getBlockStreams();
+ for (TestBlockInputStream stream : streams) {
+ Assert.assertTrue(stream.getEcReplicaIndex() > 2);
+ }
+ }
+ }
+
+ private ECBlockReconstructedStripeInputStream createInputStream(
+ OmKeyLocationInfo keyInfo) {
+ return new ECBlockReconstructedStripeInputStream(repConfig, keyInfo, true,
+ null, null, streamFactory, bufferPool, ecReconstructExecutor);
+ }
+
+ private List<Integer> indexesToList(int... indexes) {
+ List<Integer> list = new ArrayList<>();
+ for (int i : indexes) {
+ list.add(i);
+ }
+ return list;
+ }
+
+ private void addDataStreamsToFactory(ByteBuffer[] data, ByteBuffer[] parity) {
+ List<ByteBuffer> dataStreams = new ArrayList<>();
+ for (ByteBuffer b : data) {
+ dataStreams.add(b);
+ }
+ for (ByteBuffer b : parity) {
+ dataStreams.add(b);
+ }
+ streamFactory.setBlockStreamData(dataStreams);
+ }
+
+ /**
+ * Validates that the data buffer has the same contents as the source buffer,
+ * starting the checks in the src at offset and for count bytes.
+ * @param src The source of the data
+ * @param data The data which should be checked against the source
+ * @param offset The starting point in the src buffer
+ * @param count How many bytes to check.
+ */
+ private void validateContents(ByteBuffer src, ByteBuffer data, int offset,
+ int count) {
+ byte[] srcArray = src.array();
+ Assert.assertEquals(count, data.remaining());
+ for (int i = offset; i < offset + count; i++) {
+ Assert.assertEquals("Element " + i, srcArray[i], data.get());
+ }
+ data.flip();
+ }
+
+ /**
+ * Return a list of num ByteBuffers of the given size.
+ * @param num Number of buffers to create
+ * @param size The size of each buffer
+ * @return
+ */
+ private ByteBuffer[] allocateBuffers(int num, int size) {
+ ByteBuffer[] bufs = new ByteBuffer[num];
+ for (int i = 0; i < num; i++) {
+ bufs[i] = ByteBuffer.allocate(size);
+ }
+ return bufs;
+ }
+
+ private int stripeSize() {
+ return stripeSize(repConfig);
+ }
+
+ private int stripeSize(ECReplicationConfig rconfig) {
+ return rconfig.getEcChunkSize() * rconfig.getData();
+ }
+
+ private void clearBuffers(ByteBuffer[] bufs) {
+ for (ByteBuffer b : bufs) {
+ b.clear();
+ }
+ }
+
+ private ByteBuffer[] allocateByteBuffers(ECReplicationConfig rConfig) {
+ ByteBuffer[] bufs = new ByteBuffer[repConfig.getData()];
+ for (int i = 0; i < bufs.length; i++) {
+ bufs[i] = ByteBuffer.allocate(rConfig.getEcChunkSize());
+ }
+ return bufs;
+ }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java
index 7f0ab38..07c2433 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java
@@ -23,7 +23,8 @@
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.StorageUnit;
-import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
@@ -48,6 +49,7 @@
import org.junit.runners.Parameterized;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
@@ -114,7 +116,7 @@
conf.setFromObject(repConf);
cluster = MiniOzoneCluster.newBuilder(conf)
- .setNumDatanodes(4)
+ .setNumDatanodes(5)
.setTotalPipelineNumLimit(5)
.setBlockSize(BLOCK_SIZE)
.setChunkSize(CHUNK_SIZE)
@@ -169,8 +171,14 @@
}
byte[] writeKey(String keyName, int dataLength) throws Exception {
+ ReplicationConfig repConfig = RatisReplicationConfig.getInstance(THREE);
+ return writeKey(keyName, repConfig, dataLength);
+ }
+
+ byte[] writeKey(String keyName, ReplicationConfig repConfig, int dataLength)
+ throws Exception {
OzoneOutputStream key = TestHelper.createKey(keyName,
- ReplicationType.RATIS, 0, objectStore, volumeName, bucketName);
+ repConfig, 0, objectStore, volumeName, bucketName);
byte[] inputData = ContainerTestHelper.getFixedLengthString(
keyString, dataLength).getBytes(UTF_8);
@@ -182,8 +190,15 @@
byte[] writeRandomBytes(String keyName, int dataLength)
throws Exception {
+ ReplicationConfig repConfig = RatisReplicationConfig.getInstance(THREE);
+ return writeRandomBytes(keyName, repConfig, dataLength);
+ }
+
+ byte[] writeRandomBytes(String keyName, ReplicationConfig repConfig,
+ int dataLength)
+ throws Exception {
OzoneOutputStream key = TestHelper.createKey(keyName,
- ReplicationType.RATIS, 0, objectStore, volumeName, bucketName);
+ repConfig, 0, objectStore, volumeName, bucketName);
byte[] inputData = new byte[dataLength];
RAND.nextBytes(inputData);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java
index 7f9dd70..442cbfe 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java
@@ -23,8 +23,10 @@
import java.util.Arrays;
import java.util.List;
+import java.util.Random;
import java.util.concurrent.TimeoutException;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -33,6 +35,7 @@
import org.apache.hadoop.hdds.scm.XceiverClientMetrics;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
import org.apache.hadoop.hdds.scm.storage.BlockInputStream;
import org.apache.hadoop.hdds.scm.storage.ChunkInputStream;
import org.apache.hadoop.ozone.client.io.KeyInputStream;
@@ -50,6 +53,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static org.apache.hadoop.hdds.client.ECReplicationConfig.EcCodec.RS;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
import static org.apache.hadoop.ozone.container.TestHelper.countReplicas;
import static org.junit.Assert.fail;
@@ -95,6 +99,24 @@
}
/**
+ * This method does random seeks and reads and validates the reads are
+ * correct or not.
+ * @param dataLength
+ * @param keyInputStream
+ * @param inputData
+ * @param readSize
+ * @throws Exception
+ */
+ private void randomPositionSeek(int dataLength, KeyInputStream keyInputStream,
+ byte[] inputData, int readSize) throws Exception {
+ Random rand = new Random();
+ for (int i = 0; i < 100; i++) {
+ int position = rand.nextInt(dataLength - readSize);
+ validate(keyInputStream, inputData, position, readSize);
+ }
+ }
+
+ /**
* This method seeks to specified seek value and read the data specified by
* readLength and validate the read is correct or not.
* @param keyInputStream
@@ -125,6 +147,7 @@
testReadChunkWithByteArray();
testReadChunkWithByteBuffer();
testSkip();
+ testECSeek();
}
public void testInputStreams() throws Exception {
@@ -137,11 +160,13 @@
// Verify BlockStreams and ChunkStreams
int expectedNumBlockStreams = BufferUtils.getNumberOfBins(
dataLength, BLOCK_SIZE);
- List<BlockInputStream> blockStreams = keyInputStream.getBlockStreams();
+ List<BlockExtendedInputStream> blockStreams =
+ keyInputStream.getBlockStreams();
Assert.assertEquals(expectedNumBlockStreams, blockStreams.size());
int readBlockLength = 0;
- for (BlockInputStream blockStream : blockStreams) {
+ for (BlockExtendedInputStream stream : blockStreams) {
+ BlockInputStream blockStream = (BlockInputStream) stream;
int blockStreamLength = Math.min(BLOCK_SIZE,
dataLength - readBlockLength);
Assert.assertEquals(blockStreamLength, blockStream.getLength());
@@ -195,6 +220,36 @@
keyInputStream.close();
}
+ public void testECSeek() throws Exception {
+ int ecChunkSize = 1024 * 1024;
+ ECReplicationConfig repConfig = new ECReplicationConfig(3, 2, RS,
+ ecChunkSize);
+ String keyName = getNewKeyName();
+ // 3 full EC blocks plus one chunk
+ int dataLength = (9 * BLOCK_SIZE + ecChunkSize);
+
+ byte[] inputData = writeRandomBytes(keyName, repConfig, dataLength);
+ try (KeyInputStream keyInputStream = getKeyInputStream(keyName)) {
+
+ validate(keyInputStream, inputData, 0, ecChunkSize + 1234);
+
+ validate(keyInputStream, inputData, 200, ecChunkSize);
+
+ validate(keyInputStream, inputData, BLOCK_SIZE * 4, ecChunkSize);
+
+ validate(keyInputStream, inputData, BLOCK_SIZE * 4 + 200, ecChunkSize);
+
+ validate(keyInputStream, inputData, dataLength - ecChunkSize - 100,
+ ecChunkSize + 50);
+
+ randomPositionSeek(dataLength, keyInputStream, inputData,
+ ecChunkSize + 200);
+
+ // Read entire key.
+ validate(keyInputStream, inputData, 0, dataLength);
+ }
+ }
+
public void testSeek() throws Exception {
XceiverClientManager.resetXceiverClientMetrics();
XceiverClientMetrics metrics = XceiverClientManager
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStreamEC.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStreamEC.java
new file mode 100644
index 0000000..fa6688f
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStreamEC.java
@@ -0,0 +1,118 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.rpc.read;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockExtendedInputStream;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.KeyInputStream;
+import org.apache.hadoop.ozone.client.io.LengthInputStream;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.MB;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test KeyInputStream with EC keys.
+ */
+public class TestKeyInputStreamEC {
+
+ @Test
+ public void testReadAgainstLargeBlockGroup() throws IOException {
+ int dataBlocks = 10;
+ int parityBlocks = 4;
+ ECReplicationConfig ec10And4RepConfig = new ECReplicationConfig(dataBlocks,
+ parityBlocks, ECReplicationConfig.EcCodec.RS, (int)(1 * MB));
+ // default blockSize of 256MB with EC 10+4 makes a large block group
+ long blockSize = 256 * MB;
+ long blockLength = dataBlocks * blockSize;
+ OmKeyInfo keyInfo = createOmKeyInfo(ec10And4RepConfig,
+ dataBlocks + parityBlocks, blockLength);
+
+ BlockExtendedInputStream blockInputStream =
+ new ECStreamTestUtil.TestBlockInputStream(new BlockID(1, 1),
+ blockLength, ByteBuffer.allocate(100));
+
+ BlockInputStreamFactory mockStreamFactory =
+ mock(BlockInputStreamFactory.class);
+ when(mockStreamFactory.create(any(), any(), any(), any(),
+ anyBoolean(), any(), any())).thenReturn(blockInputStream);
+
+ try (LengthInputStream kis = KeyInputStream.getFromOmKeyInfo(keyInfo,
+ null, true, null, mockStreamFactory)) {
+ byte[] buf = new byte[100];
+ int readBytes = kis.read(buf, 0, 100);
+ Assert.assertEquals(100, readBytes);
+ }
+ }
+
+ private OmKeyInfo createOmKeyInfo(ReplicationConfig repConf,
+ int nodeCount, long blockLength) {
+ Map<DatanodeDetails, Integer> dnMap = new HashMap<>();
+ for (int i = 0; i < nodeCount; i++) {
+ dnMap.put(MockDatanodeDetails.randomDatanodeDetails(), i + 1);
+ }
+
+ Pipeline pipeline = Pipeline.newBuilder()
+ .setState(Pipeline.PipelineState.CLOSED)
+ .setId(PipelineID.randomId())
+ .setNodes(new ArrayList<>(dnMap.keySet()))
+ .setReplicaIndexes(dnMap)
+ .setReplicationConfig(repConf)
+ .build();
+
+ OmKeyLocationInfo blockInfo = new OmKeyLocationInfo.Builder()
+ .setBlockID(new BlockID(1, 1))
+ .setLength(blockLength)
+ .setOffset(0)
+ .setPipeline(pipeline)
+ .setPartNumber(0)
+ .build();
+
+ List<OmKeyLocationInfo> locations = new ArrayList<>();
+ locations.add(blockInfo);
+ return new OmKeyInfo.Builder()
+ .setBucketName("bucket")
+ .setVolumeName("volume")
+ .setDataSize(blockLength)
+ .setKeyName("someKey")
+ .setReplicationConfig(repConf)
+ .addOmKeyLocationInfoGroup(new OmKeyLocationInfoGroup(0, locations))
+ .build();
+ }
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java
index 7a4e1c3..dae6e38 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java
@@ -28,6 +28,8 @@
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -145,6 +147,13 @@
.createKey(keyName, size, type, factor, new HashMap<>());
}
+ public static OzoneOutputStream createKey(String keyName,
+ ReplicationConfig replicationConfig, long size, ObjectStore objectStore,
+ String volumeName, String bucketName) throws Exception {
+ return objectStore.getVolume(volumeName).getBucket(bucketName)
+ .createKey(keyName, size, replicationConfig, new HashMap<>());
+ }
+
public static void validateData(String keyName, byte[] data,
ObjectStore objectStore, String volumeName, String bucketName)
throws Exception {
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java
index d142384..b9eb231 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.ozone.freon;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -27,6 +25,7 @@
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.junit.Assert;
import org.junit.Test;
+import picocli.CommandLine;
import java.time.Duration;
@@ -64,17 +63,19 @@
}
@Test
- public void ratisTestLargeKey() throws Exception {
+ public void ratisTestLargeKey() {
RandomKeyGenerator randomKeyGenerator =
- new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(1);
- randomKeyGenerator.setNumOfBuckets(1);
- randomKeyGenerator.setNumOfKeys(1);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.setKeySize(20971520);
- randomKeyGenerator.setValidateWrites(true);
- randomKeyGenerator.call();
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "1",
+ "--num-of-buckets", "1",
+ "--num-of-keys", "1",
+ "--key-size", "20971520",
+ "--factor", "THREE",
+ "--type", "RATIS",
+ "--validate-writes"
+ );
+
Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
@@ -82,16 +83,18 @@
}
@Test
- public void validateWriteTest() throws Exception {
+ public void validateWriteTest() {
RandomKeyGenerator randomKeyGenerator =
- new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(2);
- randomKeyGenerator.setNumOfBuckets(5);
- randomKeyGenerator.setNumOfKeys(10);
- randomKeyGenerator.setValidateWrites(true);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.call();
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "2",
+ "--num-of-buckets", "5",
+ "--num-of-keys", "10",
+ "--factor", "THREE",
+ "--type", "RATIS",
+ "--validate-writes"
+ );
+
Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithDatanodeFastRestart.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithDatanodeFastRestart.java
index 957c3d6..2c11804 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithDatanodeFastRestart.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithDatanodeFastRestart.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.ozone.freon;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.container.TestHelper;
@@ -34,6 +32,7 @@
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
+import picocli.CommandLine;
/**
* Tests Freon with Datanode restarts without waiting for pipeline to close.
@@ -111,17 +110,19 @@
startFreon();
}
- private void startFreon() throws Exception {
+ private void startFreon() {
RandomKeyGenerator randomKeyGenerator =
- new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(1);
- randomKeyGenerator.setNumOfBuckets(1);
- randomKeyGenerator.setNumOfKeys(1);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.setKeySize(20971520);
- randomKeyGenerator.setValidateWrites(true);
- randomKeyGenerator.call();
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "1",
+ "--num-of-buckets", "1",
+ "--num-of-keys", "1",
+ "--key-size", "20971520",
+ "--factor", "THREE",
+ "--type", "RATIS",
+ "--validate-writes"
+ );
+
Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithPipelineDestroy.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithPipelineDestroy.java
index 290eb5d..50f4971 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithPipelineDestroy.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreonWithPipelineDestroy.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.ozone.freon;
import org.apache.hadoop.hdds.HddsConfigKeys;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -36,6 +34,7 @@
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
+import picocli.CommandLine;
import java.time.Duration;
import java.util.concurrent.TimeUnit;
@@ -102,17 +101,19 @@
startFreon();
}
- private void startFreon() throws Exception {
+ private void startFreon() {
RandomKeyGenerator randomKeyGenerator =
new RandomKeyGenerator(cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(1);
- randomKeyGenerator.setNumOfBuckets(1);
- randomKeyGenerator.setNumOfKeys(1);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.setKeySize(20971520);
- randomKeyGenerator.setValidateWrites(true);
- randomKeyGenerator.call();
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "1",
+ "--num-of-buckets", "1",
+ "--num-of-keys", "1",
+ "--key-size", "20971520",
+ "--factor", "THREE",
+ "--type", "RATIS",
+ "--validate-writes"
+ );
+
Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java
index cd6af31..132d086 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java
@@ -20,8 +20,6 @@
import java.time.Duration;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
@@ -32,6 +30,7 @@
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
+import picocli.CommandLine;
/**
* Tests Freon, with MiniOzoneCluster.
@@ -77,67 +76,89 @@
}
@Test
- public void defaultTest() throws Exception {
+ void testDefault() {
RandomKeyGenerator randomKeyGenerator =
- new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(2);
- randomKeyGenerator.setNumOfBuckets(5);
- randomKeyGenerator.setNumOfKeys(10);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.call();
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "2",
+ "--num-of-buckets", "5",
+ "--num-of-keys", "10");
+
Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
}
@Test
- public void multiThread() throws Exception {
+ void testECKey() {
RandomKeyGenerator randomKeyGenerator =
- new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(10);
- randomKeyGenerator.setNumOfBuckets(1);
- randomKeyGenerator.setNumOfKeys(10);
- randomKeyGenerator.setNumOfThreads(10);
- randomKeyGenerator.setKeySize(10240);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.call();
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "2",
+ "--num-of-buckets", "5",
+ "--num-of-keys", "10",
+ "--replication", "rs-3-2-1024k",
+ "--type", "EC"
+ );
+
+ Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
+ Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
+ Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
+ }
+
+ @Test
+ void testMultiThread() {
+ RandomKeyGenerator randomKeyGenerator =
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "10",
+ "--num-of-buckets", "1",
+ "--num-of-keys", "10",
+ "--num-of-threads", "10",
+ "--key-size", "10240",
+ "--factor", "THREE",
+ "--type", "RATIS"
+ );
+
Assert.assertEquals(10, randomKeyGenerator.getNumberOfVolumesCreated());
Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
}
@Test
- public void ratisTest3() throws Exception {
+ void testRatisKey() {
RandomKeyGenerator randomKeyGenerator =
- new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(10);
- randomKeyGenerator.setNumOfBuckets(1);
- randomKeyGenerator.setNumOfKeys(10);
- randomKeyGenerator.setNumOfThreads(10);
- randomKeyGenerator.setKeySize(10240);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.call();
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "10",
+ "--num-of-buckets", "1",
+ "--num-of-keys", "10",
+ "--num-of-threads", "10",
+ "--key-size", "10240",
+ "--factor", "THREE",
+ "--type", "RATIS"
+ );
+
Assert.assertEquals(10, randomKeyGenerator.getNumberOfVolumesCreated());
Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
}
@Test
- public void bigFileThan2GB() throws Exception {
+ void testKeyLargerThan2GB() {
RandomKeyGenerator randomKeyGenerator =
- new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(1);
- randomKeyGenerator.setNumOfBuckets(1);
- randomKeyGenerator.setNumOfKeys(1);
- randomKeyGenerator.setNumOfThreads(1);
- randomKeyGenerator.setKeySize(10L + Integer.MAX_VALUE);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.setValidateWrites(true);
- randomKeyGenerator.call();
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "1",
+ "--num-of-buckets", "1",
+ "--num-of-keys", "1",
+ "--num-of-threads", "1",
+ "--key-size", String.valueOf(10L + Integer.MAX_VALUE),
+ "--factor", "THREE",
+ "--type", "RATIS",
+ "--validate-writes"
+ );
+
Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
@@ -145,18 +166,20 @@
}
@Test
- public void fileWithSizeZero() throws Exception {
+ void testZeroSizeKey() {
RandomKeyGenerator randomKeyGenerator =
- new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(1);
- randomKeyGenerator.setNumOfBuckets(1);
- randomKeyGenerator.setNumOfKeys(1);
- randomKeyGenerator.setNumOfThreads(1);
- randomKeyGenerator.setKeySize(0);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.setValidateWrites(true);
- randomKeyGenerator.call();
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "1",
+ "--num-of-buckets", "1",
+ "--num-of-keys", "1",
+ "--num-of-threads", "1",
+ "--key-size", "0",
+ "--factor", "THREE",
+ "--type", "RATIS",
+ "--validate-writes"
+ );
+
Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
@@ -164,33 +187,37 @@
}
@Test
- public void testThreadPoolSize() throws Exception {
+ void testThreadPoolSize() {
RandomKeyGenerator randomKeyGenerator =
- new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(1);
- randomKeyGenerator.setNumOfBuckets(1);
- randomKeyGenerator.setNumOfKeys(1);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.setNumOfThreads(10);
- randomKeyGenerator.call();
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "1",
+ "--num-of-buckets", "1",
+ "--num-of-keys", "1",
+ "--num-of-threads", "10",
+ "--factor", "THREE",
+ "--type", "RATIS"
+ );
+
Assert.assertEquals(10, randomKeyGenerator.getThreadPoolSize());
Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
}
@Test
@Flaky("HDDS-5993")
- public void cleanObjectsTest() throws Exception {
+ void cleanObjectsTest() {
RandomKeyGenerator randomKeyGenerator =
- new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
- randomKeyGenerator.setNumOfVolumes(2);
- randomKeyGenerator.setNumOfBuckets(5);
- randomKeyGenerator.setNumOfKeys(10);
- randomKeyGenerator.setFactor(ReplicationFactor.THREE);
- randomKeyGenerator.setType(ReplicationType.RATIS);
- randomKeyGenerator.setNumOfThreads(10);
- randomKeyGenerator.setCleanObjects(true);
- randomKeyGenerator.call();
+ new RandomKeyGenerator(cluster.getConf());
+ CommandLine cmd = new CommandLine(randomKeyGenerator);
+ cmd.execute("--num-of-volumes", "2",
+ "--num-of-buckets", "5",
+ "--num-of-keys", "10",
+ "--num-of-threads", "10",
+ "--factor", "THREE",
+ "--type", "RATIS",
+ "--clean-objects"
+ );
+
Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java
index 9b9d6d8..c9babb8 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java
@@ -34,6 +34,10 @@
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.client.ContainerBlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.HddsWhiteboxTestUtils;
import org.apache.hadoop.hdds.scm.pipeline.MockPipeline;
import org.apache.hadoop.hdds.utils.db.Table;
@@ -251,7 +255,7 @@
@Test
public void testKeyOps() throws Exception {
// This test needs a cluster with DNs and SCM to wait on safemode
- clusterBuilder.setNumDatanodes(3);
+ clusterBuilder.setNumDatanodes(5);
conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED, true);
startCluster();
String volumeName = UUID.randomUUID().toString();
@@ -260,7 +264,8 @@
.getInternalState(ozoneManager, "keyManager");
KeyManager mockKm = Mockito.spy(keyManager);
TestDataUtil.createVolumeAndBucket(cluster, volumeName, bucketName);
- OmKeyArgs keyArgs = createKeyArgs(volumeName, bucketName);
+ OmKeyArgs keyArgs = createKeyArgs(volumeName, bucketName,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
doKeyOps(keyArgs);
MetricsRecordBuilder omMetrics = getMetrics("OMMetrics");
@@ -273,18 +278,26 @@
assertCounter("NumKeys", 0L, omMetrics);
assertCounter("NumInitiateMultipartUploads", 1L, omMetrics);
- keyArgs = createKeyArgs(volumeName, bucketName);
+ keyArgs = createKeyArgs(volumeName, bucketName,
+ new ECReplicationConfig("rs-3-2-1024K"));
+ doKeyOps(keyArgs);
+ omMetrics = getMetrics("OMMetrics");
+ assertCounter("NumKeyOps", 14L, omMetrics);
+
+ keyArgs = createKeyArgs(volumeName, bucketName,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
OpenKeySession keySession = writeClient.openKey(keyArgs);
writeClient.commitKey(keyArgs, keySession.getId());
- keyArgs = createKeyArgs(volumeName, bucketName);
+ keyArgs = createKeyArgs(volumeName, bucketName,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
keySession = writeClient.openKey(keyArgs);
writeClient.commitKey(keyArgs, keySession.getId());
- keyArgs = createKeyArgs(volumeName, bucketName);
+ keyArgs = createKeyArgs(volumeName, bucketName,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
keySession = writeClient.openKey(keyArgs);
writeClient.commitKey(keyArgs, keySession.getId());
writeClient.deleteKey(keyArgs);
-
omMetrics = getMetrics("OMMetrics");
assertCounter("NumKeys", 2L, omMetrics);
@@ -299,17 +312,18 @@
// inject exception to test for Failure Metrics on the write path
mockWritePathExceptions(OmBucketInfo.class);
- keyArgs = createKeyArgs(volumeName, bucketName);
+ keyArgs = createKeyArgs(volumeName, bucketName,
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE));
doKeyOps(keyArgs);
omMetrics = getMetrics("OMMetrics");
- assertCounter("NumKeyOps", 21L, omMetrics);
- assertCounter("NumKeyAllocate", 5L, omMetrics);
- assertCounter("NumKeyLookup", 2L, omMetrics);
- assertCounter("NumKeyDeletes", 3L, omMetrics);
- assertCounter("NumKeyLists", 2L, omMetrics);
- assertCounter("NumTrashKeyLists", 2L, omMetrics);
- assertCounter("NumInitiateMultipartUploads", 2L, omMetrics);
+ assertCounter("NumKeyOps", 28L, omMetrics);
+ assertCounter("NumKeyAllocate", 6L, omMetrics);
+ assertCounter("NumKeyLookup", 3L, omMetrics);
+ assertCounter("NumKeyDeletes", 4L, omMetrics);
+ assertCounter("NumKeyLists", 3L, omMetrics);
+ assertCounter("NumTrashKeyLists", 3L, omMetrics);
+ assertCounter("NumInitiateMultipartUploads", 3L, omMetrics);
assertCounter("NumKeyAllocateFails", 1L, omMetrics);
assertCounter("NumKeyLookupFails", 1L, omMetrics);
@@ -554,8 +568,8 @@
}
}
- private OmKeyArgs createKeyArgs(String volumeName, String bucketName)
- throws IOException {
+ private OmKeyArgs createKeyArgs(String volumeName, String bucketName,
+ ReplicationConfig repConfig) throws IOException {
OmKeyLocationInfo keyLocationInfo = new OmKeyLocationInfo.Builder()
.setBlockID(new BlockID(new ContainerBlockID(1, 1)))
.setPipeline(MockPipeline.createSingleNodePipeline())
@@ -569,8 +583,10 @@
.setBucketName(bucketName)
.setKeyName(keyName)
.setAcls(Lists.emptyList())
+ .setReplicationConfig(repConfig)
.build();
}
+
private OmVolumeArgs createVolumeArgs() {
String volumeName = UUID.randomUUID().toString();
return new OmVolumeArgs.Builder()
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java
index c0f6342..a09af96 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java
@@ -35,12 +35,19 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdds.cli.GenericCli;
import org.apache.hadoop.hdds.cli.OzoneAdmin;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.OFSPath;
import org.apache.hadoop.fs.ozone.OzoneFsShell;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl;
import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
+import org.apache.hadoop.ozone.client.io.KeyOutputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.ha.ConfUtils;
import org.apache.hadoop.ozone.om.OMConfigKeys;
import org.apache.hadoop.ozone.om.OzoneManager;
@@ -101,8 +108,8 @@
private static String testFilePathString;
private static OzoneConfiguration conf = null;
private static MiniOzoneCluster cluster = null;
- private static OzoneShell ozoneShell = null;
- private static OzoneAdmin ozoneAdminShell = null;
+ private OzoneShell ozoneShell = null;
+ private OzoneAdmin ozoneAdminShell = null;
private final ByteArrayOutputStream out = new ByteArrayOutputStream();
private final ByteArrayOutputStream err = new ByteArrayOutputStream();
@@ -134,19 +141,18 @@
testFile.getParentFile().mkdirs();
testFile.createNewFile();
- ozoneShell = new OzoneShell();
- ozoneAdminShell = new OzoneAdmin();
-
// Init HA cluster
omServiceId = "om-service-test1";
numOfOMs = 3;
clusterId = UUID.randomUUID().toString();
scmId = UUID.randomUUID().toString();
+ final int numDNs = 5;
cluster = MiniOzoneCluster.newOMHABuilder(conf)
.setClusterId(clusterId)
.setScmId(scmId)
.setOMServiceId(omServiceId)
.setNumOfOzoneManagers(numOfOMs)
+ .setNumDatanodes(numDNs)
.build();
cluster.waitForClusterToBeReady();
}
@@ -167,6 +173,8 @@
@Before
public void setup() throws UnsupportedEncodingException {
+ ozoneShell = new OzoneShell();
+ ozoneAdminShell = new OzoneAdmin();
System.setOut(new PrintStream(out, false, DEFAULT_ENCODING));
System.setErr(new PrintStream(err, false, DEFAULT_ENCODING));
}
@@ -499,17 +507,17 @@
execute(ozoneAdminShell, args);
// Test case 3: list THREE replica container
- String factor = "--factor=THREE";
+ String factor = "--replication=THREE";
args = new String[] {"container", "list", "--scm",
"localhost:" + cluster.getStorageContainerManager().getClientRpcPort(),
- factor};
+ factor, "--type=RATIS"};
execute(ozoneAdminShell, args);
// Test case 4: list ONE replica container
- factor = "--factor=ONE";
+ factor = "--replication=ONE";
args = new String[] {"container", "list", "--scm",
"localhost:" + cluster.getStorageContainerManager().getClientRpcPort(),
- factor};
+ factor, "--type=RATIS"};
execute(ozoneAdminShell, args);
}
@@ -893,6 +901,120 @@
}
@Test
+ public void testCreateBucketWithECReplicationConfig() throws Exception {
+ final String volumeName = "volume100";
+ getVolume(volumeName);
+ String[] args =
+ new String[] {"bucket", "create", "/volume100/bucket0", "-t", "EC",
+ "-r", "rs-3-2-1024k"};
+ execute(ozoneShell, args);
+
+ OzoneVolume volume =
+ cluster.getClient().getObjectStore().getVolume(volumeName);
+ OzoneBucket bucket = volume.getBucket("bucket0");
+ try (OzoneOutputStream out = bucket.createKey("myKey", 2000)) {
+ Assert.assertTrue(out.getOutputStream() instanceof ECKeyOutputStream);
+ }
+ }
+
+ @Test
+ public void testPutKeyOnBucketWithECReplicationConfig() throws Exception {
+ final String volumeName = UUID.randomUUID().toString();
+ final String bucketName = UUID.randomUUID().toString();
+ final String keyName = UUID.randomUUID().toString();
+ getVolume(volumeName);
+ String bucketPath =
+ Path.SEPARATOR + volumeName + Path.SEPARATOR + bucketName;
+ String[] args =
+ new String[] {"bucket", "create", bucketPath, "-t", "EC", "-r",
+ "rs-3-2-1024k"};
+ execute(ozoneShell, args);
+
+ args = new String[] {"key", "put", bucketPath + Path.SEPARATOR + keyName,
+ testFilePathString};
+ execute(ozoneShell, args);
+
+ OzoneKeyDetails key =
+ cluster.getClient().getObjectStore().getVolume(volumeName)
+ .getBucket(bucketName).getKey(keyName);
+ assertEquals(HddsProtos.ReplicationType.EC,
+ key.getReplicationConfig().getReplicationType());
+ }
+
+ @Test
+ public void testCreateBucketWithRatisReplicationConfig() throws Exception {
+ final String volumeName = "volume101";
+ getVolume(volumeName);
+ String[] args =
+ new String[] {"bucket", "create", "/volume101/bucket1", "-t", "RATIS",
+ "-r", "3"};
+ execute(ozoneShell, args);
+
+ OzoneVolume volume =
+ cluster.getClient().getObjectStore().getVolume(volumeName);
+ OzoneBucket bucket = volume.getBucket("bucket1");
+ try (OzoneOutputStream out = bucket.createKey("myKey", 2000)) {
+ Assert.assertTrue(out.getOutputStream() instanceof KeyOutputStream);
+ Assert.assertFalse(out.getOutputStream() instanceof ECKeyOutputStream);
+ }
+ }
+
+ @Test
+ public void testSetECReplicationConfigOnBucket() throws Exception {
+ final String volumeName = "volume110";
+ getVolume(volumeName);
+ String bucketPath = "/volume110/bucket0";
+ String[] args = new String[] {"bucket", "create", bucketPath};
+ execute(ozoneShell, args);
+
+ OzoneVolume volume =
+ cluster.getClient().getObjectStore().getVolume(volumeName);
+ OzoneBucket bucket = volume.getBucket("bucket0");
+ try (OzoneOutputStream out = bucket.createKey("myNonECKey", 1024)) {
+ Assert.assertFalse(out.getOutputStream().getClass().getName()
+ .equals(ECKeyOutputStream.class.getName()));
+ }
+
+ args = new String[] {"bucket", "set-replication-config", bucketPath, "-t",
+ "EC", "-r", "rs-3-2-1024k"};
+ execute(ozoneShell, args);
+ bucket = volume.getBucket("bucket0");
+ try (OzoneOutputStream out = bucket.createKey("newECKey", 1024)) {
+ Assert.assertTrue(out.getOutputStream().getClass().getName()
+ .equals(ECKeyOutputStream.class.getName()));
+ }
+
+ args = new String[] {"bucket", "set-replication-config", bucketPath, "-t",
+ "STAND_ALONE", "-r", "ONE"};
+ execute(ozoneShell, args);
+ bucket = volume.getBucket("bucket0");
+ try (OzoneOutputStream out = bucket.createKey("newNonECKey", 1024)) {
+ Assert.assertFalse(out.getOutputStream().getClass().getName()
+ .equals(ECKeyOutputStream.class.getName()));
+ }
+ }
+
+ @Test
+ public void testCreateBucketWithECReplicationConfigWithoutReplicationParam() {
+ getVolume("volume102");
+ String[] args =
+ new String[] {"bucket", "create", "/volume102/bucket2", "-t", "EC"};
+ try {
+ execute(ozoneShell, args);
+ Assert.fail("Must throw Exception when missing replication param");
+ } catch (Exception e) {
+ Assert.assertEquals(
+ "Replication can't be null. Replication type passed was : EC",
+ e.getCause().getMessage());
+ }
+ }
+
+ private void getVolume(String volumeName) {
+ String[] args = new String[] {"volume", "create",
+ "o3://" + omServiceId + "/" + volumeName};
+ execute(ozoneShell, args);
+ }
+
public void testListVolumeBucketKeyShouldPrintValidJsonArray()
throws UnsupportedEncodingException {
diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
index 45a191d..1bad215 100644
--- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
+++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
@@ -356,6 +356,7 @@
LAYOUT_FEATURE_FINALIZATION_FAILED = 72;
PREPARE_FAILED = 73;
NOT_SUPPORTED_OPERATION_WHEN_PREPARED = 74;
+ NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION = 75;
}
/**
@@ -546,6 +547,7 @@
optional uint64 usedNamespace = 17;
optional BucketLayoutProto bucketLayout = 18;
optional string owner = 19;
+ optional hadoop.hdds.DefaultReplicationConfig defaultReplicationConfig = 20;
}
enum StorageTypeProto {
@@ -624,6 +626,7 @@
optional uint64 quotaInBytes = 8;
optional uint64 quotaInNamespace = 9;
optional string ownerName = 10;
+ optional hadoop.hdds.DefaultReplicationConfig defaultReplicationConfig = 11;
}
message PrefixInfo {
@@ -760,8 +763,8 @@
required string bucketName = 2;
required string keyName = 3;
optional uint64 dataSize = 4;
- optional hadoop.hdds.ReplicationType type = 5;
- optional hadoop.hdds.ReplicationFactor factor = 6;
+ optional hadoop.hdds.ReplicationType type = 5 [default = NONE];
+ optional hadoop.hdds.ReplicationFactor factor = 6 [default = ZERO];
repeated KeyLocation keyLocations = 7;
optional bool isMultipartKey = 8;
optional string multipartUploadID = 9;
@@ -783,6 +786,7 @@
// When it is a head operation which is to check whether key exist
optional bool headOp = 18;
+ optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 19;
}
message KeyLocation {
@@ -816,7 +820,7 @@
required string keyName = 3;
required uint64 dataSize = 4;
required hadoop.hdds.ReplicationType type = 5;
- required hadoop.hdds.ReplicationFactor factor = 6;
+ optional hadoop.hdds.ReplicationFactor factor = 6;
repeated KeyLocationList keyLocationList = 7;
required uint64 creationTime = 8;
required uint64 modificationTime = 9;
@@ -827,6 +831,7 @@
optional uint64 objectID = 14;
optional uint64 updateID = 15;
optional uint64 parentID = 16;
+ optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 17;
}
message DirectoryInfo {
@@ -1211,11 +1216,12 @@
required string uploadID = 1;
required uint64 creationTime = 2;
required hadoop.hdds.ReplicationType type = 3;
- required hadoop.hdds.ReplicationFactor factor = 4;
+ optional hadoop.hdds.ReplicationFactor factor = 4;
repeated PartKeyInfo partKeyInfoList = 5;
optional uint64 objectID = 6;
optional uint64 updateID = 7;
optional uint64 parentID = 8;
+ optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 9;
}
message PartKeyInfo {
@@ -1273,7 +1279,7 @@
optional uint32 nextPartNumberMarker = 4;
optional bool isTruncated = 5;
repeated PartInfo partsList = 6;
-
+ optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 7;
}
message ListMultipartUploadsRequest {
@@ -1294,8 +1300,8 @@
required string uploadId = 4;
required uint64 creationTime = 5;
required hadoop.hdds.ReplicationType type = 6;
- required hadoop.hdds.ReplicationFactor factor = 7;
-
+ optional hadoop.hdds.ReplicationFactor factor = 7;
+ optional hadoop.hdds.ECReplicationConfig ecReplicationConfig = 8;
}
message PartInfo {
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 124b272..cd04acb 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
@@ -711,9 +711,10 @@
omPartInfoList.add(omPartInfo);
//if there are parts, use replication type from one of the parts
- replicationConfig = ReplicationConfig.fromProtoTypeAndFactor(
- partKeyInfo.getPartKeyInfo().getType(),
- partKeyInfo.getPartKeyInfo().getFactor());
+ replicationConfig = ReplicationConfig.fromProto(
+ partKeyInfo.getPartKeyInfo().getType(),
+ partKeyInfo.getPartKeyInfo().getFactor(),
+ partKeyInfo.getPartKeyInfo().getEcReplicationConfig());
count++;
}
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneConfigUtil.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneConfigUtil.java
new file mode 100644
index 0000000..ea559f8
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneConfigUtil.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
+/**
+ * Utility class for ozone configurations.
+ */
+public final class OzoneConfigUtil {
+ private OzoneConfigUtil() {
+ }
+
+ public static ReplicationConfig resolveReplicationConfigPreference(
+ HddsProtos.ReplicationType clientType,
+ HddsProtos.ReplicationFactor clientFactor,
+ HddsProtos.ECReplicationConfig clientECReplicationConfig,
+ DefaultReplicationConfig bucketDefaultReplicationConfig,
+ ReplicationConfig omDefaultReplicationConfig) {
+ ReplicationConfig replicationConfig = null;
+ if (clientType != HddsProtos.ReplicationType.NONE) {
+ // Client passed the replication config, so let's use it.
+ replicationConfig = ReplicationConfig
+ .fromProto(clientType, clientFactor, clientECReplicationConfig);
+ } else {
+ // type is NONE, so, let's look for the bucket defaults.
+ if (bucketDefaultReplicationConfig != null) {
+ boolean hasECReplicationConfig = bucketDefaultReplicationConfig
+ .getType() == ReplicationType.EC && bucketDefaultReplicationConfig
+ .getEcReplicationConfig() != null;
+ // Since Bucket defaults are available, let's inherit
+ replicationConfig = ReplicationConfig.fromProto(
+ ReplicationType.toProto(bucketDefaultReplicationConfig.getType()),
+ ReplicationFactor
+ .toProto(bucketDefaultReplicationConfig.getFactor()),
+ hasECReplicationConfig ?
+ bucketDefaultReplicationConfig.getEcReplicationConfig()
+ .toProto() :
+ null);
+ } else {
+ // if bucket defaults also not available, then use server defaults.
+ replicationConfig = omDefaultReplicationConfig;
+ }
+ }
+ return replicationConfig;
+ }
+}
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 1a0e7d4..7731b9e 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
@@ -60,6 +60,8 @@
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.ConfigurationException;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -237,6 +239,10 @@
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_VOLUME_LISTALL_ALLOWED_DEFAULT;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_KEY;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_TYPE_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SERVER_DEFAULT_REPLICATION_TYPE_KEY;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.DETECTED_LOOP_IN_BUCKET_LINKS;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_AUTH_METHOD;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_REQUEST;
@@ -2610,7 +2616,9 @@
auditMap.put(OzoneConsts.BUCKET, bucket);
try {
metrics.incNumBucketInfos();
- return bucketManager.getBucketInfo(volume, bucket);
+ final OmBucketInfo bucketInfo =
+ bucketManager.getBucketInfo(volume, bucket);
+ return bucketInfo;
} catch (Exception ex) {
metrics.incNumBucketInfoFails();
auditSuccess = false;
@@ -3675,6 +3683,17 @@
OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT);
}
+ public ReplicationConfig getDefaultReplicationConfig() {
+ final String replication = configuration.getTrimmed(
+ OZONE_SERVER_DEFAULT_REPLICATION_KEY,
+ OZONE_SERVER_DEFAULT_REPLICATION_DEFAULT);
+ final String type = configuration.getTrimmed(
+ OZONE_SERVER_DEFAULT_REPLICATION_TYPE_KEY,
+ OZONE_SERVER_DEFAULT_REPLICATION_TYPE_DEFAULT);
+ return ReplicationConfig.parse(ReplicationType.valueOf(type),
+ replication, configuration);
+ }
+
public String getOMDefaultBucketLayout() {
return this.defaultBucketLayout;
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
index 7644746..37d3e71 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
@@ -41,8 +41,13 @@
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.om.request.OMClientRequest;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
import org.apache.hadoop.ozone.om.response.OMClientResponse;
import org.apache.hadoop.ozone.om.response.bucket.OMBucketCreateResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketEncryptionInfoProto;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
@@ -51,6 +56,7 @@
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateBucketResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
@@ -365,4 +371,26 @@
}
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = Type.CreateBucket
+ )
+ public static OMRequest disallowCreateBucketWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager()
+ .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ if (req.getCreateBucketRequest()
+ .getBucketInfo().hasDefaultReplicationConfig()
+ && req.getCreateBucketRequest().getBucketInfo()
+ .getDefaultReplicationConfig().hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
index 17c4e39..d237c1c 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
@@ -23,12 +23,19 @@
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.audit.AuditLogger;
import org.apache.hadoop.ozone.audit.OMAction;
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -182,6 +189,13 @@
dbBucketInfo.getQuotaInNamespace());
}
+ DefaultReplicationConfig defaultReplicationConfig =
+ omBucketArgs.getDefaultReplicationConfig();
+ if (defaultReplicationConfig != null) {
+ // Resetting the default replication config.
+ bucketInfoBuilder.setDefaultReplicationConfig(defaultReplicationConfig);
+ }
+
bucketInfoBuilder.setCreationTime(dbBucketInfo.getCreationTime());
bucketInfoBuilder.setModificationTime(
setBucketPropertyRequest.getModificationTime());
@@ -295,4 +309,28 @@
return true;
}
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = Type.SetBucketProperty
+ )
+ public static OMRequest disallowSetBucketPropertyWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager()
+ .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ SetBucketPropertyRequest propReq =
+ req.getSetBucketPropertyRequest();
+ if (propReq.hasBucketArgs()
+ && propReq.getBucketArgs().hasDefaultReplicationConfig()
+ && propReq.getBucketArgs().getDefaultReplicationConfig()
+ .hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
index 223986b..62e8bc2 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
@@ -28,7 +28,9 @@
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
@@ -37,6 +39,12 @@
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.slf4j.Logger;
@@ -196,9 +204,9 @@
long baseObjId = ozoneManager.getObjectIdFromTxId(trxnLogIndex);
List<OzoneAcl> inheritAcls = omPathInfo.getAcls();
- dirKeyInfo = createDirectoryKeyInfoWithACL(keyName,
- keyArgs, baseObjId,
- OzoneAclUtil.fromProtobuf(keyArgs.getAclsList()), trxnLogIndex);
+ dirKeyInfo = createDirectoryKeyInfoWithACL(keyName, keyArgs, baseObjId,
+ OzoneAclUtil.fromProtobuf(keyArgs.getAclsList()), trxnLogIndex,
+ ozoneManager.getDefaultReplicationConfig());
missingParentInfos = getAllParentInfo(ozoneManager, keyArgs,
missingParents, inheritAcls, trxnLogIndex);
@@ -278,8 +286,10 @@
LOG.debug("missing parent {} getting added to KeyTable", missingKey);
// what about keyArgs for parent directories? TODO
- OmKeyInfo parentKeyInfo = createDirectoryKeyInfoWithACL(
- missingKey, keyArgs, nextObjId, inheritAcls, trxnLogIndex);
+ OmKeyInfo parentKeyInfo =
+ createDirectoryKeyInfoWithACL(missingKey, keyArgs, nextObjId,
+ inheritAcls, trxnLogIndex,
+ ozoneManager.getDefaultReplicationConfig());
objectCount++;
missingParentInfos.add(parentKeyInfo);
@@ -336,35 +346,73 @@
* @param keyArgs
* @param objectId
* @param transactionIndex
+ * @param serverDefaultReplConfig
* @return the OmKeyInfo structure
*/
- public static OmKeyInfo createDirectoryKeyInfoWithACL(
- String keyName, KeyArgs keyArgs, long objectId,
- List<OzoneAcl> inheritAcls, long transactionIndex) {
- return dirKeyInfoBuilderNoACL(keyName, keyArgs, objectId)
- .setAcls(inheritAcls).setUpdateID(transactionIndex).build();
+ public static OmKeyInfo createDirectoryKeyInfoWithACL(String keyName,
+ KeyArgs keyArgs, long objectId, List<OzoneAcl> inheritAcls,
+ long transactionIndex, ReplicationConfig serverDefaultReplConfig) {
+ return dirKeyInfoBuilderNoACL(keyName, keyArgs, objectId,
+ serverDefaultReplConfig).setAcls(inheritAcls)
+ .setUpdateID(transactionIndex).build();
}
private static OmKeyInfo.Builder dirKeyInfoBuilderNoACL(String keyName,
- KeyArgs keyArgs, long objectId) {
+ KeyArgs keyArgs, long objectId,
+ ReplicationConfig serverDefaultReplConfig) {
String dirName = OzoneFSUtils.addTrailingSlashIfNeeded(keyName);
- return new OmKeyInfo.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(dirName)
- .setOmKeyLocationInfos(Collections.singletonList(
- new OmKeyLocationInfoGroup(0, new ArrayList<>())))
- .setCreationTime(keyArgs.getModificationTime())
- .setModificationTime(keyArgs.getModificationTime())
- .setDataSize(0)
- .setReplicationConfig(ReplicationConfig
- .fromProtoTypeAndFactor(keyArgs.getType(), keyArgs.getFactor()))
- .setObjectID(objectId)
- .setUpdateID(objectId);
+ OmKeyInfo.Builder keyInfoBuilder =
+ new OmKeyInfo.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(dirName)
+ .setOmKeyLocationInfos(Collections.singletonList(
+ new OmKeyLocationInfoGroup(0, new ArrayList<>())))
+ .setCreationTime(keyArgs.getModificationTime())
+ .setModificationTime(keyArgs.getModificationTime())
+ .setDataSize(0);
+ if (keyArgs.getFactor() != null && keyArgs
+ .getFactor() != HddsProtos.ReplicationFactor.ZERO && keyArgs
+ .getType() != HddsProtos.ReplicationType.EC) {
+ // Factor available and not an EC replication config.
+ keyInfoBuilder.setReplicationConfig(ReplicationConfig
+ .fromProtoTypeAndFactor(keyArgs.getType(), keyArgs.getFactor()));
+ } else if (keyArgs.getType() == HddsProtos.ReplicationType.EC) {
+ // Found EC type
+ keyInfoBuilder.setReplicationConfig(
+ new ECReplicationConfig(keyArgs.getEcReplicationConfig()));
+ } else {
+ // default type
+ keyInfoBuilder.setReplicationConfig(serverDefaultReplConfig);
+ }
+
+ keyInfoBuilder.setObjectID(objectId);
+ return keyInfoBuilder;
}
static long getMaxNumOfRecursiveDirs() {
return MAX_NUM_OF_RECURSIVE_DIRS;
}
+
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = Type.CreateDirectory
+ )
+ public static OMRequest disallowCreateDirectoryWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager().
+ isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ if (req.getCreateDirectoryRequest().getKeyArgs()
+ .hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java
index 0b4f943..2199691 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java
@@ -28,13 +28,20 @@
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
import org.apache.hadoop.ozone.om.response.file.OMFileCreateResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -113,26 +120,23 @@
final long requestedSize = keyArgs.getDataSize() > 0 ?
keyArgs.getDataSize() : scmBlockSize;
- boolean useRatis = ozoneManager.shouldUseRatis();
-
HddsProtos.ReplicationFactor factor = keyArgs.getFactor();
- if (factor == null) {
- factor = useRatis ? HddsProtos.ReplicationFactor.THREE :
- HddsProtos.ReplicationFactor.ONE;
- }
-
HddsProtos.ReplicationType type = keyArgs.getType();
- if (type == null) {
- type = useRatis ? HddsProtos.ReplicationType.RATIS :
- HddsProtos.ReplicationType.STAND_ALONE;
- }
+
+ final OmBucketInfo bucketInfo = ozoneManager
+ .getBucketInfo(keyArgs.getVolumeName(), keyArgs.getBucketName());
+ final ReplicationConfig repConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(type, factor,
+ keyArgs.getEcReplicationConfig(),
+ bucketInfo.getDefaultReplicationConfig(),
+ ozoneManager.getDefaultReplicationConfig());
// TODO: Here we are allocating block with out any check for
// bucket/key/volume or not and also with out any authorization checks.
List< OmKeyLocationInfo > omKeyLocationInfoList =
allocateBlock(ozoneManager.getScmClient(),
- ozoneManager.getBlockTokenSecretManager(), type, factor,
+ ozoneManager.getBlockTokenSecretManager(), repConfig,
new ExcludeList(), requestedSize, scmBlockSize,
ozoneManager.getPreallocateBlocksMax(),
ozoneManager.isGrpcBlockTokenEnabled(),
@@ -239,14 +243,19 @@
}
// do open key
- OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get(
- omMetadataManager.getBucketKey(volumeName, bucketName));
+ omBucketInfo =
+ getBucketInfo(omMetadataManager, volumeName, bucketName);
+ final ReplicationConfig repConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(keyArgs.getType(),
+ keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+ omBucketInfo.getDefaultReplicationConfig(),
+ ozoneManager.getDefaultReplicationConfig());
omKeyInfo = prepareKeyInfo(omMetadataManager, keyArgs, dbKeyInfo,
keyArgs.getDataSize(), locations, getFileEncryptionInfo(keyArgs),
- ozoneManager.getPrefixManager(), bucketInfo, trxnLogIndex,
+ ozoneManager.getPrefixManager(), omBucketInfo, trxnLogIndex,
ozoneManager.getObjectIdFromTxId(trxnLogIndex),
- ozoneManager.isRatisEnabled());
+ ozoneManager.isRatisEnabled(), repConfig);
long openVersion = omKeyInfo.getLatestVersionLocations().getVersion();
long clientID = createFileRequest.getClientID();
@@ -262,8 +271,6 @@
.stream().map(OmKeyLocationInfo::getFromProtobuf)
.collect(Collectors.toList());
omKeyInfo.appendNewBlocks(newLocationList, false);
-
- omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
// check bucket and volume quota
long preAllocatedSpace = newLocationList.size()
* ozoneManager.getScmBlockSize()
@@ -385,4 +392,24 @@
OMException.ResultCodes.DIRECTORY_NOT_FOUND);
}
}
+
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = CreateFile
+ )
+ public static OMRequest disallowCreateFileWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager()
+ .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ if (req.getCreateFileRequest().getKeyArgs().hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestWithFSO.java
index 762163b..9a74dfa 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestWithFSO.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestWithFSO.java
@@ -19,9 +19,11 @@
package org.apache.hadoop.ozone.om.request.file;
import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.ozone.audit.OMAction;
import org.apache.hadoop.ozone.om.OMMetadataManager;
import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
import org.apache.hadoop.ozone.om.OzoneManager;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
@@ -162,13 +164,18 @@
// do open key
OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get(
omMetadataManager.getBucketKey(volumeName, bucketName));
+ final ReplicationConfig repConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(keyArgs.getType(),
+ keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+ bucketInfo.getDefaultReplicationConfig(),
+ ozoneManager.getDefaultReplicationConfig());
OmKeyInfo omFileInfo = prepareFileInfo(omMetadataManager, keyArgs,
dbFileInfo, keyArgs.getDataSize(), locations,
getFileEncryptionInfo(keyArgs), ozoneManager.getPrefixManager(),
bucketInfo, pathInfoFSO, trxnLogIndex,
pathInfoFSO.getLeafNodeObjectId(),
- ozoneManager.isRatisEnabled());
+ ozoneManager.isRatisEnabled(), repConfig);
long openVersion = omFileInfo.getLatestVersionLocations().getVersion();
long clientID = createFileRequest.getClientID();
@@ -184,9 +191,9 @@
omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
// check bucket and volume quota
- long preAllocatedSpace = newLocationList.size()
- * ozoneManager.getScmBlockSize()
- * omFileInfo.getReplicationConfig().getRequiredNodes();
+ long preAllocatedSpace =
+ newLocationList.size() * ozoneManager.getScmBlockSize() * repConfig
+ .getRequiredNodes();
checkBucketQuotaInBytes(omBucketInfo, preAllocatedSpace);
checkBucketQuotaInNamespace(omBucketInfo, 1L);
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java
index 39f090b..c151bcb 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java
@@ -25,10 +25,17 @@
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
@@ -99,14 +106,15 @@
// one uses direct omclient we might be in trouble.
+ ReplicationConfig repConfig = ReplicationConfig.fromProto(keyArgs.getType(),
+ keyArgs.getFactor(), keyArgs.getEcReplicationConfig());
// To allocate atleast one block passing requested size and scmBlockSize
// as same value. When allocating block requested size is same as
// scmBlockSize.
List<OmKeyLocationInfo> omKeyLocationInfoList =
allocateBlock(ozoneManager.getScmClient(),
- ozoneManager.getBlockTokenSecretManager(), keyArgs.getType(),
- keyArgs.getFactor(), excludeList, ozoneManager.getScmBlockSize(),
- ozoneManager.getScmBlockSize(),
+ ozoneManager.getBlockTokenSecretManager(), repConfig, excludeList,
+ ozoneManager.getScmBlockSize(), ozoneManager.getScmBlockSize(),
ozoneManager.getPreallocateBlocksMax(),
ozoneManager.isGrpcBlockTokenEnabled(), ozoneManager.getOMNodeId());
@@ -224,8 +232,9 @@
omBucketInfo.incrUsedBytes(preAllocatedSpace);
omResponse.setAllocateBlockResponse(AllocateBlockResponse.newBuilder()
.setKeyLocation(blockLocation).build());
+ OmBucketInfo shortBucketInfo = omBucketInfo.copyObject();
omClientResponse = new OMAllocateBlockResponse(omResponse.build(),
- openKeyInfo, clientID, omBucketInfo.copyObject(), getBucketLayout());
+ openKeyInfo, clientID, shortBucketInfo, getBucketLayout());
LOG.debug("Allocated block for Volume:{}, Bucket:{}, OpenKey:{}",
volumeName, bucketName, openKeyName);
@@ -250,4 +259,24 @@
return omClientResponse;
}
+
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = Type.AllocateBlock
+ )
+ public static OMRequest disallowAllocateBlockWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager()
+ .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ if (req.getAllocateBlockRequest().getKeyArgs().hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
index 3195e92..047b5f3 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
@@ -38,6 +38,12 @@
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -241,12 +247,11 @@
// AllocatedBlock. The space occupied by the Key shall be based on
// the actual Key size, and the total Block size applied before should
// be subtracted.
- long correctedSpace = omKeyInfo.getDataSize() * factor -
+ long correctedSpace = omKeyInfo.getReplicatedSize() -
allocatedLocationInfoList.size() * scmBlockSize * factor;
// Subtract the size of blocks to be overwritten.
if (keyToDelete != null) {
- correctedSpace -= keyToDelete.getDataSize() *
- keyToDelete.getReplicationConfig().getRequiredNodes();
+ correctedSpace -= keyToDelete.getReplicatedSize();
}
omBucketInfo.incrUsedBytes(correctedSpace);
@@ -322,4 +327,24 @@
commitKeyRequest);
}
}
+
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = Type.CommitKey
+ )
+ public static OMRequest disallowCommitKeyWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager()
+ .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ if (req.getCommitKeyRequest().getKeyArgs().hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java
index 32dbe04..5e2ef2d 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestWithFSO.java
@@ -179,12 +179,11 @@
// AllocatedBlock. The space occupied by the Key shall be based on
// the actual Key size, and the total Block size applied before should
// be subtracted.
- long correctedSpace = omKeyInfo.getDataSize() * factor -
+ long correctedSpace = omKeyInfo.getReplicatedSize() -
allocatedLocationInfoList.size() * scmBlockSize * factor;
// Subtract the size of blocks to be overwritten.
if (keyToDelete != null) {
- correctedSpace -= keyToDelete.getDataSize() *
- keyToDelete.getReplicationConfig().getRequiredNodes();
+ correctedSpace -= keyToDelete.getReplicatedSize();
}
omBucketInfo.incrUsedBytes(correctedSpace);
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java
index 6e41211..d45b2b2 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java
@@ -27,14 +27,21 @@
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.hadoop.ozone.om.request.file.OMDirectoryCreateRequest;
import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -121,19 +128,16 @@
final long requestedSize = keyArgs.getDataSize() > 0 ?
keyArgs.getDataSize() : scmBlockSize;
- boolean useRatis = ozoneManager.shouldUseRatis();
-
HddsProtos.ReplicationFactor factor = keyArgs.getFactor();
- if (factor == null) {
- factor = useRatis ? HddsProtos.ReplicationFactor.THREE :
- HddsProtos.ReplicationFactor.ONE;
- }
-
HddsProtos.ReplicationType type = keyArgs.getType();
- if (type == null) {
- type = useRatis ? HddsProtos.ReplicationType.RATIS :
- HddsProtos.ReplicationType.STAND_ALONE;
- }
+
+ final OmBucketInfo bucketInfo = ozoneManager
+ .getBucketInfo(keyArgs.getVolumeName(), keyArgs.getBucketName());
+ final ReplicationConfig repConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(type, factor,
+ keyArgs.getEcReplicationConfig(),
+ bucketInfo.getDefaultReplicationConfig(),
+ ozoneManager.getDefaultReplicationConfig());
// TODO: Here we are allocating block with out any check for
// bucket/key/volume or not and also with out any authorization checks.
@@ -142,7 +146,7 @@
List<OmKeyLocationInfo> omKeyLocationInfoList =
allocateBlock(ozoneManager.getScmClient(),
- ozoneManager.getBlockTokenSecretManager(), type, factor,
+ ozoneManager.getBlockTokenSecretManager(), repConfig,
new ExcludeList(), requestedSize, scmBlockSize,
ozoneManager.getPreallocateBlocksMax(),
ozoneManager.isGrpcBlockTokenEnabled(),
@@ -264,11 +268,17 @@
numMissingParents = missingParentInfos.size();
}
+ ReplicationConfig replicationConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(keyArgs.getType(),
+ keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+ bucketInfo.getDefaultReplicationConfig(),
+ ozoneManager.getDefaultReplicationConfig());
+
omKeyInfo = prepareKeyInfo(omMetadataManager, keyArgs, dbKeyInfo,
keyArgs.getDataSize(), locations, getFileEncryptionInfo(keyArgs),
ozoneManager.getPrefixManager(), bucketInfo, trxnLogIndex,
ozoneManager.getObjectIdFromTxId(trxnLogIndex),
- ozoneManager.isRatisEnabled());
+ ozoneManager.isRatisEnabled(), replicationConfig);
long openVersion = omKeyInfo.getLatestVersionLocations().getVersion();
long clientID = createKeyRequest.getClientID();
@@ -291,7 +301,7 @@
// commitKey.
long preAllocatedSpace = newLocationList.size()
* ozoneManager.getScmBlockSize()
- * omKeyInfo.getReplicationConfig().getRequiredNodes();
+ * replicationConfig.getRequiredNodes();
// check bucket and volume quota
checkBucketQuotaInBytes(omBucketInfo, preAllocatedSpace);
checkBucketQuotaInNamespace(omBucketInfo, 1L);
@@ -369,4 +379,24 @@
createKeyRequest);
}
}
+
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = Type.CreateKey
+ )
+ public static OMRequest disallowCreateKeyWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager()
+ .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ if (req.getCreateKeyRequest().getKeyArgs().hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequestWithFSO.java
index 7bd9f86..ed6ca16 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequestWithFSO.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequestWithFSO.java
@@ -19,9 +19,11 @@
package org.apache.hadoop.ozone.om.request.key;
import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.ozone.audit.OMAction;
import org.apache.hadoop.ozone.om.OMMetadataManager;
import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
import org.apache.hadoop.ozone.om.OzoneManager;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
@@ -146,13 +148,18 @@
// do open key
OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get(
omMetadataManager.getBucketKey(volumeName, bucketName));
+ final ReplicationConfig repConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(keyArgs.getType(),
+ keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+ bucketInfo.getDefaultReplicationConfig(),
+ ozoneManager.getDefaultReplicationConfig());
OmKeyInfo omFileInfo = prepareFileInfo(omMetadataManager, keyArgs,
dbFileInfo, keyArgs.getDataSize(), locations,
getFileEncryptionInfo(keyArgs), ozoneManager.getPrefixManager(),
bucketInfo, pathInfoFSO, trxnLogIndex,
pathInfoFSO.getLeafNodeObjectId(),
- ozoneManager.isRatisEnabled());
+ ozoneManager.isRatisEnabled(), repConfig);
long openVersion = omFileInfo.getLatestVersionLocations().getVersion();
long clientID = createKeyRequest.getClientID();
@@ -168,9 +175,9 @@
omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
// check bucket and volume quota
- long preAllocatedSpace = newLocationList.size()
- * ozoneManager.getScmBlockSize()
- * omFileInfo.getReplicationConfig().getRequiredNodes();
+ long preAllocatedSpace =
+ newLocationList.size() * ozoneManager.getScmBlockSize() * repConfig
+ .getRequiredNodes();
checkBucketQuotaInBytes(omBucketInfo, preAllocatedSpace);
checkBucketQuotaInNamespace(omBucketInfo, 1L);
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java
index 26a68b8..8a90246 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java
@@ -32,6 +32,7 @@
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
@@ -41,6 +42,7 @@
import org.apache.hadoop.ozone.om.PrefixManager;
import org.apache.hadoop.ozone.om.ResolvedBucket;
import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.hadoop.ozone.om.helpers.KeyValueUtil;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
@@ -49,7 +51,7 @@
import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo;
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
-import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.QuotaUtil;
import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
import org.apache.hadoop.ozone.om.request.OMClientRequestUtils;
import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
@@ -63,7 +65,6 @@
.EncryptedKeyVersion;
import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
@@ -130,23 +131,21 @@
@SuppressWarnings("parameternumber")
protected List< OmKeyLocationInfo > allocateBlock(ScmClient scmClient,
OzoneBlockTokenSecretManager secretManager,
- HddsProtos.ReplicationType replicationType,
- HddsProtos.ReplicationFactor replicationFactor,
- ExcludeList excludeList, long requestedSize, long scmBlockSize,
- int preallocateBlocksMax, boolean grpcBlockTokenEnabled, String omID)
- throws IOException {
-
- int numBlocks = Math.min((int) ((requestedSize - 1) / scmBlockSize + 1),
- preallocateBlocksMax);
+ ReplicationConfig replicationConfig, ExcludeList excludeList,
+ long requestedSize, long scmBlockSize, int preallocateBlocksMax,
+ boolean grpcBlockTokenEnabled, String omID) throws IOException {
+ int dataGroupSize = replicationConfig instanceof ECReplicationConfig
+ ? ((ECReplicationConfig) replicationConfig).getData() : 1;
+ int numBlocks = (int) Math.min(preallocateBlocksMax,
+ (requestedSize - 1) / (scmBlockSize * dataGroupSize) + 1);
List<OmKeyLocationInfo> locationInfos = new ArrayList<>(numBlocks);
String remoteUser = getRemoteUser().getShortUserName();
List<AllocatedBlock> allocatedBlocks;
try {
- allocatedBlocks = scmClient.getBlockClient().allocateBlock(scmBlockSize,
- numBlocks, ReplicationConfig.fromProtoTypeAndFactor(replicationType,
- replicationFactor),
- omID, excludeList);
+ allocatedBlocks = scmClient.getBlockClient()
+ .allocateBlock(scmBlockSize, numBlocks, replicationConfig, omID,
+ excludeList);
} catch (SCMException ex) {
if (ex.getResult()
.equals(SCMException.ResultCodes.SAFE_MODE_EXCEPTION)) {
@@ -579,11 +578,10 @@
*/
protected static long sumBlockLengths(OmKeyInfo omKeyInfo) {
long bytesUsed = 0;
- int keyFactor = omKeyInfo.getReplicationConfig().getRequiredNodes();
-
for (OmKeyLocationInfoGroup group: omKeyInfo.getKeyLocationVersions()) {
for (OmKeyLocationInfo locationInfo : group.getLocationList()) {
- bytesUsed += locationInfo.getLength() * keyFactor;
+ bytesUsed += QuotaUtil.getReplicatedSize(
+ locationInfo.getLength(), omKeyInfo.getReplicationConfig());
}
}
@@ -617,12 +615,13 @@
@Nullable FileEncryptionInfo encInfo,
@Nonnull PrefixManager prefixManager,
@Nullable OmBucketInfo omBucketInfo,
- long transactionLogIndex, long objectID, boolean isRatisEnabled)
+ long transactionLogIndex, long objectID, boolean isRatisEnabled,
+ ReplicationConfig replicationConfig)
throws IOException {
return prepareFileInfo(omMetadataManager, keyArgs, dbKeyInfo, size,
locations, encInfo, prefixManager, omBucketInfo, null,
- transactionLogIndex, objectID, isRatisEnabled);
+ transactionLogIndex, objectID, isRatisEnabled, replicationConfig);
}
/**
@@ -640,7 +639,7 @@
@Nullable OmBucketInfo omBucketInfo,
OMFileRequest.OMPathInfoWithFSO omPathInfo,
long transactionLogIndex, long objectID,
- boolean isRatisEnabled)
+ boolean isRatisEnabled, ReplicationConfig replicationConfig)
throws IOException {
if (keyArgs.getIsMultipartKey()) {
return prepareMultipartFileInfo(omMetadataManager, keyArgs,
@@ -663,14 +662,13 @@
// modification time.
dbKeyInfo.setModificationTime(keyArgs.getModificationTime());
dbKeyInfo.setUpdateID(transactionLogIndex, isRatisEnabled);
+ dbKeyInfo.setReplicationConfig(replicationConfig);
return dbKeyInfo;
}
// the key does not exist, create a new object.
// Blocks will be appended as version 0.
- return createFileInfo(keyArgs, locations,
- ReplicationConfig.fromProtoTypeAndFactor(
- keyArgs.getType(), keyArgs.getFactor()),
+ return createFileInfo(keyArgs, locations, replicationConfig,
keyArgs.getDataSize(), encInfo, prefixManager,
omBucketInfo, omPathInfo, transactionLogIndex, objectID);
}
@@ -689,7 +687,6 @@
@Nullable OmBucketInfo omBucketInfo,
OMFileRequest.OMPathInfoWithFSO omPathInfo,
long transactionLogIndex, long objectID) {
-
OmKeyInfo.Builder builder = new OmKeyInfo.Builder();
builder.setVolumeName(keyArgs.getVolumeName())
.setBucketName(keyArgs.getBucketName())
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequest.java
index c54faf6..4aaec04 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequest.java
@@ -23,7 +23,9 @@
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.ozone.audit.OMAction;
import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
@@ -32,13 +34,19 @@
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.om.request.key.OMKeyRequest;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
import org.apache.hadoop.ozone.om.response.OMClientResponse;
import org.apache.hadoop.ozone.om.response.s3.multipart.S3InitiateMultipartUploadResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartInfoInitiateRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartInfoInitiateResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
@@ -168,10 +176,14 @@
// care of in the final complete multipart upload. AWS S3 behavior is
// also like this, even when key exists in a bucket, user can still
// initiate MPU.
-
- final ReplicationConfig replicationConfig =
- ReplicationConfig.fromProtoTypeAndFactor(
- keyArgs.getType(), keyArgs.getFactor());
+ final OmBucketInfo bucketInfo = omMetadataManager.getBucketTable()
+ .get(omMetadataManager.getBucketKey(volumeName, bucketName));
+ final ReplicationConfig replicationConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(keyArgs.getType(),
+ keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+ bucketInfo != null ?
+ bucketInfo.getDefaultReplicationConfig() :
+ null, ozoneManager.getDefaultReplicationConfig());
multipartKeyInfo = new OmMultipartKeyInfo.Builder()
.setUploadID(keyArgs.getMultipartUploadID())
@@ -182,9 +194,6 @@
.setUpdateID(transactionLogIndex)
.build();
- OmBucketInfo bucketInfo = omMetadataManager.getBucketTable().get(
- omMetadataManager.getBucketKey(volumeName, bucketName));
-
omKeyInfo = new OmKeyInfo.Builder()
.setVolumeName(volumeName)
.setBucketName(bucketName)
@@ -267,4 +276,26 @@
multipartInfoInitiateRequest);
}
}
+
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = Type.InitiateMultiPartUpload
+ )
+ public static OMRequest
+ disallowInitiateMultiPartUploadWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager()
+ .isAllowed(OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ if (req.getInitiateMultiPartUploadRequest().getKeyArgs()
+ .hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequestWithFSO.java
index f1abc21..3f73ffb 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequestWithFSO.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3InitiateMultipartUploadRequestWithFSO.java
@@ -24,8 +24,10 @@
import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneConfigUtil;
import org.apache.hadoop.ozone.om.OzoneManager;
import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
@@ -151,9 +153,14 @@
// care of in the final complete multipart upload. AWS S3 behavior is
// also like this, even when key exists in a bucket, user can still
// initiate MPU.
- final ReplicationConfig replicationConfig =
- ReplicationConfig.fromProtoTypeAndFactor(
- keyArgs.getType(), keyArgs.getFactor());
+ final OmBucketInfo bucketInfo = omMetadataManager.getBucketTable()
+ .get(omMetadataManager.getBucketKey(volumeName, bucketName));
+ final ReplicationConfig replicationConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(keyArgs.getType(),
+ keyArgs.getFactor(), keyArgs.getEcReplicationConfig(),
+ bucketInfo != null ?
+ bucketInfo.getDefaultReplicationConfig() :
+ null, ozoneManager.getDefaultReplicationConfig());
multipartKeyInfo = new OmMultipartKeyInfo.Builder()
.setUploadID(keyArgs.getMultipartUploadID())
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadAbortRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadAbortRequest.java
index f524275..0959c71 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadAbortRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadAbortRequest.java
@@ -25,8 +25,15 @@
import com.google.common.base.Optional;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.QuotaUtil;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.slf4j.Logger;
@@ -163,14 +170,14 @@
// When abort uploaded key, we need to subtract the PartKey length from
// the volume usedBytes.
long quotaReleased = 0;
- int keyFactor = omKeyInfo.getReplicationConfig().getRequiredNodes();
Iterator iter =
multipartKeyInfo.getPartKeyInfoMap().entrySet().iterator();
while (iter.hasNext()) {
Map.Entry entry = (Map.Entry)iter.next();
PartKeyInfo iterPartKeyInfo = (PartKeyInfo)entry.getValue();
- quotaReleased +=
- iterPartKeyInfo.getPartKeyInfo().getDataSize() * keyFactor;
+ quotaReleased += QuotaUtil.getReplicatedSize(
+ iterPartKeyInfo.getPartKeyInfo().getDataSize(),
+ omKeyInfo.getReplicationConfig());
}
omBucketInfo.incrUsedBytes(-quotaReleased);
@@ -254,4 +261,25 @@
volumeName, bucketName, keyName, multipartUploadID);
return multipartKey;
}
+
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = Type.AbortMultiPartUpload
+ )
+ public static OMRequest disallowAbortMultiPartUploadWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager().isAllowed(
+ OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ if (req.getAbortMultiPartUploadRequest().getKeyArgs()
+ .hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequest.java
index 517a46f..901f583 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCommitPartRequest.java
@@ -33,9 +33,14 @@
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.om.request.key.OMKeyRequest;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
import org.apache.hadoop.ozone.om.response.OMClientResponse;
import org.apache.hadoop.ozone.om.response.s3.multipart
.S3MultipartUploadCommitPartResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
.KeyArgs;
@@ -47,6 +52,7 @@
.OMRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.apache.hadoop.util.Time;
@@ -227,7 +233,7 @@
// AllocatedBlock. The space occupied by the Key shall be based on
// the actual Key size, and the total Block size applied before should
// be subtracted.
- long correctedSpace = omKeyInfo.getDataSize() * factor -
+ long correctedSpace = omKeyInfo.getReplicatedSize() -
keyArgs.getKeyLocationsList().size() * scmBlockSize * factor;
omBucketInfo.incrUsedBytes(correctedSpace);
@@ -334,5 +340,25 @@
return omMetadataManager.getMultipartKey(volumeName, bucketName,
keyName, uploadID);
}
-}
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = Type.CommitMultiPartUpload
+ )
+ public static OMRequest disallowCommitMultiPartUploadWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager().isAllowed(
+ OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ if (req.getCommitMultiPartUploadRequest().getKeyArgs()
+ .hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
index 3111a1c..3f205a8 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
@@ -30,7 +30,6 @@
import java.util.TreeMap;
import org.apache.hadoop.hdds.client.ReplicationConfig;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
import org.apache.hadoop.ozone.OzoneConsts;
@@ -49,8 +48,13 @@
import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
import org.apache.hadoop.ozone.om.request.key.OMKeyRequest;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
import org.apache.hadoop.ozone.om.response.OMClientResponse;
import org.apache.hadoop.ozone.om.response.s3.multipart.S3MultipartUploadCompleteResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadCompleteRequest;
@@ -58,6 +62,7 @@
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.apache.hadoop.util.Time;
@@ -352,10 +357,8 @@
String ozoneKey, TreeMap<Integer, PartKeyInfo> partKeyInfoMap,
List<OmKeyLocationInfo> partLocationInfos, long dataSize)
throws IOException {
- HddsProtos.ReplicationType type = partKeyInfoMap.lastEntry().getValue()
- .getPartKeyInfo().getType();
- HddsProtos.ReplicationFactor factor =
- partKeyInfoMap.lastEntry().getValue().getPartKeyInfo().getFactor();
+ OzoneManagerProtocolProtos.KeyInfo partKeyInfo =
+ partKeyInfoMap.lastEntry().getValue().getPartKeyInfo();
OmKeyInfo omKeyInfo = getOmKeyInfoFromKeyTable(ozoneKey, keyName,
omMetadataManager);
@@ -372,8 +375,9 @@
OmKeyInfo.Builder builder =
new OmKeyInfo.Builder().setVolumeName(volumeName)
.setBucketName(bucketName).setKeyName(dbOpenKeyInfo.getKeyName())
- .setReplicationConfig(
- ReplicationConfig.fromProtoTypeAndFactor(type, factor))
+ .setReplicationConfig(ReplicationConfig.fromProto(
+ partKeyInfo.getType(), partKeyInfo.getFactor(),
+ partKeyInfo.getEcReplicationConfig()))
.setCreationTime(keyArgs.getModificationTime())
.setModificationTime(keyArgs.getModificationTime())
.setDataSize(dataSize)
@@ -567,5 +571,26 @@
new CacheValue<>(Optional.of(omBucketInfo), transactionLogIndex));
}
}
-}
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+ processingPhase = RequestProcessingPhase.PRE_PROCESS,
+ requestType = Type.CompleteMultiPartUpload
+ )
+ public static OMRequest
+ disallowCompleteMultiPartUploadWithECReplicationConfig(
+ OMRequest req, ValidationContext ctx) throws OMException {
+ if (!ctx.versionManager().isAllowed(
+ OMLayoutFeature.ERASURE_CODED_STORAGE_SUPPORT)) {
+ if (req.getCompleteMultiPartUploadRequest().getKeyArgs()
+ .hasEcReplicationConfig()) {
+ throw new OMException("Cluster does not have the Erasure Coded"
+ + " Storage support feature finalized yet, but the request contains"
+ + " an Erasure Coded replication type. Rejecting the request,"
+ + " please finalize the cluster upgrade and then try again.",
+ OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+ }
+ }
+ return req;
+ }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/RequestValidations.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/RequestValidations.java
index fe34b74..e2e127a 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/RequestValidations.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/RequestValidations.java
@@ -17,6 +17,7 @@
package org.apache.hadoop.ozone.om.request.validation;
import com.google.protobuf.ServiceException;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
import org.slf4j.Logger;
@@ -59,20 +60,26 @@
return this;
}
- public OMRequest validateRequest(OMRequest request) throws ServiceException {
+ public OMRequest validateRequest(OMRequest request)
+ throws Exception {
List<Method> validations = registry.validationsFor(
conditions(request), request.getCmdType(), PRE_PROCESS);
OMRequest validatedRequest = request.toBuilder().build();
try {
for (Method m : validations) {
- validatedRequest =
- (OMRequest) m.invoke(null, validatedRequest, context);
LOG.debug("Running the {} request pre-process validation from {}.{}",
m.getName(), m.getDeclaringClass().getPackage().getName(),
m.getDeclaringClass().getSimpleName());
+ validatedRequest =
+ (OMRequest) m.invoke(null, validatedRequest, context);
}
- } catch (IllegalAccessException | InvocationTargetException e) {
+ } catch (InvocationTargetException e) {
+ if (e.getCause() instanceof OMException) {
+ throw (OMException) e.getCause();
+ }
+ throw new ServiceException(e);
+ } catch (IllegalAccessException e) {
throw new ServiceException(e);
}
return validatedRequest;
@@ -86,11 +93,11 @@
OMResponse validatedResponse = response.toBuilder().build();
try {
for (Method m : validations) {
- validatedResponse =
- (OMResponse) m.invoke(null, request, response, context);
LOG.debug("Running the {} request post-process validation from {}.{}",
m.getName(), m.getDeclaringClass().getPackage().getName(),
m.getDeclaringClass().getSimpleName());
+ validatedResponse =
+ (OMResponse) m.invoke(null, request, response, context);
}
} catch (InvocationTargetException | IllegalAccessException e) {
throw new ServiceException(e);
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
index 5338c16..aa639b2 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
@@ -29,7 +29,10 @@
*/
public enum OMLayoutFeature implements LayoutFeature {
////////////////////////////// //////////////////////////////
- INITIAL_VERSION(0, "Initial Layout Version");
+ INITIAL_VERSION(0, "Initial Layout Version"),
+
+ ERASURE_CODED_STORAGE_SUPPORT(1, "Ozone version with built in support for"
+ + " Erasure Coded block data storage.");
/////////////////////////////// /////////////////////////////
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
index a6fe773..cb1eadb 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.ozone.om.upgrade;
-import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION;
import java.io.IOException;
import java.lang.reflect.Method;
@@ -83,7 +83,7 @@
layoutFeature.name(),
layoutFeature.layoutVersion(),
lvm.getMetadataLayoutVersion()),
- NOT_SUPPORTED_OPERATION);
+ NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
}
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
index 06aa6c5..a2efa6f 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
@@ -30,6 +30,7 @@
import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException;
import org.apache.hadoop.ozone.om.exceptions.OMNotLeaderException;
import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
@@ -131,7 +132,15 @@
@Override
public OMResponse submitRequest(RpcController controller,
OMRequest request) throws ServiceException {
- OMRequest validatedRequest = requestValidations.validateRequest(request);
+ OMRequest validatedRequest;
+ try {
+ validatedRequest = requestValidations.validateRequest(request);
+ } catch (Exception e) {
+ if (e instanceof OMException) {
+ return createErrorResponse(request, (OMException) e);
+ }
+ throw new ServiceException(e);
+ }
OMResponse response =
dispatcher.processRequest(validatedRequest, this::processRequest,
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
index fd98078..ecec945 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
@@ -22,6 +22,9 @@
import java.util.List;
import java.util.stream.Collectors;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.UpgradeFinalizationStatus;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.utils.db.SequenceNumberNotFoundException;
@@ -45,6 +48,10 @@
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
import org.apache.hadoop.ozone.om.request.OMClientRequest;
import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase;
+import org.apache.hadoop.ozone.om.request.validation.ValidationCondition;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
import org.apache.hadoop.ozone.om.response.OMClientResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CheckVolumeAccessRequest;
@@ -58,6 +65,7 @@
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.InfoVolumeRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.InfoVolumeResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListBucketsRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListBucketsResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListKeysRequest;
@@ -72,7 +80,9 @@
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadListPartsResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneFileStatusProto;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PrepareStatusResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RepeatedKeyInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServiceListRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServiceListResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
@@ -393,10 +403,36 @@
resp.setKeyInfo(keyInfo.getProtobuf(keyArgs.getHeadOp(), clientVersion));
-
return resp.build();
}
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+ processingPhase = RequestProcessingPhase.POST_PROCESS,
+ requestType = Type.LookupKey
+ )
+ public static OMResponse disallowLookupKeyResponseWithECReplicationConfig(
+ OMRequest req, OMResponse resp, ValidationContext ctx)
+ throws ServiceException {
+ if (!resp.hasLookupKeyResponse()) {
+ return resp;
+ }
+ if (resp.getLookupKeyResponse().getKeyInfo().hasEcReplicationConfig()) {
+ resp = resp.toBuilder()
+ .setStatus(Status.NOT_SUPPORTED_OPERATION)
+ .setMessage("Key is a key with Erasure Coded replication, which"
+ + " the client can not understand.\n"
+ + "Please upgrade the client before trying to read the key: "
+ + req.getLookupKeyRequest().getKeyArgs().getVolumeName()
+ + "/" + req.getLookupKeyRequest().getKeyArgs().getBucketName()
+ + "/" + req.getLookupKeyRequest().getKeyArgs().getKeyName()
+ + ".")
+ .clearLookupKeyResponse()
+ .build();
+ }
+ return resp;
+ }
+
private ListBucketsResponse listBuckets(ListBucketsRequest request)
throws IOException {
ListBucketsResponse.Builder resp =
@@ -432,6 +468,33 @@
return resp.build();
}
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+ processingPhase = RequestProcessingPhase.POST_PROCESS,
+ requestType = Type.ListKeys
+ )
+ public static OMResponse disallowListKeysResponseWithECReplicationConfig(
+ OMRequest req, OMResponse resp, ValidationContext ctx)
+ throws ServiceException {
+ if (!resp.hasListKeysResponse()) {
+ return resp;
+ }
+ List<KeyInfo> keys = resp.getListKeysResponse().getKeyInfoList();
+ for (KeyInfo key : keys) {
+ if (key.hasEcReplicationConfig()) {
+ resp = resp.toBuilder()
+ .setStatus(Status.NOT_SUPPORTED_OPERATION)
+ .setMessage("The list of keys contains keys with Erasure Coded"
+ + " replication set, hence the client is not able to"
+ + " represent all the keys returned. Please upgrade the"
+ + " client to get the list of keys.")
+ .clearListKeysResponse()
+ .build();
+ }
+ }
+ return resp;
+ }
+
private ListTrashResponse listTrash(ListTrashRequest request,
int clientVersion) throws IOException {
@@ -452,6 +515,36 @@
return resp.build();
}
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+ processingPhase = RequestProcessingPhase.POST_PROCESS,
+ requestType = Type.ListTrash
+ )
+ public static OMResponse disallowListTrashWithECReplicationConfig(
+ OMRequest req, OMResponse resp, ValidationContext ctx)
+ throws ServiceException {
+ if (!resp.hasListTrashResponse()) {
+ return resp;
+ }
+ List<RepeatedKeyInfo> repeatedKeys =
+ resp.getListTrashResponse().getDeletedKeysList();
+ for (RepeatedKeyInfo repeatedKey : repeatedKeys) {
+ for (KeyInfo key : repeatedKey.getKeyInfoList()) {
+ if (key.hasEcReplicationConfig()) {
+ resp = resp.toBuilder()
+ .setStatus(Status.NOT_SUPPORTED_OPERATION)
+ .setMessage("The list of keys contains keys with Erasure Coded"
+ + " replication set, hence the client is not able to"
+ + " represent all the keys returned. Please upgrade the"
+ + " client to get the list of keys.")
+ .clearListTrashResponse()
+ .build();
+ }
+ }
+ }
+ return resp;
+ }
+
private ServiceListResponse getServiceList(ServiceListRequest request)
throws IOException {
ServiceListResponse.Builder resp = ServiceListResponse.newBuilder();
@@ -500,13 +593,18 @@
omPartInfoList.forEach(partInfo -> partInfoList.add(partInfo.getProto()));
- response.setType(
- omMultipartUploadListParts
- .getReplicationConfig()
- .getReplicationType());
- response.setFactor(
- ReplicationConfig.getLegacyFactor(
- omMultipartUploadListParts.getReplicationConfig()));
+ HddsProtos.ReplicationType repType = omMultipartUploadListParts
+ .getReplicationConfig()
+ .getReplicationType();
+ response.setType(repType);
+ if (repType == HddsProtos.ReplicationType.EC) {
+ response.setEcReplicationConfig(
+ ((ECReplicationConfig)omMultipartUploadListParts
+ .getReplicationConfig()).toProto());
+ } else {
+ response.setFactor(ReplicationConfig.getLegacyFactor(
+ omMultipartUploadListParts.getReplicationConfig()));
+ }
response.setNextPartNumberMarker(
omMultipartUploadListParts.getNextPartNumberMarker());
response.setIsTruncated(omMultipartUploadListParts.isTruncated());
@@ -527,17 +625,27 @@
List<MultipartUploadInfo> info = omMultipartUploadList
.getUploads()
.stream()
- .map(upload -> MultipartUploadInfo.newBuilder()
- .setVolumeName(upload.getVolumeName())
- .setBucketName(upload.getBucketName())
- .setKeyName(upload.getKeyName())
- .setUploadId(upload.getUploadId())
- .setType(upload.getReplicationConfig().getReplicationType())
- .setFactor(
- ReplicationConfig.getLegacyFactor(
- upload.getReplicationConfig()))
- .setCreationTime(upload.getCreationTime().toEpochMilli())
- .build())
+ .map(upload -> {
+ MultipartUploadInfo.Builder bldr = MultipartUploadInfo.newBuilder()
+ .setVolumeName(upload.getVolumeName())
+ .setBucketName(upload.getBucketName())
+ .setKeyName(upload.getKeyName())
+ .setUploadId(upload.getUploadId());
+
+ HddsProtos.ReplicationType repType = upload.getReplicationConfig()
+ .getReplicationType();
+ bldr.setType(repType);
+ if (repType == HddsProtos.ReplicationType.EC) {
+ bldr.setEcReplicationConfig(
+ ((ECReplicationConfig)upload.getReplicationConfig())
+ .toProto());
+ } else {
+ bldr.setFactor(ReplicationConfig.getLegacyFactor(
+ upload.getReplicationConfig()));
+ }
+ bldr.setCreationTime(upload.getCreationTime().toEpochMilli());
+ return bldr.build();
+ })
.collect(Collectors.toList());
ListMultipartUploadsResponse response =
@@ -564,6 +672,35 @@
return rb.build();
}
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+ processingPhase = RequestProcessingPhase.POST_PROCESS,
+ requestType = Type.GetFileStatus
+ )
+ public static OMResponse disallowGetFileStatusWithECReplicationConfig(
+ OMRequest req, OMResponse resp, ValidationContext ctx)
+ throws ServiceException {
+ if (!resp.hasGetFileStatusResponse()) {
+ return resp;
+ }
+ if (resp.getGetFileStatusResponse().getStatus().getKeyInfo()
+ .hasEcReplicationConfig()) {
+ resp = resp.toBuilder()
+ .setStatus(Status.NOT_SUPPORTED_OPERATION)
+ .setMessage("Key is a key with Erasure Coded replication, which"
+ + " the client can not understand."
+ + " Please upgrade the client before trying to read the key info"
+ + " for "
+ + req.getGetFileStatusRequest().getKeyArgs().getVolumeName()
+ + "/" + req.getGetFileStatusRequest().getKeyArgs().getBucketName()
+ + "/" + req.getGetFileStatusRequest().getKeyArgs().getKeyName()
+ + ".")
+ .clearGetFileStatusResponse()
+ .build();
+ }
+ return resp;
+ }
+
private LookupFileResponse lookupFile(LookupFileRequest request,
int clientVersion) throws IOException {
KeyArgs keyArgs = request.getKeyArgs();
@@ -580,6 +717,34 @@
.build();
}
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+ processingPhase = RequestProcessingPhase.POST_PROCESS,
+ requestType = Type.LookupFile
+ )
+ public static OMResponse disallowLookupFileWithECReplicationConfig(
+ OMRequest req, OMResponse resp, ValidationContext ctx)
+ throws ServiceException {
+ if (!resp.hasLookupFileResponse()) {
+ return resp;
+ }
+ if (resp.getLookupFileResponse().getKeyInfo().hasEcReplicationConfig()) {
+ resp = resp.toBuilder()
+ .setStatus(Status.NOT_SUPPORTED_OPERATION)
+ .setMessage("Key is a key with Erasure Coded replication, which the"
+ + " client can not understand."
+ + " Please upgrade the client before trying to read the key info"
+ + " for "
+ + req.getLookupFileRequest().getKeyArgs().getVolumeName()
+ + "/" + req.getLookupFileRequest().getKeyArgs().getBucketName()
+ + "/" + req.getLookupFileRequest().getKeyArgs().getKeyName()
+ + ".")
+ .clearLookupFileResponse()
+ .build();
+ }
+ return resp;
+ }
+
private ListStatusResponse listStatus(
ListStatusRequest request, int clientVersion) throws IOException {
KeyArgs keyArgs = request.getKeyArgs();
@@ -603,6 +768,34 @@
return listStatusResponseBuilder.build();
}
+ @RequestFeatureValidator(
+ conditions = ValidationCondition.OLDER_CLIENT_REQUESTS,
+ processingPhase = RequestProcessingPhase.POST_PROCESS,
+ requestType = Type.ListStatus
+ )
+ public static OMResponse disallowListStatusResponseWithECReplicationConfig(
+ OMRequest req, OMResponse resp, ValidationContext ctx)
+ throws ServiceException {
+ if (!resp.hasListStatusResponse()) {
+ return resp;
+ }
+ List<OzoneFileStatusProto> statuses =
+ resp.getListStatusResponse().getStatusesList();
+ for (OzoneFileStatusProto status : statuses) {
+ if (status.getKeyInfo().hasEcReplicationConfig()) {
+ resp = resp.toBuilder()
+ .setStatus(Status.NOT_SUPPORTED_OPERATION)
+ .setMessage("The list of keys contains keys with Erasure Coded"
+ + " replication set, hence the client is not able to"
+ + " represent all the keys returned."
+ + " Please upgrade the client to get the list of keys.")
+ .clearListStatusResponse()
+ .build();
+ }
+ }
+ return resp;
+ }
+
private FinalizeUpgradeProgressResponse reportUpgradeProgress(
FinalizeUpgradeProgressRequest request) throws IOException {
String upgradeClientId = request.getUpgradeClientId();
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneConfigUtil.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneConfigUtil.java
new file mode 100644
index 0000000..c888ee7
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneConfigUtil.java
@@ -0,0 +1,105 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.apache.hadoop.hdds.client.DefaultReplicationConfig;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests the server side replication config preference logic.
+ */
+public class TestOzoneConfigUtil {
+ private ReplicationConfig ratis3ReplicationConfig =
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE);
+ private HddsProtos.ReplicationType noneType = HddsProtos.ReplicationType.NONE;
+ private HddsProtos.ReplicationFactor zeroFactor =
+ HddsProtos.ReplicationFactor.ZERO;
+ private HddsProtos.ECReplicationConfig clientECReplicationConfig =
+ new ECReplicationConfig("rs-3-2-1024K").toProto();
+ private DefaultReplicationConfig bucketECConfig =
+ new DefaultReplicationConfig(
+ new ECReplicationConfig(clientECReplicationConfig));
+
+ /**
+ * Tests EC bucket defaults.
+ */
+ @Test
+ public void testResolveClientSideRepConfigWhenBucketHasEC() {
+ ReplicationConfig replicationConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(noneType, zeroFactor,
+ clientECReplicationConfig, bucketECConfig, ratis3ReplicationConfig);
+ // Client has no preference, so we should bucket defaults as we passed.
+ Assert.assertEquals(bucketECConfig.getEcReplicationConfig(),
+ replicationConfig);
+ }
+
+ /**
+ * Tests server defaults.
+ */
+ @Test
+ public void testResolveClientSideRepConfigWithNoClientAndBucketDefaults() {
+ ReplicationConfig replicationConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(noneType, zeroFactor,
+ clientECReplicationConfig, null, ratis3ReplicationConfig);
+ // Client has no preference, no bucket defaults, so it should return server
+ // defaults.
+ Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * Tests client preference of EC.
+ */
+ @Test
+ public void testResolveClientSideRepConfigWhenClientPassEC() {
+ ReplicationConfig replicationConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(HddsProtos.ReplicationType.EC,
+ zeroFactor, clientECReplicationConfig, null,
+ ratis3ReplicationConfig);
+ // Client has preference of type EC, no bucket defaults, so it should return
+ // client preference.
+ Assert.assertEquals(new ECReplicationConfig("rs-3-2-1024K"),
+ replicationConfig);
+ }
+
+ /**
+ * Tests bucket ratis defaults.
+ */
+ @Test
+ public void testResolveClientSideRepConfigWhenBucketHasEC3() {
+ DefaultReplicationConfig ratisBucketDefaults =
+ new DefaultReplicationConfig(ReplicationType.RATIS,
+ ReplicationFactor.THREE);
+ ReplicationConfig replicationConfig = OzoneConfigUtil
+ .resolveReplicationConfigPreference(noneType, zeroFactor,
+ clientECReplicationConfig, ratisBucketDefaults,
+ ratis3ReplicationConfig);
+ // Client has no preference of type and bucket has ratis defaults, so it
+ // should return ratis.
+ Assert.assertEquals(ratisBucketDefaults.getType().name(),
+ replicationConfig.getReplicationType().name());
+ Assert.assertEquals(ratisBucketDefaults.getFactor(),
+ ReplicationFactor.valueOf(replicationConfig.getRequiredNodes()));
+ }
+
+}
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java
index cb5bb23..f0af5e7 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java
@@ -24,6 +24,8 @@
import java.util.List;
import java.util.UUID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.om.exceptions.OMException;
@@ -44,6 +46,7 @@
.OMRequest;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS;
import static org.apache.hadoop.ozone.om.request.OMRequestTestUtils.addVolumeAndBucketToDB;
@@ -59,12 +62,33 @@
@Test
public void testPreExecuteWithNormalKey() throws Exception {
- doPreExecute(createKeyRequest(false, 0));
+ ReplicationConfig ratis3Config =
+ ReplicationConfig.fromProtoTypeAndFactor(RATIS, THREE);
+ preExecuteTest(false, 0, ratis3Config);
+ }
+
+ @Test
+ public void testPreExecuteWithECKey() throws Exception {
+ ReplicationConfig ec3Plus2Config = new ECReplicationConfig("rs-3-2-1024k");
+ preExecuteTest(false, 0, ec3Plus2Config);
}
@Test
public void testPreExecuteWithMultipartKey() throws Exception {
- doPreExecute(createKeyRequest(true, 1));
+ ReplicationConfig ratis3Config =
+ ReplicationConfig.fromProtoTypeAndFactor(RATIS, THREE);
+ preExecuteTest(true, 1, ratis3Config);
+ }
+
+ private void preExecuteTest(boolean isMultipartKey, int partNumber,
+ ReplicationConfig repConfig) throws Exception {
+ long scmBlockSize = ozoneManager.getScmBlockSize();
+ for (int i = 0; i <= repConfig.getRequiredNodes(); i++) {
+ doPreExecute(createKeyRequest(isMultipartKey, partNumber,
+ scmBlockSize * i, repConfig));
+ doPreExecute(createKeyRequest(isMultipartKey, partNumber,
+ scmBlockSize * i + 1, repConfig));
+ }
}
@@ -163,10 +187,13 @@
omMetadataManager.getOpenKeyTable(bucketLayout).get(openKey);
Assert.assertNotNull(omKeyInfo);
+ Assert.assertNotNull(omKeyInfo.getLatestVersionLocations());
+ // As our data size is 100, and scmBlockSize is default to 1000, so we
+ // shall have only one block.
List<OmKeyLocationInfo> omKeyLocationInfoList =
omKeyInfo.getLatestVersionLocations().getLocationList();
- Assert.assertTrue(omKeyLocationInfoList.size() == 1);
+ Assert.assertEquals(1, omKeyLocationInfoList.size());
OmKeyLocationInfo omKeyLocationInfo = omKeyLocationInfoList.get(0);
@@ -342,6 +369,12 @@
modifiedOmRequest.getCreateKeyRequest();
KeyArgs keyArgs = createKeyRequest.getKeyArgs();
+ int dataGroupSize = keyArgs.hasEcReplicationConfig() ?
+ keyArgs.getEcReplicationConfig().getData() : 1;
+ long blockSize = ozoneManager.getScmBlockSize();
+ long preAllocatedBlocks = Math.min(ozoneManager.getPreallocateBlocksMax(),
+ (keyArgs.getDataSize() - 1) / (blockSize * dataGroupSize) + 1);
+
// Time should be set
Assert.assertTrue(keyArgs.getModificationTime() > 0);
@@ -354,12 +387,10 @@
if (!originalOMRequest.getCreateKeyRequest().getKeyArgs()
.getIsMultipartKey()) {
- // As our data size is 100, and scmBlockSize is default to 1000, so we
- // shall have only one block.
- List< OzoneManagerProtocolProtos.KeyLocation> keyLocations =
+ List<OzoneManagerProtocolProtos.KeyLocation> keyLocations =
keyArgs.getKeyLocationsList();
// KeyLocation should be set.
- Assert.assertTrue(keyLocations.size() == 1);
+ Assert.assertEquals(preAllocatedBlocks, keyLocations.size());
Assert.assertEquals(CONTAINER_ID,
keyLocations.get(0).getBlockID().getContainerBlockID()
.getContainerID());
@@ -373,7 +404,7 @@
Assert.assertEquals(scmBlockSize, keyLocations.get(0).getLength());
} else {
// We don't create blocks for multipart key in createKey preExecute.
- Assert.assertTrue(keyArgs.getKeyLocationsList().size() == 0);
+ Assert.assertEquals(0, keyArgs.getKeyLocationsList().size());
}
return modifiedOmRequest;
@@ -414,6 +445,36 @@
.setCreateKeyRequest(createKeyRequest).build();
}
+ private OMRequest createKeyRequest(boolean isMultipartKey, int partNumber,
+ long keyLength, ReplicationConfig repConfig) {
+
+ KeyArgs.Builder keyArgs = KeyArgs.newBuilder()
+ .setVolumeName(volumeName).setBucketName(bucketName)
+ .setKeyName(keyName).setIsMultipartKey(isMultipartKey)
+ .setType(repConfig.getReplicationType())
+ .setLatestVersionLocation(true)
+ .setDataSize(keyLength);
+
+ if (repConfig.getReplicationType() == EC) {
+ keyArgs.setEcReplicationConfig(
+ ((ECReplicationConfig) repConfig).toProto());
+ } else {
+ keyArgs.setFactor(ReplicationConfig.getLegacyFactor(repConfig));
+ }
+
+ if (isMultipartKey) {
+ keyArgs.setMultipartNumber(partNumber);
+ }
+
+ OzoneManagerProtocolProtos.CreateKeyRequest createKeyRequest =
+ CreateKeyRequest.newBuilder().setKeyArgs(keyArgs).build();
+
+ return OMRequest.newBuilder()
+ .setCmdType(OzoneManagerProtocolProtos.Type.CreateKey)
+ .setClientId(UUID.randomUUID().toString())
+ .setCreateKeyRequest(createKeyRequest).build();
+ }
+
@Test
public void testKeyCreateWithFileSystemPathsEnabled() throws Exception {
@@ -426,11 +487,11 @@
addVolumeAndBucketToDB(volumeName, bucketName,
omMetadataManager, getBucketLayout());
- keyName = "dir1/dir2/dir3/file1";
+ String keyName = "dir1/dir2/dir3/file1";
createAndCheck(keyName);
// Key with leading '/'.
- String keyName = "/a/b/c/file1";
+ keyName = "/a/b/c/file1";
createAndCheck(keyName);
// Commit openKey entry.
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java
index 33e09ab..d538f20 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java
@@ -24,11 +24,14 @@
import java.util.UUID;
import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.om.OzoneManagerPrepareState;
import org.apache.hadoop.ozone.om.ResolvedBucket;
import org.apache.hadoop.ozone.om.KeyManager;
import org.apache.hadoop.ozone.om.KeyManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
@@ -134,6 +137,8 @@
when(ozoneManager.isAdmin(any(String.class))).thenReturn(true);
when(ozoneManager.isAdmin(any(UserGroupInformation.class)))
.thenReturn(true);
+ when(ozoneManager.getBucketInfo(anyString(), anyString())).thenReturn(
+ new OmBucketInfo.Builder().setVolumeName("").setBucketName("").build());
Mockito.doNothing().when(auditLogger).logWrite(any(AuditMessage.class));
scmClient = Mockito.mock(ScmClient.class);
@@ -163,17 +168,21 @@
.setNodes(new ArrayList<>())
.build();
- AllocatedBlock allocatedBlock =
- new AllocatedBlock.Builder()
- .setContainerBlockID(new ContainerBlockID(CONTAINER_ID, LOCAL_ID))
- .setPipeline(pipeline).build();
+ AllocatedBlock.Builder blockBuilder = new AllocatedBlock.Builder()
+ .setPipeline(pipeline);
- List<AllocatedBlock> allocatedBlocks = new ArrayList<>();
-
- allocatedBlocks.add(allocatedBlock);
-
- when(scmBlockLocationProtocol.allocateBlock(anyLong(), anyInt(), any(),
- anyString(), any())).thenReturn(allocatedBlocks);
+ when(scmBlockLocationProtocol.allocateBlock(anyLong(), anyInt(),
+ any(ReplicationConfig.class),
+ anyString(), any(ExcludeList.class))).thenAnswer(invocation -> {
+ int num = invocation.getArgument(1);
+ List<AllocatedBlock> allocatedBlocks = new ArrayList<>(num);
+ for (int i = 0; i < num; i++) {
+ blockBuilder.setContainerBlockID(
+ new ContainerBlockID(CONTAINER_ID + i, LOCAL_ID + i));
+ allocatedBlocks.add(blockBuilder.build());
+ }
+ return allocatedBlocks;
+ });
volumeName = UUID.randomUUID().toString();
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java
index 581bd2a..591c3bf 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java
@@ -22,6 +22,7 @@
import java.io.IOException;
import java.util.List;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.ozone.om.helpers.BucketLayout;
import org.apache.hadoop.ozone.om.request.OMClientRequest;
import org.junit.After;
@@ -83,6 +84,8 @@
when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
auditLogger = Mockito.mock(AuditLogger.class);
when(ozoneManager.getAuditLogger()).thenReturn(auditLogger);
+ when(ozoneManager.getDefaultReplicationConfig()).thenReturn(
+ ReplicationConfig.getDefault(ozoneConfiguration));
Mockito.doNothing().when(auditLogger).logWrite(any(AuditMessage.class));
when(ozoneManager.resolveBucketLink(any(KeyArgs.class),
any(OMClientRequest.class)))
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
index 4508eaf..8b97bba 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
@@ -16,7 +16,6 @@
*/
package org.apache.hadoop.ozone.om.request.validation;
-import com.google.protobuf.ServiceException;
import org.apache.hadoop.ozone.ClientVersion;
import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting;
import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.ValidationListener;
@@ -69,7 +68,7 @@
}
@Test(expected = NullPointerException.class)
- public void testUsingRegistryWithoutLoading() throws ServiceException {
+ public void testUsingRegistryWithoutLoading() throws Exception {
new RequestValidations()
.fromPackage(PACKAGE)
.withinContext(of(aFinalizedVersionManager()))
@@ -77,7 +76,7 @@
}
@Test(expected = NullPointerException.class)
- public void testUsingRegistryWithoutContext() throws ServiceException {
+ public void testUsingRegistryWithoutContext() throws Exception {
new RequestValidations()
.fromPackage(PACKAGE)
.load()
@@ -85,7 +84,7 @@
}
@Test
- public void testUsingRegistryWithoutPackage() throws ServiceException {
+ public void testUsingRegistryWithoutPackage() throws Exception {
new RequestValidations()
.withinContext(of(aFinalizedVersionManager()))
.load()
@@ -96,7 +95,7 @@
@Test
public void testNoPreValidationsWithoutValidationMethods()
- throws ServiceException {
+ throws Exception {
int omVersion = 0;
ValidationContext ctx = of(aFinalizedVersionManager());
RequestValidations validations = loadEmptyValidations(ctx);
@@ -108,7 +107,7 @@
@Test
public void testNoPostValidationsWithoutValidationMethods()
- throws ServiceException {
+ throws Exception {
ValidationContext ctx = of(aFinalizedVersionManager());
RequestValidations validations = loadEmptyValidations(ctx);
@@ -120,7 +119,7 @@
@Test
public void testNoPreValidationsRunningForRequestTypeWithoutValidators()
- throws ServiceException {
+ throws Exception {
ValidationContext ctx = of(aFinalizedVersionManager());
RequestValidations validations = loadValidations(ctx);
@@ -131,7 +130,7 @@
@Test
public void testNoPostValidationsAreRunningForRequestTypeWithoutValidators()
- throws ServiceException {
+ throws Exception {
ValidationContext ctx = of(aFinalizedVersionManager());
RequestValidations validations = loadValidations(ctx);
@@ -142,14 +141,14 @@
}
@Test
- public void testPreProcessorExceptionHandling() {
+ public void testPreProcessorExceptionHandling() throws Exception {
ValidationContext ctx = of(aFinalizedVersionManager());
RequestValidations validations = loadValidations(ctx);
try {
validations.validateRequest(aDeleteKeysRequest(olderClientVersion()));
fail("ServiceException was expected but was not thrown.");
- } catch (ServiceException ignored) { }
+ } catch (Exception ignored) { }
validationListener.assertNumOfEvents(1);
validationListener.assertExactListOfValidatorsCalled(
@@ -165,7 +164,7 @@
validations.validateResponse(
aDeleteKeysRequest(olderClientVersion()), aDeleteKeysResponse());
fail("ServiceException was expected but was not thrown.");
- } catch (ServiceException ignored) { }
+ } catch (Exception ignored) { }
validationListener.assertNumOfEvents(1);
validationListener.assertExactListOfValidatorsCalled(
@@ -174,7 +173,7 @@
@Test
public void testOldClientConditionIsRecognizedAndPreValidatorsApplied()
- throws ServiceException {
+ throws Exception {
ValidationContext ctx = of(aFinalizedVersionManager());
RequestValidations validations = loadValidations(ctx);
@@ -187,7 +186,7 @@
@Test
public void testOldClientConditionIsRecognizedAndPostValidatorsApplied()
- throws ServiceException {
+ throws Exception {
ValidationContext ctx = of(aFinalizedVersionManager());
RequestValidations validations = loadValidations(ctx);
@@ -202,7 +201,7 @@
@Test
public void testPreFinalizedWithOldClientConditionPreProcValidatorsApplied()
- throws ServiceException {
+ throws Exception {
ValidationContext ctx = of(anUnfinalizedVersionManager());
RequestValidations validations = loadValidations(ctx);
@@ -216,7 +215,7 @@
@Test
public void testPreFinalizedWithOldClientConditionPostProcValidatorsApplied()
- throws ServiceException {
+ throws Exception {
ValidationContext ctx = of(anUnfinalizedVersionManager());
RequestValidations validations = loadValidations(ctx);
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java
index f3b470c..f9d72b6 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java
@@ -21,6 +21,7 @@
import com.google.common.collect.Iterators;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -135,6 +136,8 @@
);
when(om.getAclsEnabled()).thenReturn(false);
when(om.getAuditLogger()).thenReturn(mock(AuditLogger.class));
+ when(om.getDefaultReplicationConfig()).thenReturn(ReplicationConfig
+ .getDefault(new OzoneConfiguration()));
addVolumeToMetaTable(aVolumeArgs());
addBucketToMetaTable(aBucketInfo());
}
diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java
index 55bce59..bec00e9 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java
@@ -21,11 +21,14 @@
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
+import java.time.Clock;
+import java.time.ZoneOffset;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.HashSet;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.crypto.key.KeyProvider;
import org.apache.hadoop.fs.BlockLocation;
@@ -51,6 +54,7 @@
import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.common.MonotonicClock;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
@@ -65,6 +69,7 @@
import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -84,10 +89,15 @@
private ObjectStore objectStore;
private OzoneVolume volume;
private OzoneBucket bucket;
- private ReplicationConfig replicationConfig;
+ private ReplicationConfig bucketReplicationConfig;
+ // Client side configured replication config.
+ private ReplicationConfig clientConfiguredReplicationConfig;
private boolean securityEnabled;
private int configuredDnPort;
private OzoneConfiguration config;
+ private long nextReplicationConfigRefreshTime;
+ private long bucketRepConfigRefreshPeriodMS;
+ private java.time.Clock clock = new MonotonicClock(ZoneOffset.UTC);
/**
* Create new OzoneClientAdapter implementation.
@@ -112,6 +122,11 @@
throws IOException {
OzoneConfiguration conf = OzoneConfiguration.of(hadoopConf);
+ bucketRepConfigRefreshPeriodMS = conf.getLong(
+ OzoneConfigKeys
+ .OZONE_CLIENT_BUCKET_REPLICATION_CONFIG_REFRESH_PERIOD_MS,
+ OzoneConfigKeys
+ .OZONE_CLIENT_BUCKET_REPLICATION_CONFIG_REFRESH_PERIOD_DEFAULT_MS);
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.
@@ -141,7 +156,8 @@
this.securityEnabled = true;
}
- replicationConfig = ReplicationConfig.getDefault(conf);
+ clientConfiguredReplicationConfig =
+ OzoneClientUtils.getClientConfiguredReplicationConfig(conf);
if (OmUtils.isOmHAServiceId(conf, omHost)) {
// omHost is listed as one of the service ids in the config,
@@ -158,6 +174,9 @@
objectStore = ozoneClient.getObjectStore();
this.volume = objectStore.getVolume(volumeStr);
this.bucket = volume.getBucket(bucketStr);
+ bucketReplicationConfig = this.bucket.getReplicationConfig();
+ nextReplicationConfigRefreshTime =
+ clock.millis() + bucketRepConfigRefreshPeriodMS;
// resolve the bucket layout in case of Link Bucket
BucketLayout resolvedBucketLayout =
@@ -172,9 +191,20 @@
this.config = conf;
}
+ /**
+ * This API returns the value what is configured at client side only. It could
+ * differ from the server side default values. If no replication config
+ * configured at client, it will return 3.
+ */
@Override
public short getDefaultReplication() {
- return (short) replicationConfig.getRequiredNodes();
+ if (clientConfiguredReplicationConfig == null) {
+ // to provide backward compatibility, we are just retuning 3;
+ // However we need to handle with the correct behavior.
+ // TODO: Please see HDDS-5646
+ return (short) ReplicationFactor.THREE.getValue();
+ }
+ return (short) clientConfiguredReplicationConfig.getRequiredNodes();
}
@Override
@@ -207,21 +237,11 @@
boolean overWrite, boolean recursive) throws IOException {
incrementCounter(Statistic.OBJECTS_CREATED, 1);
try {
- OzoneOutputStream ozoneOutputStream = null;
- if (replication == ReplicationFactor.ONE.getValue()
- || replication == ReplicationFactor.THREE.getValue()) {
-
- ReplicationConfig customReplicationConfig =
- ReplicationConfig.adjustReplication(
- replicationConfig, replication, config
- );
- ozoneOutputStream =
- bucket.createFile(key, 0, customReplicationConfig, overWrite,
- recursive);
- } else {
- ozoneOutputStream =
- bucket.createFile(key, 0, replicationConfig, overWrite, recursive);
- }
+ OzoneOutputStream ozoneOutputStream = bucket.createFile(key, 0,
+ OzoneClientUtils.resolveClientSideReplicationConfig(replication,
+ this.clientConfiguredReplicationConfig,
+ getReplicationConfigWithRefreshCheck(), config), overWrite,
+ recursive);
return new OzoneFSOutputStream(ozoneOutputStream.getOutputStream());
} catch (OMException ex) {
if (ex.getResult() == OMException.ResultCodes.FILE_ALREADY_EXISTS
@@ -234,6 +254,17 @@
}
}
+ private ReplicationConfig getReplicationConfigWithRefreshCheck()
+ throws IOException {
+ if (clock.millis() > nextReplicationConfigRefreshTime) {
+ this.bucketReplicationConfig =
+ volume.getBucket(bucket.getName()).getReplicationConfig();
+ nextReplicationConfigRefreshTime =
+ clock.millis() + bucketRepConfigRefreshPeriodMS;
+ }
+ return this.bucketReplicationConfig;
+ }
+
@Override
public void renameKey(String key, String newKeyName) throws IOException {
incrementCounter(Statistic.OBJECTS_RENAMED, 1);
@@ -399,6 +430,11 @@
return objectStore.getCanonicalServiceName();
}
+ @VisibleForTesting
+ void setClock(Clock monotonicClock) {
+ this.clock = monotonicClock;
+ }
+
/**
* Ozone Delegation Token Renewer.
*/
diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java
index 35a027d..f880c5e 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java
@@ -101,7 +101,7 @@
private OzoneClient ozoneClient;
private ObjectStore objectStore;
private ClientProtocol proxy;
- private ReplicationConfig replicationConfig;
+ private ReplicationConfig clientConfiguredReplicationConfig;
private boolean securityEnabled;
private int configuredDnPort;
private BucketLayout defaultOFSBucketLayout;
@@ -171,7 +171,8 @@
this.securityEnabled = true;
}
- replicationConfig = ReplicationConfig.getDefault(conf);
+ clientConfiguredReplicationConfig =
+ OzoneClientUtils.getClientConfiguredReplicationConfig(conf);
if (OmUtils.isOmHAServiceId(conf, omHost)) {
// omHost is listed as one of the service ids in the config,
@@ -288,9 +289,20 @@
return bucket;
}
+ /**
+ * This API returns the value what is configured at client side only. It could
+ * differ from the server side default values. If no replication config
+ * configured at client, it will return 3.
+ */
@Override
public short getDefaultReplication() {
- return (short) replicationConfig.getRequiredNodes();
+ if (clientConfiguredReplicationConfig == null) {
+ // to provide backward compatibility, we are just retuning 3;
+ // However we need to handle with the correct behavior.
+ // TODO: Please see HDDS-5646
+ return (short) ReplicationFactor.THREE.getValue();
+ }
+ return (short) clientConfiguredReplicationConfig.getRequiredNodes();
}
@Override
@@ -333,18 +345,10 @@
try {
// Hadoop CopyCommands class always sets recursive to true
OzoneBucket bucket = getBucket(ofsPath, recursive);
- OzoneOutputStream ozoneOutputStream = null;
- if (replication == ReplicationFactor.ONE.getValue()
- || replication == ReplicationFactor.THREE.getValue()) {
-
- ozoneOutputStream = bucket.createFile(key, 0,
- ReplicationConfig.adjustReplication(
- replicationConfig, replication, config),
- overWrite, recursive);
- } else {
- ozoneOutputStream =
- bucket.createFile(key, 0, replicationConfig, overWrite, recursive);
- }
+ OzoneOutputStream ozoneOutputStream = bucket.createFile(key, 0,
+ OzoneClientUtils.resolveClientSideReplicationConfig(replication,
+ this.clientConfiguredReplicationConfig,
+ bucket.getReplicationConfig(), config), overWrite, recursive);
return new OzoneFSOutputStream(ozoneOutputStream.getOutputStream());
} catch (OMException ex) {
if (ex.getResult() == OMException.ResultCodes.FILE_ALREADY_EXISTS
diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java
index c440a6b..14021da 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java
@@ -18,6 +18,11 @@
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.ozone.client.checksum.BaseFileChecksumHelper;
@@ -32,6 +37,9 @@
import java.io.IOException;
import java.util.Set;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.DETECTED_LOOP_IN_BUCKET_LINKS;
/**
@@ -72,6 +80,122 @@
return bucket.getBucketLayout();
}
+ /**
+ * This API used to resolve the client side configuration preference for file
+ * system layer implementations.
+ *
+ * @param replication - replication value passed from FS API.
+ * @param clientConfiguredReplConfig - Client side configured replication
+ * config.
+ * @param bucketReplConfig - server side bucket default replication
+ * config.
+ * @param config - Ozone configuration object.
+ * @return client resolved replication config.
+ */
+ public static ReplicationConfig resolveClientSideReplicationConfig(
+ short replication, ReplicationConfig clientConfiguredReplConfig,
+ ReplicationConfig bucketReplConfig, ConfigurationSource config) {
+ ReplicationConfig clientDeterminedReplConfig = null;
+
+ boolean isECBucket = bucketReplConfig != null && bucketReplConfig
+ .getReplicationType() == HddsProtos.ReplicationType.EC;
+
+ // if bucket replication config configured with EC, we will give high
+ // preference to server side bucket defaults.
+ // Why we give high prefernce to EC is, there is no way for file system
+ // interfaces to pass EC replication. So, if one configures EC at bucket,
+ // we consider EC to take preference. in short, keys created from file
+ // system under EC bucket will always be EC'd.
+ if (isECBucket) {
+ // if bucket is EC, don't bother client provided configs, let's pass
+ // bucket config.
+ clientDeterminedReplConfig = bucketReplConfig;
+ } else {
+ // Let's validate the client side available replication configs.
+ boolean isReplicationInSupportedList =
+ (replication == ReplicationFactor.ONE
+ .getValue() || replication == ReplicationFactor.THREE.getValue());
+ if (isReplicationInSupportedList) {
+ if (clientConfiguredReplConfig != null) {
+ // Uses the replication(short value) passed from file system API and
+ // construct replication config object.
+ // In case if client explicitely configured EC in configurations, we
+ // always take EC as priority as EC replication can't be expressed in
+ // filesystem API.
+ clientDeterminedReplConfig = ReplicationConfig
+ .adjustReplication(clientConfiguredReplConfig, replication,
+ config);
+ } else {
+ // In file system layers, replication parameter always passed.
+ // so, to respect the API provided replication value, we take RATIS as
+ // default type.
+ clientDeterminedReplConfig = ReplicationConfig
+ .parse(ReplicationType.RATIS, Short.toString(replication),
+ config);
+ }
+ } else {
+ // API passed replication number is not in supported replication list.
+ // So, let's use whatever available in client side configured.
+ // By default it will be null, so server will use server defaults.
+ clientDeterminedReplConfig = clientConfiguredReplConfig;
+ }
+ }
+ return clientDeterminedReplConfig;
+ }
+
+ public static ReplicationConfig getClientConfiguredReplicationConfig(
+ ConfigurationSource config) {
+ String replication = config.get(OZONE_REPLICATION);
+ if (replication == null) {
+ return null;
+ }
+ return ReplicationConfig.parse(ReplicationType.valueOf(
+ config.get(OZONE_REPLICATION_TYPE, OZONE_REPLICATION_TYPE_DEFAULT)),
+ replication, config);
+ }
+
+ /**
+ * Gets the client side replication config by checking user passed values vs
+ * client configured values.
+ * @param userPassedType - User provided replication type.
+ * @param userPassedReplication - User provided replication.
+ * @param clientSideConfig - Client side configuration.
+ * @return ReplicationConfig.
+ */
+ public static ReplicationConfig validateAndGetClientReplicationConfig(
+ ReplicationType userPassedType, String userPassedReplication,
+ ConfigurationSource clientSideConfig) {
+ // Priority 1: User passed replication config values.
+ // Priority 2: Client side configured replication config values.
+ /* if above two are not available, we should just return null and clients
+ can pass null replication config to server. Now server will take the
+ decision of finding the replication config( either from bucket defaults
+ or server defaults). */
+ ReplicationType clientReplicationType = userPassedType;
+ String clientReplication = userPassedReplication;
+ String clientConfiguredDefaultType =
+ clientSideConfig.get(OZONE_REPLICATION_TYPE);
+ if (userPassedType == null && clientConfiguredDefaultType != null) {
+ clientReplicationType =
+ ReplicationType.valueOf(clientConfiguredDefaultType);
+ }
+
+ String clientConfiguredDefaultReplication =
+ clientSideConfig.get(OZONE_REPLICATION);
+ if (userPassedReplication == null
+ && clientConfiguredDefaultReplication != null) {
+ clientReplication = clientConfiguredDefaultReplication;
+ }
+
+ // if clientReplicationType or clientReplication is null, then we just pass
+ // replication config as null, so that server will take decision.
+ if (clientReplicationType == null || clientReplication == null) {
+ return null;
+ }
+ return ReplicationConfig
+ .parse(clientReplicationType, clientReplication, clientSideConfig);
+ }
+
public static FileChecksum getFileChecksumWithCombineMode(OzoneVolume volume,
OzoneBucket bucket, String keyName, long length,
OzoneClientConfig.ChecksumCombineMode combineMode,
diff --git a/hadoop-ozone/ozonefs-common/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneClientUtils.java b/hadoop-ozone/ozonefs-common/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneClientUtils.java
index a9d4f98..561cc34 100644
--- a/hadoop-ozone/ozonefs-common/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneClientUtils.java
+++ b/hadoop-ozone/ozonefs-common/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneClientUtils.java
@@ -17,10 +17,18 @@
package org.apache.hadoop.fs.ozone;
import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
@@ -29,9 +37,16 @@
import static org.mockito.Mockito.mock;
/**
- * Unit tests for OzoneClientUtils.
+ * Tests the behavior of OzoneClientUtils APIs.
*/
public class TestOzoneClientUtils {
+ private ReplicationConfig ecReplicationConfig =
+ new ECReplicationConfig("rs-3-2-1024K");
+ private ReplicationConfig ratis3ReplicationConfig =
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE);
+ private ReplicationConfig ratis1ReplicationConfig =
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE);
+
@Test(expected = IllegalArgumentException.class)
public void testNegativeLength() throws IOException {
OzoneVolume volume = mock(OzoneVolume.class);
@@ -56,4 +71,191 @@
assertNull(checksum);
}
-}
\ No newline at end of file
+
+ @Test
+ public void testResolveClientSideRepConfigWhenBucketHasEC() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .resolveClientSideReplicationConfig(
+ (short) 3, null,
+ ecReplicationConfig, new OzoneConfiguration());
+ // Bucket default is EC.
+ Assert.assertEquals(ecReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * When bucket replication is null and it should respect fs passed value.
+ */
+ @Test
+ public void testResolveClientSideRepConfigWhenBucketHasNull() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .resolveClientSideReplicationConfig(
+ (short) 3, null, null,
+ new OzoneConfiguration());
+ // Passed replication is 3 - Ozone mapped replication is ratis THREE
+ Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * When bucket replication is null and it should return null if fs passed
+ * value is invalid.
+ */
+ @Test
+ public void testResolveClientSideRepConfigWhenFSPassedReplicationIsInvalid() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .resolveClientSideReplicationConfig(
+ (short) -1, null, null,
+ new OzoneConfiguration());
+ // client configured value also null.
+ // This API caller should leave the decision to server.
+ Assert.assertNull(replicationConfig);
+ }
+
+ /**
+ * When bucket default is non-EC and client side values are not valid, we
+ * would just return null, so servers can make decision in this case.
+ */
+ @Test
+ public void testResolveRepConfWhenFSPassedIsInvalidButBucketDefaultNonEC() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .resolveClientSideReplicationConfig(
+ (short) -1, null, ratis3ReplicationConfig,
+ new OzoneConfiguration());
+ // Configured client config also null.
+ Assert.assertNull(replicationConfig);
+ }
+
+ /**
+ * When bucket default is non-EC and client side value is valid, we
+ * would should return client side valid value.
+ */
+ @Test
+ public void testResolveRepConfWhenFSPassedIsValidButBucketDefaultNonEC() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .resolveClientSideReplicationConfig(
+ (short) 1, null, ratis3ReplicationConfig,
+ new OzoneConfiguration());
+ // Passed value is replication one - Ozone mapped value is ratis ONE
+ Assert.assertEquals(ratis1ReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * When bucket default is EC and client side value also valid, we would just
+ * return bucket default EC.
+ */
+ @Test
+ public void testResolveRepConfWhenFSPassedIsValidButBucketDefaultEC() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .resolveClientSideReplicationConfig(
+ (short) 3, ratis3ReplicationConfig,
+ ecReplicationConfig, new OzoneConfiguration());
+ // Bucket default is EC
+ Assert.assertEquals(ecReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * When bucket default is non-EC and client side passed value also not valid
+ * but configured value is valid, we would just return configured value.
+ */
+ @Test
+ public void testResolveRepConfWhenFSPassedIsInvalidAndBucketDefaultNonEC() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .resolveClientSideReplicationConfig(
+ (short) -1, ratis3ReplicationConfig, ratis1ReplicationConfig,
+ new OzoneConfiguration());
+ // Configured value is ratis THREE
+ Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * Tests validateAndGetClientReplicationConfig with user passed valid config
+ * values.
+ */
+ @Test
+ public void testValidateAndGetRepConfWhenValidUserPassedValues() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .validateAndGetClientReplicationConfig(ReplicationType.RATIS, "1",
+ new OzoneConfiguration());
+ // Configured value is ratis ONE
+ Assert.assertEquals(ratis1ReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * Tests validateAndGetClientReplicationConfig with user passed null values.
+ */
+ @Test
+ public void testValidateAndGetRepConfWhenValidUserPassedNullValues() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .validateAndGetClientReplicationConfig(null, null,
+ new OzoneConfiguration());
+ Assert.assertNull(replicationConfig);
+ }
+
+ /**
+ * Tests validateAndGetClientReplicationConfig with user passed null values
+ * but client config has valid values.
+ */
+ @Test
+ public void testValidateAndGetRepConfWhenValidConfigValues() {
+ OzoneConfiguration clientSideConfig = new OzoneConfiguration();
+ clientSideConfig.set(OzoneConfigKeys.OZONE_REPLICATION_TYPE, "EC");
+ clientSideConfig.set(OzoneConfigKeys.OZONE_REPLICATION, "rs-3-2-1024K");
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .validateAndGetClientReplicationConfig(null, null, clientSideConfig);
+ Assert.assertEquals(ecReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * Tests validateAndGetClientReplicationConfig with user passed null values
+ * but client config has valid values.
+ */
+ @Test
+ public void testValidateAndGetRepConfWhenNullTypeFromUser() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .validateAndGetClientReplicationConfig(null, "3",
+ new OzoneConfiguration());
+ Assert.assertNull(replicationConfig);
+ }
+
+ /**
+ * Tests validateAndGetClientReplicationConfig with user passed null
+ * replication but valid type.
+ */
+ @Test
+ public void testValidateAndGetRepConfWhenNullReplicationFromUser() {
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .validateAndGetClientReplicationConfig(ReplicationType.EC, null,
+ new OzoneConfiguration());
+ Assert.assertNull(replicationConfig);
+ }
+
+ /**
+ * Tests validateAndGetClientReplicationConfig with user pass null values but
+ * config has only replication configured.
+ */
+ @Test
+ public void testValidateAndGetRepConfWhenNullTypeConfigValues() {
+ OzoneConfiguration clientSideConfig = new OzoneConfiguration();
+ clientSideConfig.set(OzoneConfigKeys.OZONE_REPLICATION, "rs-3-2-1024K");
+ //By default config values are null. Let's don't set type to keep it as
+ // null.
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .validateAndGetClientReplicationConfig(null, null, clientSideConfig);
+ Assert.assertNull(replicationConfig);
+ }
+
+ /**
+ * Tests validateAndGetClientReplicationConfig with user pass null values but
+ * config has only type configured.
+ */
+ @Test
+ public void testValidateAndGetRepConfWhenNullReplicationConfigValues() {
+ OzoneConfiguration clientSideConfig = new OzoneConfiguration();
+ clientSideConfig.set(OzoneConfigKeys.OZONE_REPLICATION_TYPE, "EC");
+ //By default config values are null. Let's don't set replication to keep it
+ // as null.
+ ReplicationConfig replicationConfig = OzoneClientUtils
+ .validateAndGetClientReplicationConfig(null, null, clientSideConfig);
+ Assert.assertNull(replicationConfig);
+ }
+
+}
diff --git a/hadoop-ozone/pom.xml b/hadoop-ozone/pom.xml
index 04a967d..96c5bea 100644
--- a/hadoop-ozone/pom.xml
+++ b/hadoop-ozone/pom.xml
@@ -134,6 +134,11 @@
</dependency>
<dependency>
<groupId>org.apache.ozone</groupId>
+ <artifactId>hdds-erasurecode</artifactId>
+ <version>${hdds.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.ozone</groupId>
<artifactId>hdds-interface-client</artifactId>
<version>${hdds.version}</version>
</dependency>
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java
index 4e6ee3a..d4b126f 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.ozone.recon.api;
-import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
@@ -105,13 +104,9 @@
try {
Pipeline pipeline = pipelineManager.getPipeline(pipelineID);
String leaderNode = pipeline.getLeaderNode().getHostName();
- DatanodePipeline datanodePipeline = new DatanodePipeline(
- pipelineID.getId(),
- pipeline.getReplicationConfig().getReplicationType().toString(),
- ReplicationConfig.getLegacyFactor(pipeline.getReplicationConfig())
- .getNumber(),
- leaderNode
- );
+ DatanodePipeline datanodePipeline =
+ new DatanodePipeline(pipelineID.getId(),
+ pipeline.getReplicationConfig(), leaderNode);
pipelines.add(datanodePipeline);
if (datanode.getUuid().equals(pipeline.getLeaderId())) {
leaderCount.getAndIncrement();
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/PipelineEndpoint.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/PipelineEndpoint.java
index d475be4..0595bbf 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/PipelineEndpoint.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/PipelineEndpoint.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.ozone.recon.api;
-import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager;
import org.apache.hadoop.ozone.recon.MetricsServiceProviderFactory;
@@ -105,9 +104,7 @@
.setDatanodes(datanodes)
.setDuration(duration)
.setStatus(pipeline.getPipelineState())
- .setReplicationFactor(ReplicationConfig
- .getLegacyFactor(pipeline.getReplicationConfig()).getNumber())
- .setReplicationType(pipeline.getType().toString());
+ .setReplicationConfig(pipeline.getReplicationConfig());
// If any metrics service providers like Prometheus
// is configured, then query it for metrics and populate
// leader election count and last leader election time
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DatanodePipeline.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DatanodePipeline.java
index cc38cd5..e302cad 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DatanodePipeline.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DatanodePipeline.java
@@ -17,6 +17,8 @@
*/
package org.apache.hadoop.ozone.recon.api.types;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+
import java.util.UUID;
/**
@@ -25,14 +27,14 @@
public class DatanodePipeline {
private UUID pipelineID;
private String replicationType;
- private int replicationFactor;
+ private String replicationFactor;
private String leaderNode;
- public DatanodePipeline(UUID pipelineID, String replicationType,
- int replicationFactor, String leaderNode) {
+ public DatanodePipeline(UUID pipelineID, ReplicationConfig replicationConfig,
+ String leaderNode) {
this.pipelineID = pipelineID;
- this.replicationType = replicationType;
- this.replicationFactor = replicationFactor;
+ this.replicationType = replicationConfig.getReplicationType().toString();
+ this.replicationFactor = replicationConfig.getReplication();
this.leaderNode = leaderNode;
}
@@ -44,7 +46,7 @@
return replicationType;
}
- public int getReplicationFactor() {
+ public String getReplicationFactor() {
return replicationFactor;
}
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/PipelineMetadata.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/PipelineMetadata.java
index 7af171f..e7facbf 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/PipelineMetadata.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/PipelineMetadata.java
@@ -19,6 +19,7 @@
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState;
import javax.xml.bind.annotation.XmlAccessType;
@@ -57,8 +58,10 @@
@XmlElement(name = "replicationType")
private String replicationType;
+ // TODO: name can be changed to just "replication". Currently EC replication
+ // also showed with below parameter but in String format.
@XmlElement(name = "replicationFactor")
- private int replicationFactor;
+ private String replicationFactor;
@XmlElement(name = "containers")
private int containers;
@@ -95,7 +98,7 @@
return replicationType;
}
- public int getReplicationFactor() {
+ public String getReplicationFactor() {
return replicationFactor;
}
@@ -139,7 +142,7 @@
private long duration;
private long leaderElections;
private String replicationType;
- private int replicationFactor;
+ private String replicationFactor;
private int containers;
public Builder() {
@@ -200,13 +203,9 @@
return this;
}
- public Builder setReplicationType(String replicationType) {
- this.replicationType = replicationType;
- return this;
- }
-
- public Builder setReplicationFactor(int replicationFactor) {
- this.replicationFactor = replicationFactor;
+ public Builder setReplicationConfig(ReplicationConfig replicationConfig) {
+ this.replicationType = replicationConfig.getReplicationType().toString();
+ this.replicationFactor = replicationConfig.getReplication();
return this;
}
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java
index 02f2751..1f4fb53 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java
@@ -19,10 +19,12 @@
package org.apache.hadoop.ozone.recon.scm;
import java.util.List;
+import java.util.Set;
import org.apache.commons.collections.map.DefaultedMap;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineFactory;
import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider;
@@ -52,10 +54,27 @@
@Override
public Pipeline create(ReplicationConfig config,
+ List<DatanodeDetails> excludedNodes,
+ List<DatanodeDetails> favoredNodes) {
+ // We don't expect this to be called at all. But adding this as a red
+ // flag for troubleshooting.
+ throw new UnsupportedOperationException(
+ "Trying to create pipeline in Recon, which is prohibited!");
+ }
+
+ @Override
+ public Pipeline create(ReplicationConfig config,
List<DatanodeDetails> nodes) {
throw new UnsupportedOperationException(
"Trying to create pipeline in Recon, which is prohibited!");
}
+
+ @Override
+ public Pipeline createForRead(ReplicationConfig config,
+ Set<ContainerReplica> replicas) {
+ throw new UnsupportedOperationException(
+ "Trying to create pipeline in Recon, which is prohibited!");
+ }
@Override
public void close(Pipeline pipeline) {
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx
index 62093d6..1781002 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/datanodes/datanodes.tsx
@@ -82,7 +82,7 @@
interface IPipeline {
pipelineID: string;
replicationType: string;
- replicationFactor: number;
+ replicationFactor: string;
leaderNode: string;
}
diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx
index 342a8bd..6810cba 100644
--- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx
+++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/pipelines/pipelines.tsx
@@ -44,7 +44,7 @@
lastLeaderElection: number;
duration: number;
leaderElections: number;
- replicationFactor: number;
+ replicationFactor: string;
containers: number;
}
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java
index 68e6d38..fd01a11 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java
@@ -419,7 +419,7 @@
Assert.assertEquals(1, datanodeMetadata.getPipelines().size());
Assert.assertEquals(pipelineId,
datanodeMetadata.getPipelines().get(0).getPipelineID().toString());
- Assert.assertEquals(pipeline.getReplicationConfig().getRequiredNodes(),
+ Assert.assertEquals(pipeline.getReplicationConfig().getReplication(),
datanodeMetadata.getPipelines().get(0).getReplicationFactor());
Assert.assertEquals(pipeline.getType().toString(),
datanodeMetadata.getPipelines().get(0).getReplicationType());
@@ -527,7 +527,7 @@
Assert.assertEquals(1, pipelineMetadata.getDatanodes().size());
Assert.assertEquals(pipeline.getType().toString(),
pipelineMetadata.getReplicationType());
- Assert.assertEquals(pipeline.getReplicationConfig().getRequiredNodes(),
+ Assert.assertEquals(pipeline.getReplicationConfig().getReplication(),
pipelineMetadata.getReplicationFactor());
Assert.assertEquals(datanodeDetails.getHostName(),
pipelineMetadata.getLeaderNode());
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
index 808a8f0..d71a99b 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
@@ -309,8 +309,7 @@
upload.getKeyName(),
upload.getUploadId(),
upload.getCreationTime(),
- S3StorageType.fromReplicationType(upload.getReplicationType(),
- upload.getReplicationFactor())
+ S3StorageType.fromReplicationConfig(upload.getReplicationConfig())
)));
AUDIT.logReadSuccess(buildAuditMessageForSuccess(s3GAction, auditParams));
getMetrics().incListMultipartUploadsSuccess();
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
index a852a3c..8824e98 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
@@ -53,7 +53,8 @@
import java.util.Map;
import java.util.OptionalLong;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.conf.StorageUnit;
@@ -78,6 +79,7 @@
import org.apache.hadoop.ozone.s3.util.RangeHeader;
import org.apache.hadoop.ozone.s3.util.RangeHeaderParserUtil;
import org.apache.hadoop.ozone.s3.util.S3StorageType;
+import org.apache.hadoop.ozone.s3.util.S3Utils;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.util.Time;
@@ -88,6 +90,9 @@
import org.apache.commons.lang3.tuple.Pair;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS;
import static org.apache.hadoop.ozone.s3.S3GatewayConfigKeys.OZONE_S3G_CLIENT_BUFFER_SIZE_DEFAULT;
import static org.apache.hadoop.ozone.s3.S3GatewayConfigKeys.OZONE_S3G_CLIENT_BUFFER_SIZE_KEY;
@@ -188,34 +193,24 @@
copyHeader = headers.getHeaderString(COPY_SOURCE_HEADER);
storageType = headers.getHeaderString(STORAGE_CLASS_HEADER);
+ boolean storageTypeDefault = StringUtils.isEmpty(storageType);
- S3StorageType s3StorageType;
- boolean storageTypeDefault;
- if (storageType == null || storageType.equals("")) {
- s3StorageType = S3StorageType.getDefault(ozoneConfiguration);
- storageTypeDefault = true;
- } else {
- s3StorageType = toS3StorageType(storageType);
- storageTypeDefault = false;
- }
- ReplicationType replicationType = s3StorageType.getType();
- ReplicationFactor replicationFactor = s3StorageType.getFactor();
+ // Normal put object
+ OzoneBucket bucket = getBucket(bucketName);
+ ReplicationConfig replicationConfig =
+ getReplicationConfig(bucket, storageType);
if (copyHeader != null) {
//Copy object, as copy source available.
s3GAction = S3GAction.COPY_OBJECT;
CopyObjectResponse copyObjectResponse = copyObject(
- copyHeader, bucketName, keyPath, replicationType,
- replicationFactor, storageTypeDefault);
+ copyHeader, bucket, keyPath, replicationConfig, storageTypeDefault);
return Response.status(Status.OK).entity(copyObjectResponse).header(
"Connection", "close").build();
}
- // Normal put object
- OzoneBucket bucket = getBucket(bucketName);
-
- output = bucket.createKey(keyPath, length, replicationType,
- replicationFactor, new HashMap<>());
+ output =
+ bucket.createKey(keyPath, length, replicationConfig, new HashMap<>());
if ("STREAMING-AWS4-HMAC-SHA256-PAYLOAD"
.equals(headers.getHeaderString("x-amz-content-sha256"))) {
@@ -578,17 +573,11 @@
OzoneBucket ozoneBucket = getBucket(bucket);
String storageType = headers.getHeaderString(STORAGE_CLASS_HEADER);
- S3StorageType s3StorageType;
- if (storageType == null || storageType.equals("")) {
- s3StorageType = S3StorageType.getDefault(ozoneConfiguration);
- } else {
- s3StorageType = toS3StorageType(storageType);
- }
- ReplicationType replicationType = s3StorageType.getType();
- ReplicationFactor replicationFactor = s3StorageType.getFactor();
+ ReplicationConfig replicationConfig =
+ getReplicationConfig(ozoneBucket, storageType);
- OmMultipartInfo multipartInfo = ozoneBucket
- .initiateMultipartUpload(key, replicationType, replicationFactor);
+ OmMultipartInfo multipartInfo =
+ ozoneBucket.initiateMultipartUpload(key, replicationConfig);
MultipartUploadInitiateResponse multipartUploadInitiateResponse = new
MultipartUploadInitiateResponse();
@@ -619,6 +608,20 @@
}
}
+ private ReplicationConfig getReplicationConfig(OzoneBucket ozoneBucket,
+ String storageType) throws OS3Exception {
+ ReplicationConfig clientConfiguredReplicationConfig = null;
+ String replication = ozoneConfiguration.get(OZONE_REPLICATION);
+ if (replication != null) {
+ clientConfiguredReplicationConfig = ReplicationConfig.parse(
+ ReplicationType.valueOf(ozoneConfiguration
+ .get(OZONE_REPLICATION_TYPE, OZONE_REPLICATION_TYPE_DEFAULT)),
+ replication, ozoneConfiguration);
+ }
+ return S3Utils.resolveS3ClientSideReplicationConfig(storageType,
+ clientConfiguredReplicationConfig, ozoneBucket.getReplicationConfig());
+ }
+
/**
* Complete a multipart upload.
*/
@@ -818,9 +821,8 @@
listPartsResponse.setPartNumberMarker(partNumberMarker);
listPartsResponse.setTruncated(false);
- listPartsResponse.setStorageClass(S3StorageType.fromReplicationType(
- ozoneMultipartUploadPartListParts.getReplicationType(),
- ozoneMultipartUploadPartListParts.getReplicationFactor()).toString());
+ listPartsResponse.setStorageClass(S3StorageType.fromReplicationConfig(
+ ozoneMultipartUploadPartListParts.getReplicationConfig()).toString());
if (ozoneMultipartUploadPartListParts.isTruncated()) {
listPartsResponse.setTruncated(
@@ -858,10 +860,9 @@
}
private CopyObjectResponse copyObject(String copyHeader,
- String destBucket,
+ OzoneBucket destBucket,
String destkey,
- ReplicationType replicationType,
- ReplicationFactor replicationFactor,
+ ReplicationConfig replicationConfig,
boolean storageTypeDefault)
throws OS3Exception, IOException {
@@ -875,7 +876,8 @@
try {
// Checking whether we trying to copying to it self.
- if (sourceBucket.equals(destBucket) && sourceKey.equals(destkey)) {
+ if (sourceBucket.equals(destBucket.getName()) && sourceKey
+ .equals(destkey)) {
// When copying to same storage type when storage type is provided,
// we should not throw exception, as aws cli checks if any of the
// options like storage type are provided or not when source and
@@ -901,15 +903,15 @@
OzoneBucket sourceOzoneBucket = getBucket(sourceBucket);
- OzoneBucket destOzoneBucket = getBucket(destBucket);
+ OzoneBucket destOzoneBucket = destBucket;
OzoneKeyDetails sourceKeyDetails = sourceOzoneBucket.getKey(sourceKey);
long sourceKeyLen = sourceKeyDetails.getDataSize();
sourceInputStream = sourceOzoneBucket.readKey(sourceKey);
- destOutputStream = destOzoneBucket.createKey(destkey, sourceKeyLen,
- replicationType, replicationFactor, new HashMap<>());
+ destOutputStream = destOzoneBucket
+ .createKey(destkey, sourceKeyLen, replicationConfig, new HashMap<>());
IOUtils.copy(sourceInputStream, destOutputStream);
@@ -977,15 +979,6 @@
}
}
- private static S3StorageType toS3StorageType(String storageType)
- throws OS3Exception {
- try {
- return S3StorageType.valueOf(storageType);
- } catch (IllegalArgumentException ex) {
- throw newError(INVALID_ARGUMENT, storageType, ex);
- }
- }
-
private static int parsePartNumberMarker(String partNumberMarker) {
int partMarker = 0;
if (partNumberMarker != null) {
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3StorageType.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3StorageType.java
index 21b3b8d..ae42e81 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3StorageType.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3StorageType.java
@@ -18,9 +18,12 @@
package org.apache.hadoop.ozone.s3.util;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.OzoneConfigKeys;
/**
@@ -70,10 +73,12 @@
? REDUCED_REDUNDANCY : STANDARD;
}
- public static S3StorageType fromReplicationType(
- ReplicationType replicationType, ReplicationFactor factor) {
- if ((replicationType == ReplicationType.STAND_ALONE) ||
- (factor == ReplicationFactor.ONE)) {
+ public static S3StorageType fromReplicationConfig(ReplicationConfig config) {
+ if (config instanceof ECReplicationConfig) {
+ return S3StorageType.STANDARD;
+ }
+ if (config.getReplicationType() == HddsProtos.ReplicationType.STAND_ALONE ||
+ config.getRequiredNodes() == 1) {
return S3StorageType.REDUCED_REDUNDANCY;
} else {
return S3StorageType.STANDARD;
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
index 7a07a89..5f110e8 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
@@ -17,6 +17,11 @@
*/
package org.apache.hadoop.ozone.s3.util;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
import org.apache.commons.lang3.StringUtils;
import java.io.UnsupportedEncodingException;
@@ -26,6 +31,8 @@
import java.util.TreeMap;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.INVALID_ARGUMENT;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.newError;
/**
* Utilities.
@@ -61,4 +68,71 @@
private S3Utils() {
// no instances
}
+
+ /**
+ * This API used to resolve the client side configuration preference for file
+ * system layer implementations.
+ *
+ * @param s3StorageTypeHeader - s3 user passed storage type
+ * header.
+ * @param clientConfiguredReplConfig - Client side configured replication
+ * config.
+ * @param bucketReplConfig - server side bucket default replication
+ * config.
+ * @return client resolved replication config.
+ */
+ public static ReplicationConfig resolveS3ClientSideReplicationConfig(
+ String s3StorageTypeHeader, ReplicationConfig clientConfiguredReplConfig,
+ ReplicationConfig bucketReplConfig)
+ throws OS3Exception {
+ ReplicationConfig clientDeterminedReplConfig = null;
+
+ // Let's map the user provided s3 storage type header to ozone s3 storage
+ // type.
+ S3StorageType s3StorageType = null;
+ if (s3StorageTypeHeader != null && !s3StorageTypeHeader.equals("")) {
+ s3StorageType = toS3StorageType(s3StorageTypeHeader);
+ }
+
+ boolean isECBucket = bucketReplConfig != null && bucketReplConfig
+ .getReplicationType() == HddsProtos.ReplicationType.EC;
+
+ // if bucket replication config configured with EC, we will give high
+ // preference to server side bucket defaults.
+ // Why we give high preference to EC is, there is no way for file system
+ // interfaces to pass EC replication. So, if one configures EC at bucket,
+ // we consider EC to take preference. in short, keys created from file
+ // system under EC bucket will always be EC'd.
+ if (isECBucket) {
+ // if bucket is EC, don't bother client provided configs, let's pass
+ // bucket config.
+ clientDeterminedReplConfig = bucketReplConfig;
+ } else {
+ // Let's validate the client side available replication configs.
+ boolean isUserPassedReplicationInSupportedList =
+ s3StorageType != null && (s3StorageType.getFactor()
+ .getValue() == ReplicationFactor.ONE.getValue() || s3StorageType
+ .getFactor().getValue() == ReplicationFactor.THREE.getValue());
+ if (isUserPassedReplicationInSupportedList) {
+ clientDeterminedReplConfig = ReplicationConfig.fromProtoTypeAndFactor(
+ ReplicationType.toProto(s3StorageType.getType()),
+ ReplicationFactor.toProto(s3StorageType.getFactor()));
+ } else {
+ // API passed replication number is not in supported replication list.
+ // So, let's use whatever available in client side configured.
+ // By default it will be null, so server will use server defaults.
+ clientDeterminedReplConfig = clientConfiguredReplConfig;
+ }
+ }
+ return clientDeterminedReplConfig;
+ }
+
+ public static S3StorageType toS3StorageType(String storageType)
+ throws OS3Exception {
+ try {
+ return S3StorageType.valueOf(storageType);
+ } catch (IllegalArgumentException ex) {
+ throw newError(INVALID_ARGUMENT, storageType, ex);
+ }
+ }
}
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
index ad12e65..20e506d 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
@@ -32,10 +32,10 @@
import java.util.stream.Collectors;
import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
-import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.OzoneAcl;
@@ -62,6 +62,7 @@
private Map<String, Map<Integer, Part>> partList = new HashMap<>();
private ArrayList<OzoneAcl> aclList = new ArrayList<>();
+ private ReplicationConfig replicationConfig;
/**
* Constructs OzoneBucket instance.
@@ -79,11 +80,11 @@
long creationTime) {
super(volumeName,
bucketName,
- StandaloneReplicationConfig
- .getInstance(HddsProtos.ReplicationFactor.ONE),
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE),
storageType,
versioning,
creationTime);
+ this.replicationConfig = super.getReplicationConfig();
}
@Override
@@ -120,6 +121,37 @@
}
@Override
+ public OzoneOutputStream createKey(String key, long size,
+ ReplicationConfig rConfig, Map<String, String> metadata)
+ throws IOException {
+ final ReplicationConfig repConfig;
+ if (rConfig == null) {
+ repConfig = getReplicationConfig();
+ } else {
+ repConfig = rConfig;
+ }
+ ReplicationConfig finalReplicationCon = repConfig;
+ ByteArrayOutputStream byteArrayOutputStream =
+ new ByteArrayOutputStream((int) size) {
+ @Override
+ public void close() throws IOException {
+ keyContents.put(key, toByteArray());
+ keyDetails.put(key, new OzoneKeyDetails(
+ getVolumeName(),
+ getName(),
+ key,
+ size,
+ System.currentTimeMillis(),
+ System.currentTimeMillis(),
+ new ArrayList<>(), finalReplicationCon, metadata, null
+ ));
+ super.close();
+ }
+ };
+ return new OzoneOutputStream(byteArrayOutputStream);
+ }
+
+ @Override
public OzoneInputStream readKey(String key) throws IOException {
return new OzoneInputStream(new ByteArrayInputStream(keyContents.get(key)));
}
@@ -143,10 +175,7 @@
ozoneKeyDetails.getDataSize(),
ozoneKeyDetails.getCreationTime().toEpochMilli(),
ozoneKeyDetails.getModificationTime().toEpochMilli(),
- ReplicationConfig.fromTypeAndFactor(
- ozoneKeyDetails.getReplicationType(),
- ReplicationFactor.valueOf(ozoneKeyDetails.getReplicationFactor())
- ));
+ ozoneKeyDetails.getReplicationConfig());
} else {
throw new OMException(ResultCodes.KEY_NOT_FOUND);
}
@@ -196,6 +225,14 @@
}
@Override
+ public OmMultipartInfo initiateMultipartUpload(String keyName,
+ ReplicationConfig repConfig) throws IOException {
+ String uploadID = UUID.randomUUID().toString();
+ multipartUploadIdMap.put(keyName, uploadID);
+ return new OmMultipartInfo(getVolumeName(), getName(), keyName, uploadID);
+ }
+
+ @Override
public OzoneOutputStream createMultipartKey(String key, long size,
int partNumber, String uploadID)
throws IOException {
@@ -278,8 +315,9 @@
List<PartInfo> partInfoList = new ArrayList<>();
if (partList.get(key) == null) {
- return new OzoneMultipartUploadPartListParts(ReplicationType.RATIS,
- ReplicationFactor.ONE, 0, false);
+ return new OzoneMultipartUploadPartListParts(
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE),
+ 0, false);
} else {
Map<Integer, Part> partMap = partList.get(key);
Iterator<Map.Entry<Integer, Part>> partIterator =
@@ -308,8 +346,7 @@
}
OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
- new OzoneMultipartUploadPartListParts(ReplicationType.RATIS,
- ReplicationFactor.ONE,
+ new OzoneMultipartUploadPartListParts(replicationConfig,
nextPartNumberMarker, truncated);
ozoneMultipartUploadPartListParts.addAllParts(partInfoList);
@@ -359,4 +396,14 @@
return content.clone();
}
}
+
+ @Override
+ public void setReplicationConfig(ReplicationConfig replicationConfig) {
+ this.replicationConfig = replicationConfig;
+ }
+
+ @Override
+ public ReplicationConfig getReplicationConfig() {
+ return this.replicationConfig;
+ }
}
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestInitiateMultipartUpload.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestInitiateMultipartUpload.java
index 2fa396f..8bd0b84 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestInitiateMultipartUpload.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestInitiateMultipartUpload.java
@@ -20,10 +20,12 @@
package org.apache.hadoop.ozone.s3.endpoint;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneClientStub;
+import org.jetbrains.annotations.NotNull;
import org.junit.Test;
import org.mockito.Mockito;
@@ -53,10 +55,7 @@
when(headers.getHeaderString(STORAGE_CLASS_HEADER)).thenReturn(
"STANDARD");
- ObjectEndpoint rest = new ObjectEndpoint();
- rest.setHeaders(headers);
- rest.setClient(client);
- rest.setOzoneConfiguration(new OzoneConfiguration());
+ ObjectEndpoint rest = getObjectEndpoint(client, headers);
Response response = rest.initializeMultipartUpload(bucket, key);
@@ -74,4 +73,32 @@
assertNotNull(multipartUploadInitiateResponse.getUploadID());
assertNotEquals(multipartUploadInitiateResponse.getUploadID(), uploadID);
}
+
+ @Test
+ public void testInitiateMultipartUploadWithECKey() throws Exception {
+ String bucket = OzoneConsts.S3_BUCKET;
+ String key = OzoneConsts.KEY;
+ OzoneClient client = new OzoneClientStub();
+ client.getObjectStore().createS3Bucket(bucket);
+ HttpHeaders headers = Mockito.mock(HttpHeaders.class);
+ ObjectEndpoint rest = getObjectEndpoint(client, headers);
+ client.getObjectStore().getS3Bucket(bucket)
+ .setReplicationConfig(new ECReplicationConfig("rs-3-2-1024K"));
+ Response response = rest.initializeMultipartUpload(bucket, key);
+
+ assertEquals(200, response.getStatus());
+ MultipartUploadInitiateResponse multipartUploadInitiateResponse =
+ (MultipartUploadInitiateResponse) response.getEntity();
+ assertNotNull(multipartUploadInitiateResponse.getUploadID());
+ }
+
+ @NotNull
+ private ObjectEndpoint getObjectEndpoint(OzoneClient client,
+ HttpHeaders headers) {
+ ObjectEndpoint rest = new ObjectEndpoint();
+ rest.setHeaders(headers);
+ rest.setClient(client);
+ rest.setOzoneConfiguration(new OzoneConfiguration());
+ return rest;
+ }
}
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java
index cf44854..170981ff 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java
@@ -25,6 +25,7 @@
import java.io.ByteArrayInputStream;
import java.io.IOException;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.client.OzoneClient;
@@ -101,6 +102,34 @@
}
@Test
+ public void testPutObjectWithECReplicationConfig()
+ throws IOException, OS3Exception {
+ //GIVEN
+ HttpHeaders headers = Mockito.mock(HttpHeaders.class);
+ ByteArrayInputStream body =
+ new ByteArrayInputStream(CONTENT.getBytes(UTF_8));
+ objectEndpoint.setHeaders(headers);
+ ECReplicationConfig ecReplicationConfig =
+ new ECReplicationConfig("rs-3-2-1024K");
+ clientStub.getObjectStore().getS3Bucket(bucketName)
+ .setReplicationConfig(ecReplicationConfig);
+ Response response = objectEndpoint.put(bucketName, keyName, CONTENT
+ .length(), 1, null, body);
+
+ Assert.assertEquals(ecReplicationConfig,
+ clientStub.getObjectStore().getS3Bucket(bucketName).getKey(keyName)
+ .getReplicationConfig());
+ OzoneInputStream ozoneInputStream =
+ clientStub.getObjectStore().getS3Bucket(bucketName)
+ .readKey(keyName);
+ String keyContent =
+ IOUtils.toString(ozoneInputStream, UTF_8);
+
+ Assert.assertEquals(200, response.getStatus());
+ Assert.assertEquals(CONTENT, keyContent);
+ }
+
+ @Test
public void testPutObjectWithSignedChunks() throws IOException, OS3Exception {
//GIVEN
HttpHeaders headers = Mockito.mock(HttpHeaders.class);
@@ -251,11 +280,11 @@
objectEndpoint.put(bucketName, keyName, CONTENT
.length(), 1, null, body);
-
OzoneKeyDetails key =
clientStub.getObjectStore().getS3Bucket(bucketName)
.getKey(keyName);
+
//default type is set
Assert.assertEquals(ReplicationType.RATIS, key.getReplicationType());
}
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
index ca20dd8..07ba62c 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
@@ -268,7 +268,7 @@
public void testPutKey() throws IOException {
Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket);
doThrow(exception).when(bucket)
- .createKey(anyString(), anyLong(), any(), any(), any());
+ .createKey(anyString(), anyLong(), any(), any());
ObjectEndpoint objectEndpoint = new ObjectEndpoint();
objectEndpoint.setClient(client);
objectEndpoint.setHeaders(headers);
@@ -304,8 +304,7 @@
@Test
public void testMultiUploadKey() throws IOException {
Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket);
- doThrow(exception).when(bucket)
- .initiateMultipartUpload(anyString(), any(), any());
+ doThrow(exception).when(bucket).initiateMultipartUpload(anyString(), any());
ObjectEndpoint objectEndpoint = new ObjectEndpoint();
objectEndpoint.setClient(client);
objectEndpoint.setHeaders(headers);
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3Utils.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3Utils.java
new file mode 100644
index 0000000..6b8c9fc
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3Utils.java
@@ -0,0 +1,143 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests the S3Utils APIs.
+ */
+public class TestS3Utils {
+ private ReplicationConfig ecReplicationConfig =
+ new ECReplicationConfig("rs-3-2-1024K");
+ private ReplicationConfig ratis3ReplicationConfig =
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE);
+ private ReplicationConfig ratis1ReplicationConfig =
+ RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.ONE);
+
+ @Test
+ public void testResolveClientSideRepConfigWhenBucketHasEC()
+ throws OS3Exception {
+ ReplicationConfig replicationConfig = S3Utils
+ .resolveS3ClientSideReplicationConfig(S3StorageType.STANDARD.name(),
+ null, ecReplicationConfig);
+ // Bucket default is EC.
+ Assert.assertEquals(ecReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * When bucket replication is null and it should respect user passed value.
+ */
+ @Test
+ public void testResolveClientSideRepConfigWhenBucketHasNull()
+ throws OS3Exception {
+ ReplicationConfig replicationConfig = S3Utils
+ .resolveS3ClientSideReplicationConfig(S3StorageType.STANDARD.name(),
+ null, null);
+ // Passed replication is 3 - Ozone mapped replication is ratis THREE
+ Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * When bucket replication is null and it should return null if user passed
+ * value is invalid.
+ */
+ @Test
+ public void testResolveClientSideRepConfigWhenUserPassedReplicationIsEmpty()
+ throws OS3Exception {
+ ReplicationConfig replicationConfig =
+ S3Utils.resolveS3ClientSideReplicationConfig("", null, null);
+ // client configured value also null.
+ // This API caller should leave the decision to server.
+ Assert.assertNull(replicationConfig);
+ }
+
+ /**
+ * When bucket default is non-EC and client side values are not valid, we
+ * would just return null, so servers can make decision in this case.
+ */
+ @Test
+ public void testResolveRepConfWhenUserPassedIsInvalidButBucketDefaultNonEC()
+ throws OS3Exception {
+ ReplicationConfig replicationConfig = S3Utils
+ .resolveS3ClientSideReplicationConfig(null, null,
+ ratis3ReplicationConfig);
+ // Configured client config also null.
+ Assert.assertNull(replicationConfig);
+ }
+
+ /**
+ * When bucket default is non-EC and client side value is valid, we
+ * would should return client side valid value.
+ */
+ @Test
+ public void testResolveRepConfWhenUserPassedIsValidButBucketDefaultNonEC()
+ throws OS3Exception {
+ ReplicationConfig replicationConfig = S3Utils
+ .resolveS3ClientSideReplicationConfig(
+ S3StorageType.REDUCED_REDUNDANCY.name(), null,
+ ratis3ReplicationConfig);
+ // Passed value is replication one - Ozone mapped value is ratis ONE
+ Assert.assertEquals(ratis1ReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * When bucket default is EC and client side value also valid, we would just
+ * return bucket default EC.
+ */
+ @Test
+ public void testResolveRepConfWhenUserPassedIsValidButBucketDefaultEC()
+ throws OS3Exception {
+ ReplicationConfig replicationConfig = S3Utils
+ .resolveS3ClientSideReplicationConfig(S3StorageType.STANDARD.name(),
+ ratis3ReplicationConfig, ecReplicationConfig);
+ // Bucket default is EC
+ Assert.assertEquals(ecReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * When bucket default is non-EC and client side passed value also not valid
+ * but configured value is valid, we would just return configured value.
+ */
+ @Test
+ public void testResolveRepConfWhenUserPassedIsInvalidAndBucketDefaultNonEC()
+ throws OS3Exception {
+ ReplicationConfig replicationConfig = S3Utils
+ .resolveS3ClientSideReplicationConfig(null, ratis3ReplicationConfig,
+ ratis1ReplicationConfig);
+ // Configured value is ratis THREE
+ Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+ }
+
+ /**
+ * When bucket default is non-EC and client side passed value also not valid
+ * but configured value is valid, we would just return configured value.
+ */
+ @Test(expected = OS3Exception.class)
+ public void testResolveRepConfWhenUserPassedIsInvalid() throws OS3Exception {
+ S3Utils.resolveS3ClientSideReplicationConfig("INVALID",
+ ratis3ReplicationConfig, ratis1ReplicationConfig);
+ }
+
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java
index a7b330c..e983229 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java
@@ -27,6 +27,7 @@
import java.util.function.Consumer;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import java.util.stream.Stream;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -345,19 +346,24 @@
return HAUtils.getScmContainerClient(ozoneConf);
}
+ @SuppressWarnings("java:S3864") // Stream.peek (for debug)
public static Pipeline findPipelineForTest(String pipelineId,
StorageContainerLocationProtocol client, Logger log) throws IOException {
- List<Pipeline> pipelines = client.listPipelines();
+ Stream<Pipeline> pipelines = client.listPipelines().stream();
Pipeline pipeline;
+ if (log.isDebugEnabled()) {
+ pipelines = pipelines
+ .peek(p -> log.debug("Found pipeline {}", p.getId().getId()));
+ }
if (pipelineId != null && pipelineId.length() > 0) {
- pipeline = pipelines.stream()
+ pipeline = pipelines
.filter(p -> p.getId().toString().equals(pipelineId))
.findFirst()
.orElseThrow(() -> new IllegalArgumentException(
"Pipeline ID is defined, but there is no such pipeline: "
+ pipelineId));
} else {
- pipeline = pipelines.stream()
+ pipeline = pipelines
.filter(p -> p.getReplicationConfig().getRequiredNodes() == 3)
.findFirst()
.orElseThrow(() -> new IllegalArgumentException(
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java
index 5d86bc5..b290da2 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java
@@ -18,15 +18,12 @@
import java.io.IOException;
import java.util.concurrent.Callable;
-import java.util.stream.Stream;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
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.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -79,7 +76,6 @@
@Override
- @SuppressWarnings("java:S3864") // Stream.peek (for debug)
public Void call() throws Exception {
init();
@@ -91,33 +87,9 @@
);
}
- try (StorageContainerLocationProtocol scmLocationClient =
+ try (StorageContainerLocationProtocol scmClient =
createStorageContainerLocationClient(ozoneConf)) {
- Stream<Pipeline> pipelines = scmLocationClient.listPipelines().stream();
- if (LOG.isDebugEnabled()) {
- pipelines = pipelines
- .peek(p -> LOG.debug("Found pipeline {}", p.getId()));
- }
- Pipeline pipeline;
- if (pipelineId != null && pipelineId.length() > 0) {
- pipeline = pipelines
- .filter(p -> p.getId().getId().toString().equals(pipelineId))
- .findFirst()
- .orElseThrow(() -> new IllegalArgumentException(
- "Pipeline ID is defined, but there is no such pipeline: "
- + pipelineId));
-
- } else {
- pipeline = pipelines
- .filter(
- p -> ReplicationConfig.getLegacyFactor(p.getReplicationConfig())
- == HddsProtos.ReplicationFactor.THREE)
- .findFirst()
- .orElseThrow(() -> new IllegalArgumentException(
- "Pipeline ID is NOT defined, and no pipeline " +
- "has been found with factor=THREE"));
- LOG.info("Using pipeline {}", pipeline.getId());
- }
+ Pipeline pipeline = findPipelineForTest(pipelineId, scmClient, LOG);
try (XceiverClientManager xceiverClientManager =
new XceiverClientManager(ozoneConf)) {
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FreonReplicationOptions.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FreonReplicationOptions.java
new file mode 100644
index 0000000..3e5ac58
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FreonReplicationOptions.java
@@ -0,0 +1,73 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.freon;
+
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.ozone.shell.ReplicationOptions;
+import picocli.CommandLine.Model.CommandSpec;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Spec;
+
+import java.util.Optional;
+
+import static picocli.CommandLine.Spec.Target.MIXEE;
+
+/**
+ * Options for specifying replication config for Freon.
+ */
+public class FreonReplicationOptions extends ReplicationOptions {
+
+ private static final String FACTOR_OPT = "--factor";
+
+ private ReplicationFactor factor;
+
+ @Spec(MIXEE)
+ private CommandSpec spec;
+
+ @Option(names = { "-F", FACTOR_OPT },
+ description = "[deprecated] Replication factor (ONE, THREE)",
+ defaultValue = "THREE"
+ )
+ public void setFactor(ReplicationFactor factor) {
+ this.factor = factor;
+ }
+
+ // -t is already taken for number of threads
+ @Option(names = {"--type", "--replication-type"},
+ description = "Replication type. Supported types are: RATIS, EC")
+ @Override
+ public void setType(ReplicationType type) {
+ super.setType(type);
+ }
+
+ /**
+ * Support legacy --factor option.
+ */
+ @Override
+ public Optional<ReplicationConfig> fromParams(ConfigurationSource conf) {
+ if (spec.commandLine().getParseResult().hasMatchedOption(FACTOR_OPT)) {
+ return Optional.of(ReplicationConfig.fromTypeAndFactor(
+ ReplicationType.RATIS, factor));
+ }
+
+ return super.fromParams(conf);
+ }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java
index 53a7705..28d3328 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmKeyGenerator.java
@@ -20,9 +20,8 @@
import java.util.concurrent.Callable;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs.Builder;
@@ -33,6 +32,7 @@
import com.codahale.metrics.Timer;
import org.apache.hadoop.security.UserGroupInformation;
import picocli.CommandLine.Command;
+import picocli.CommandLine.Mixin;
import picocli.CommandLine.Option;
import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.ALL;
@@ -61,11 +61,8 @@
defaultValue = "bucket1")
private String bucketName;
- @Option(names = { "-F", "--factor" },
- description = "Replication factor (ONE, THREE)",
- defaultValue = "THREE"
- )
- private ReplicationFactor factor = ReplicationFactor.THREE;
+ @Mixin
+ private FreonReplicationOptions replication;
@Option(
names = "--om-service-id",
@@ -76,19 +73,20 @@
private OzoneManagerProtocol ozoneManagerClient;
private Timer timer;
+ private ReplicationConfig replicationConfig;
@Override
public Void call() throws Exception {
init();
- OzoneConfiguration ozoneConfiguration = createOzoneConfiguration();
+ OzoneConfiguration conf = createOzoneConfiguration();
+ replicationConfig = replication.fromParams(conf).orElse(null);
- try (OzoneClient rpcClient = createOzoneClient(omServiceID,
- ozoneConfiguration)) {
+ try (OzoneClient rpcClient = createOzoneClient(omServiceID, conf)) {
ensureVolumeAndBucketExist(rpcClient, volumeName, bucketName);
- ozoneManagerClient = createOmClient(ozoneConfiguration, omServiceID);
+ ozoneManagerClient = createOmClient(conf, omServiceID);
timer = getMetrics().timer("key-create");
@@ -107,8 +105,8 @@
OmKeyArgs keyArgs = new Builder()
.setBucketName(bucketName)
.setVolumeName(volumeName)
- .setReplicationConfig(RatisReplicationConfig.getInstance(factor))
.setKeyName(generateObjectName(counter))
+ .setReplicationConfig(replicationConfig)
.setLocationInfoList(new ArrayList<>())
.setAcls(OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroupNames(),
ALL, ALL))
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyGenerator.java
index cf5e260..74cd0d0 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyGenerator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyGenerator.java
@@ -22,14 +22,14 @@
import java.util.concurrent.Callable;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneClient;
import com.codahale.metrics.Timer;
import picocli.CommandLine.Command;
+import picocli.CommandLine.Mixin;
import picocli.CommandLine.Option;
/**
@@ -66,23 +66,20 @@
defaultValue = "4096")
private int bufferSize;
- @Option(names = { "-F", "--factor" },
- description = "Replication factor (ONE, THREE)",
- defaultValue = "THREE"
- )
- private ReplicationFactor factor = ReplicationFactor.THREE;
-
- @Option(
- names = "--om-service-id",
+ @Option(names = "--om-service-id",
description = "OM Service ID"
)
- private String omServiceID = null;
+ private String omServiceID;
+
+ @Mixin
+ private FreonReplicationOptions replication;
private Timer timer;
private OzoneBucket bucket;
private ContentGenerator contentGenerator;
private Map<String, String> metadata;
+ private ReplicationConfig replicationConfig;
@Override
public Void call() throws Exception {
@@ -94,6 +91,8 @@
contentGenerator = new ContentGenerator(keySize, bufferSize);
metadata = new HashMap<>();
+ replicationConfig = replication.fromParamsOrConfig(ozoneConfiguration);
+
try (OzoneClient rpcClient = createOzoneClient(omServiceID,
ozoneConfiguration)) {
ensureVolumeAndBucketExist(rpcClient, volumeName, bucketName);
@@ -112,7 +111,7 @@
timer.time(() -> {
try (OutputStream stream = bucket.createKey(key, keySize,
- ReplicationType.RATIS, factor, metadata)) {
+ replicationConfig, metadata)) {
contentGenerator.write(stream);
stream.flush();
}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyValidator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyValidator.java
index 03de15c..1784ec3 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyValidator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyValidator.java
@@ -93,6 +93,8 @@
runTests(this::validateKey);
+ rpcClient.close();
+
return null;
}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
index 18ab210..75b1743 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
@@ -41,8 +41,7 @@
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.StringUtils;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.tracing.TracingUtil;
import org.apache.hadoop.ozone.OzoneConsts;
@@ -72,6 +71,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import picocli.CommandLine.Command;
+import picocli.CommandLine.Mixin;
import picocli.CommandLine.Option;
import picocli.CommandLine.ParentCommand;
@@ -177,20 +177,8 @@
)
private String jsonDir;
- @Option(
- names = {"--replication-type", "--replicationType"},
- description = "Replication type (STAND_ALONE, RATIS). Full name " +
- "--replicationType will be removed in later versions.",
- defaultValue = "STAND_ALONE"
- )
- private ReplicationType type = ReplicationType.STAND_ALONE;
-
- @Option(
- names = "--factor",
- description = "Replication factor (ONE, THREE)",
- defaultValue = "ONE"
- )
- private ReplicationFactor factor = ReplicationFactor.ONE;
+ @Mixin
+ private FreonReplicationOptions replication;
@Option(
names = "--om-service-id",
@@ -205,6 +193,8 @@
)
private boolean cleanObjects = false;
+ private ReplicationConfig replicationConfig;
+
private int threadPoolSize;
private OzoneClient ozoneClient;
@@ -248,6 +238,7 @@
private ProgressBar progressbar;
RandomKeyGenerator() {
+ // for picocli
}
@VisibleForTesting
@@ -290,18 +281,19 @@
@Override
public Void call() throws Exception {
- if (ozoneConfiguration != null) {
- if (!ozoneConfiguration.getBoolean(
- HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA,
- HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA_DEFAULT)) {
- LOG.info("Override validateWrites to false, because "
- + HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA + " is set to false.");
- validateWrites = false;
- }
- init(ozoneConfiguration);
- } else {
- init(freon.createOzoneConfiguration());
+ if (ozoneConfiguration == null) {
+ ozoneConfiguration = freon.createOzoneConfiguration();
}
+ if (!ozoneConfiguration.getBoolean(
+ HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA,
+ HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA_DEFAULT)) {
+ LOG.info("Override validateWrites to false, because "
+ + HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA + " is set to false.");
+ validateWrites = false;
+ }
+ init(ozoneConfiguration);
+
+ replicationConfig = replication.fromParamsOrConfig(ozoneConfiguration);
keyValueBuffer = StringUtils.string2Bytes(
RandomStringUtils.randomAscii(bufferSize));
@@ -491,8 +483,8 @@
out.println("Number of Volumes created: " + numberOfVolumesCreated);
out.println("Number of Buckets created: " + numberOfBucketsCreated);
out.println("Number of Keys added: " + numberOfKeysAdded);
- out.println("Ratis replication factor: " + factor.name());
- out.println("Ratis replication type: " + type.name());
+ out.println("Replication: " + replicationConfig.getReplication());
+ out.println("Replication type: " + replicationConfig.getReplicationType());
out.println(
"Average Time spent in volume creation: " + prettyAverageVolumeTime);
out.println(
@@ -812,8 +804,8 @@
try {
try (AutoCloseable scope = TracingUtil.createActivatedSpan("createKey")) {
long keyCreateStart = System.nanoTime();
- try (OzoneOutputStream os = bucket.createKey(keyName, keySize, type,
- factor, new HashMap<>())) {
+ try (OzoneOutputStream os = bucket.createKey(keyName, keySize,
+ replicationConfig, new HashMap<>())) {
long keyCreationDuration = System.nanoTime() - keyCreateStart;
histograms.get(FreonOps.KEY_CREATE.ordinal())
.update(keyCreationDuration);
@@ -936,7 +928,7 @@
private int numOfThreads;
private String dataWritten;
private String execTime;
- private String replicationFactor;
+ private String replication;
private String replicationType;
private long keySize;
@@ -969,8 +961,8 @@
this.keySize = RandomKeyGenerator.this.keySize;
this.bufferSize = RandomKeyGenerator.this.bufferSize;
this.jobStartTime = Time.formatTime(RandomKeyGenerator.this.jobStartTime);
- this.replicationFactor = RandomKeyGenerator.this.factor.name();
- this.replicationType = RandomKeyGenerator.this.type.name();
+ replicationType = replicationConfig.getReplicationType().name();
+ replication = replicationConfig.getReplication();
long totalBytes =
(long) numOfVolumes * numOfBuckets * numOfKeys * keySize;
@@ -1107,8 +1099,8 @@
return execTime;
}
- public String getReplicationFactor() {
- return replicationFactor;
+ public String getReplication() {
+ return replication;
}
public String getReplicationType() {
@@ -1229,52 +1221,7 @@
}
@VisibleForTesting
- public void setNumOfVolumes(int numOfVolumes) {
- this.numOfVolumes = numOfVolumes;
- }
-
- @VisibleForTesting
- public void setNumOfBuckets(int numOfBuckets) {
- this.numOfBuckets = numOfBuckets;
- }
-
- @VisibleForTesting
- public void setNumOfKeys(int numOfKeys) {
- this.numOfKeys = numOfKeys;
- }
-
- @VisibleForTesting
- public void setNumOfThreads(int numOfThreads) {
- this.numOfThreads = numOfThreads;
- }
-
- @VisibleForTesting
- public void setKeySize(long keySize) {
- this.keySize = keySize;
- }
-
- @VisibleForTesting
- public void setType(ReplicationType type) {
- this.type = type;
- }
-
- @VisibleForTesting
- public void setFactor(ReplicationFactor factor) {
- this.factor = factor;
- }
-
- @VisibleForTesting
- public void setValidateWrites(boolean validateWrites) {
- this.validateWrites = validateWrites;
- }
-
- @VisibleForTesting
public int getThreadPoolSize() {
return threadPoolSize;
}
-
- @VisibleForTesting
- public void setCleanObjects(boolean cleanObjects) {
- this.cleanObjects = cleanObjects;
- }
}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ReplicationOptions.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ReplicationOptions.java
new file mode 100644
index 0000000..bb4e0a7
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ReplicationOptions.java
@@ -0,0 +1,87 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.shell;
+
+import org.apache.hadoop.fs.ozone.OzoneClientUtils;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import picocli.CommandLine;
+
+import java.util.Optional;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE;
+
+/**
+ * Common options for specifying replication config: specialized for
+ * Ozone Shell and Freon commands.
+ */
+public abstract class ReplicationOptions {
+
+ private ReplicationType type;
+ private String replication;
+
+ public Optional<ReplicationConfig> fromParams(ConfigurationSource conf) {
+ if (replication == null && type == null) {
+ return Optional.empty();
+ }
+
+ // special case for backward compatibility: factor from config for RATIS
+ if (ReplicationType.RATIS == type && replication == null) {
+ replication = conf.get(OZONE_REPLICATION, OZONE_REPLICATION_DEFAULT);
+ }
+
+ return Optional.of(
+ ReplicationConfig.parseWithoutFallback(type, replication, conf));
+ }
+
+ public Optional<ReplicationConfig> fromConfig(ConfigurationSource conf) {
+ ReplicationType defaultType = Optional
+ .ofNullable(conf.get(OZONE_REPLICATION_TYPE))
+ .map(ReplicationType::valueOf)
+ .orElse(null);
+ String defaultReplication =
+ conf.get(OZONE_REPLICATION, OZONE_REPLICATION_DEFAULT);
+
+ return Optional.ofNullable(
+ OzoneClientUtils.validateAndGetClientReplicationConfig(
+ defaultType, defaultReplication, conf));
+ }
+
+ public ReplicationConfig fromParamsOrConfig(ConfigurationSource conf) {
+ return fromParams(conf)
+ .orElseGet(() -> fromConfig(conf)
+ .orElse(null));
+ }
+
+ @CommandLine.Option(names = {"--replication", "-r"},
+ description = "Replication definition. Valid values are replication"
+ + " type-specific. For RATIS: ONE or THREE."
+ + " In case of EC, pass CODEC-DATA-PARITY-CHUNKSIZE, "
+ + " e.g. rs-3-2-1024k, rs-6-3-1024k, rs-10-4-1024k")
+ public void setReplication(String replication) {
+ this.replication = replication;
+ }
+
+ // Option is defined in subclasses
+ public void setType(ReplicationType type) {
+ this.type = type;
+ }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ShellReplicationOptions.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ShellReplicationOptions.java
new file mode 100644
index 0000000..3ee20c8
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ShellReplicationOptions.java
@@ -0,0 +1,34 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.shell;
+
+import org.apache.hadoop.hdds.client.ReplicationType;
+import picocli.CommandLine.Option;
+
+/**
+ * Options for specifying replication config in 'ozone shell' commands.
+ */
+public class ShellReplicationOptions extends ReplicationOptions {
+
+ @Option(names = {"-t", "--type", "--replication-type"},
+ description = "Replication type. Supported types are: RATIS, EC")
+ @Override
+ public void setType(ReplicationType type) {
+ super.setType(type);
+ }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/BucketCommands.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/BucketCommands.java
index 2de2290..454660e 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/BucketCommands.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/BucketCommands.java
@@ -49,6 +49,7 @@
GetAclBucketHandler.class,
SetAclBucketHandler.class,
ClearQuotaHandler.class,
+ SetReplicationConfigHandler.class,
UpdateBucketHandler.class
},
mixinStandardHelpOptions = true,
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
index e1592e5..ea66429 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.ozone.shell.bucket;
import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
import org.apache.hadoop.hdds.client.OzoneQuota;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.ozone.OzoneConsts;
@@ -29,6 +30,7 @@
import org.apache.hadoop.ozone.shell.OzoneAddress;
import org.apache.hadoop.ozone.shell.SetSpaceQuotaOptions;
+import org.apache.hadoop.ozone.shell.ShellReplicationOptions;
import org.apache.hadoop.security.UserGroupInformation;
import picocli.CommandLine;
import picocli.CommandLine.Command;
@@ -65,6 +67,9 @@
private AllowedBucketLayouts allowedBucketLayout;
@CommandLine.Mixin
+ private ShellReplicationOptions replication;
+
+ @CommandLine.Mixin
private SetSpaceQuotaOptions quotaOptions;
/**
@@ -103,6 +108,9 @@
}
}
+ replication.fromParams(getConf()).ifPresent(config ->
+ bb.setDefaultReplicationConfig(new DefaultReplicationConfig(config)));
+
if (!Strings.isNullOrEmpty(quotaOptions.getQuotaInBytes())) {
bb.setQuotaInBytes(OzoneQuota.parseSpaceQuota(
quotaOptions.getQuotaInBytes()).getQuotaInBytes());
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/SetReplicationConfigHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/SetReplicationConfigHandler.java
new file mode 100644
index 0000000..45d66fd
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/SetReplicationConfigHandler.java
@@ -0,0 +1,54 @@
+/*
+ * 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.shell.bucket;
+
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.ozone.OzoneIllegalArgumentException;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.ShellReplicationOptions;
+import picocli.CommandLine;
+
+import java.io.IOException;
+
+/**
+ * set replication configuration of the bucket.
+ */
+@CommandLine.Command(name = "set-replication-config",
+ description = "Set replication config on bucket")
+public class SetReplicationConfigHandler extends BucketHandler {
+
+ @CommandLine.Mixin
+ private ShellReplicationOptions replication;
+
+ @Override
+ protected void execute(OzoneClient client, OzoneAddress address)
+ throws IOException, OzoneClientException {
+ ReplicationConfig replicationConfig = replication.fromParams(getConf())
+ .orElseThrow(() -> new OzoneIllegalArgumentException(
+ "Replication type and config must be specified."));
+
+ String volumeName = address.getVolumeName();
+ String bucketName = address.getBucketName();
+ OzoneBucket bucket =
+ client.getObjectStore().getVolume(volumeName).getBucket(bucketName);
+ bucket.setReplicationConfig(replicationConfig);
+ }
+}
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/CopyKeyHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/CopyKeyHandler.java
index 5c0aaa3..81d2dbc 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/CopyKeyHandler.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/CopyKeyHandler.java
@@ -25,7 +25,6 @@
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.client.ReplicationConfig;
-import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -34,12 +33,14 @@
import org.apache.hadoop.ozone.client.OzoneKeyDetails;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.ShellReplicationOptions;
import org.apache.hadoop.ozone.shell.bucket.BucketHandler;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY;
+
import picocli.CommandLine.Command;
-import picocli.CommandLine.Option;
+import picocli.CommandLine.Mixin;
import picocli.CommandLine.Parameters;
/**
@@ -58,17 +59,8 @@
description = "The new desired name of the key")
private String toKey;
- @Option(names = {"-r", "--replication"},
- description =
- "Replication configuration of the new key. (this is replication "
- + "specific. for RATIS/STANDALONE you can use ONE or THREE) "
- + "Default is specified in the cluster-wide config.")
- private String replication;
-
- @Option(names = {"-t", "--type"},
- description = "Replication type of the new key. (use RATIS or " +
- "STAND_ALONE) Default is specified in the cluster-wide config.")
- private ReplicationType replicationType;
+ @Mixin
+ private ShellReplicationOptions replication;
@Override
protected void execute(OzoneClient client, OzoneAddress address)
@@ -81,7 +73,7 @@
OzoneBucket bucket = vol.getBucket(bucketName);
ReplicationConfig replicationConfig =
- ReplicationConfig.parse(replicationType, replication, getConf());
+ replication.fromParamsOrConfig(getConf());
OzoneKeyDetails keyDetail = bucket.getKey(fromKey);
Map<String, String> keyMetadata = new HashMap<>(keyDetail.getMetadata());
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java
index b7fbdd1..7d7885d 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java
@@ -28,7 +28,6 @@
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.client.ReplicationConfig;
-import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -40,8 +39,10 @@
import org.apache.commons.codec.digest.DigestUtils;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY;
+
+import org.apache.hadoop.ozone.shell.ShellReplicationOptions;
import picocli.CommandLine.Command;
-import picocli.CommandLine.Option;
+import picocli.CommandLine.Mixin;
import picocli.CommandLine.Parameters;
/**
@@ -54,17 +55,8 @@
@Parameters(index = "1", arity = "1..1", description = "File to upload")
private String fileName;
- @Option(names = {"-r", "--replication"},
- description =
- "Replication configuration of the new key. (this is replication "
- + "specific. for RATIS/STANDALONE you can use ONE or THREE) "
- + "Default is specified in the cluster-wide config.")
- private String replication;
-
- @Option(names = {"-t", "--type"},
- description = "Replication type of the new key. (use RATIS or " +
- "STAND_ALONE) Default is specified in the cluster-wide config.")
- private ReplicationType replicationType;
+ @Mixin
+ private ShellReplicationOptions replication;
@Override
protected void execute(OzoneClient client, OzoneAddress address)
@@ -84,7 +76,7 @@
}
ReplicationConfig replicationConfig =
- ReplicationConfig.parse(replicationType, replication, getConf());
+ replication.fromParamsOrConfig(getConf());
OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
OzoneBucket bucket = vol.getBucket(bucketName);
diff --git a/pom.xml b/pom.xml
index b31df37..fd0578c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1175,6 +1175,12 @@
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>org.junit.jupiter</groupId>
+ <artifactId>junit-jupiter-params</artifactId>
+ <version>${junit.jupiter.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
<groupId>org.junit.vintage</groupId>
<artifactId>junit-vintage-engine</artifactId>
<version>${junit.jupiter.version}</version>