diff --git a/hadoop-hdds/container-service/pom.xml b/hadoop-hdds/container-service/pom.xml
index ed2f8f4dcd61..0c64c8ffede9 100644
--- a/hadoop-hdds/container-service/pom.xml
+++ b/hadoop-hdds/container-service/pom.xml
@@ -115,6 +115,18 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
+ * 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.container.stream; + +import java.nio.file.Path; +import java.nio.file.Paths; + +/** + * Streaming binaries to single directory. + */ +public class DirectoryServerDestination implements StreamingDestination { + + private Path root; + + public DirectoryServerDestination(Path path) { + root = path; + } + + @Override + public Path mapToDestination(String name) { + return root.resolve(Paths.get(name)); + } +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/DirectoryServerSource.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/DirectoryServerSource.java new file mode 100644 index 000000000000..8759d09f7f3d --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/DirectoryServerSource.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *
+ * http://www.apache.org/licenses/LICENSE-2.0 + *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.stream;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Streaming files from single directory.
+ */
+public class DirectoryServerSource implements StreamingSource {
+
+ private Path root;
+
+ public DirectoryServerSource(Path root) {
+ this.root = root;
+ }
+
+ @Override
+ /**
+ * Return logicalNames and real file path to replicate.
+ *
+ * @param id name of the subdirectory to replitace relative to root.
+ */
+ public Map
+ * 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.container.stream;
+
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.util.ByteProcessor;
+import io.netty.util.ReferenceCountUtil;
+
+/**
+ * Protocol definition from streaming binary files.
+ *
+ * Format of the protocol (TCP/IP):
+ *
+ * LOGICAL_NAME SIZE
+ * ... (binary content)
+ * LOGICAL_NAME SIZE
+ * ... (binary content)
+ * END 0
+ */
+public class DirstreamClientHandler extends ChannelInboundHandlerAdapter {
+
+ private final StreamingDestination destination;
+ private boolean headerMode = true;
+ private StringBuilder currentFileName = new StringBuilder();
+ private RandomAccessFile destFile;
+
+ private FileChannel destFileChannel;
+
+ private long remaining;
+
+ public DirstreamClientHandler(StreamingDestination streamingDestination) {
+ this.destination = streamingDestination;
+ }
+
+ @Override
+ public void channelRead(ChannelHandlerContext ctx, Object msg)
+ throws IOException {
+ try {
+ ByteBuf buffer = (ByteBuf) msg;
+ doRead(ctx, buffer);
+ } finally {
+ ReferenceCountUtil.release(msg);
+ }
+ }
+
+ public void doRead(ChannelHandlerContext ctx, ByteBuf buffer)
+ throws IOException {
+ if (headerMode) {
+ int eolPosition = buffer.forEachByte(ByteProcessor.FIND_LF) - buffer
+ .readerIndex();
+ if (eolPosition > 0) {
+ headerMode = false;
+ final ByteBuf name = buffer.readBytes(eolPosition);
+ currentFileName.append(name
+ .toString(StandardCharsets.UTF_8));
+ name.release();
+ buffer.skipBytes(1);
+ String[] parts = currentFileName.toString().split(" ", 2);
+ remaining = Long.parseLong(parts[0]);
+ Path destFilePath = destination.mapToDestination(parts[1]);
+ final Path destfileParent = destFilePath.getParent();
+ if (destfileParent == null) {
+ throw new IllegalArgumentException("Streaming destination " +
+ "provider return with invalid path: " + destFilePath);
+ }
+ Files.createDirectories(destfileParent);
+ this.destFile =
+ new RandomAccessFile(destFilePath.toFile(), "rw");
+ destFileChannel = this.destFile.getChannel();
+
+ } else {
+ currentFileName
+ .append(buffer.toString(StandardCharsets.UTF_8));
+ }
+ }
+ if (!headerMode) {
+ final int readableBytes = buffer.readableBytes();
+ if (remaining >= readableBytes) {
+ remaining -=
+ buffer.readBytes(destFileChannel, readableBytes);
+ } else {
+ remaining -= buffer.readBytes(destFileChannel, (int) remaining);
+ currentFileName = new StringBuilder();
+ headerMode = true;
+ destFile.close();
+ if (readableBytes > 0) {
+ doRead(ctx, buffer);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void channelUnregistered(ChannelHandlerContext ctx) {
+ try {
+ if (destFile != null) {
+ destFile.close();
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+ cause.printStackTrace();
+ try {
+ destFileChannel.close();
+ destFile.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ ctx.close();
+ }
+
+ public String getCurrentFileName() {
+ return currentFileName.toString();
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/DirstreamServerHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/DirstreamServerHandler.java
new file mode 100644
index 000000000000..c609c2c85d62
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/DirstreamServerHandler.java
@@ -0,0 +1,135 @@
+/*
+ * 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.container.stream;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.DefaultFileRegion;
+import io.netty.util.ByteProcessor;
+
+/**
+ * Protocol definition of the streaming.
+ */
+public class DirstreamServerHandler extends ChannelInboundHandlerAdapter {
+
+ public static final String END_MARKER = "0 END";
+
+ private final StringBuilder id = new StringBuilder();
+
+ private StreamingSource source;
+
+ private boolean headerProcessed = false;
+
+ public DirstreamServerHandler(StreamingSource source) {
+ this.source = source;
+ }
+
+ @Override
+ public void channelRead(ChannelHandlerContext ctx, Object msg)
+ throws Exception {
+ if (!headerProcessed) {
+ ByteBuf buffer = (ByteBuf) msg;
+ int eolPosition = buffer.forEachByte(ByteProcessor.FIND_LF) - buffer
+ .readerIndex();
+ if (eolPosition > 0) {
+ headerProcessed = true;
+ id.append(buffer.toString(Charset.defaultCharset()));
+ } else {
+ id.append(buffer.toString(0, eolPosition, Charset.defaultCharset()));
+ }
+ buffer.release();
+ }
+
+ if (headerProcessed) {
+ final List
+ * 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.container.stream;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.codec.string.StringEncoder;
+import io.netty.util.CharsetUtil;
+
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.container.stream.DirstreamServerHandler.END_MARKER;
+
+/**
+ * Client to stream huge binaries from a streamling server.
+ */
+public class StreamingClient implements AutoCloseable {
+
+ private final Bootstrap bootstrap;
+ private final DirstreamClientHandler dirstreamClientHandler;
+ private EventLoopGroup group;
+ private int port;
+ private String host;
+
+ public StreamingClient(
+ String host,
+ int port,
+ StreamingDestination streamingDestination
+ ) throws InterruptedException {
+ this.port = port;
+ this.host = host;
+
+ group = new NioEventLoopGroup(100);
+ dirstreamClientHandler = new DirstreamClientHandler(streamingDestination);
+ bootstrap = new Bootstrap();
+ bootstrap.group(group)
+ .channel(NioSocketChannel.class)
+ .option(ChannelOption.SO_RCVBUF, 1024 * 1024)
+ .option(ChannelOption.SO_KEEPALIVE, true)
+ .handler(new ChannelInitializer
+ * 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.container.stream;
+
+import java.nio.file.Path;
+
+/**
+ * Interface defines the mapping to the destination.
+ */
+public interface StreamingDestination {
+
+ /**
+ * Returns destination path to each logical name.
+ */
+ Path mapToDestination(String name);
+
+}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/StreamingServer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/StreamingServer.java
new file mode 100644
index 000000000000..4ceb1569bec0
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/StreamingServer.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * 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.container.stream;
+
+import java.net.InetSocketAddress;
+
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.handler.stream.ChunkedWriteHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Netty based streaming server to replicate files from a directory.
+ */
+public class StreamingServer implements AutoCloseable {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(StreamingServer.class);
+
+ private int port;
+
+ private StreamingSource source;
+
+ private EventLoopGroup bossGroup;
+
+ private EventLoopGroup workerGroup;
+
+ public StreamingServer(
+ StreamingSource source, int port
+ ) {
+ this.port = port;
+ this.source = source;
+ }
+
+ public void start() throws InterruptedException {
+ ServerBootstrap b = new ServerBootstrap();
+ bossGroup = new NioEventLoopGroup(100);
+ workerGroup = new NioEventLoopGroup(100);
+
+ b.group(bossGroup, workerGroup)
+ .channel(NioServerSocketChannel.class)
+ .option(ChannelOption.SO_BACKLOG, 100)
+ .childHandler(new ChannelInitializer
+ * 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.container.stream;
+
+import java.nio.file.Path;
+import java.util.Map;
+
+/**
+ * Interface to define which files should be replicated for a given id.
+ */
+public interface StreamingSource {
+
+ /**
+ *
+ * @param id: custom identifier
+ *
+ * @return map of files which should be copied (logical name -> real path)
+ */
+ Map
+ * 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.container.stream;
+
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+
+/**
+ * Testing stream server.
+ */
+public class TestStreamingServer {
+
+ private static final String SUBDIR = "test1";
+
+ private static final byte[] CONTENT = "Stream it if you can"
+ .getBytes(StandardCharsets.UTF_8);
+
+ @Test
+ public void simpleStream() throws Exception {
+ Path sourceDir = GenericTestUtils.getRandomizedTestDir().toPath();
+ Path destDir = GenericTestUtils.getRandomizedTestDir().toPath();
+ Files.createDirectories(sourceDir.resolve(SUBDIR));
+ Files.createDirectories(destDir.resolve(SUBDIR));
+
+ //GIVEN: generate file
+ Files.write(sourceDir.resolve(SUBDIR).resolve("file1"), CONTENT);
+
+ //WHEN: stream subdir
+ streamDir(sourceDir, destDir, SUBDIR);
+
+ //THEN: compare the files
+ final byte[] targetContent = Files
+ .readAllBytes(destDir.resolve(SUBDIR).resolve("file1"));
+ Assert.assertArrayEquals(CONTENT, targetContent);
+
+ }
+
+
+ @Test(expected = RuntimeException.class)
+ public void failedStream() throws Exception {
+ Path sourceDir = GenericTestUtils.getRandomizedTestDir().toPath();
+ Path destDir = GenericTestUtils.getRandomizedTestDir().toPath();
+ Files.createDirectories(sourceDir.resolve(SUBDIR));
+ Files.createDirectories(destDir.resolve(SUBDIR));
+
+ //GIVEN: generate file
+ Files.write(sourceDir.resolve(SUBDIR).resolve("file1"), CONTENT);
+
+ //WHEN: stream subdir
+ streamDir(sourceDir, destDir, "NO_SUCH_ID");
+
+ //THEN: compare the files
+ //exception is expected
+
+ }
+
+ @Test(expected = RuntimeException.class)
+ public void timeout() throws Exception {
+ Path sourceDir = GenericTestUtils.getRandomizedTestDir().toPath();
+ Path destDir = GenericTestUtils.getRandomizedTestDir().toPath();
+ Files.createDirectories(sourceDir.resolve(SUBDIR));
+ Files.createDirectories(destDir.resolve(SUBDIR));
+
+ //GIVEN: generate file
+ Files.write(sourceDir.resolve(SUBDIR).resolve("file1"), CONTENT);
+
+ //WHEN: stream subdir
+ try (StreamingServer server =
+ new StreamingServer(new DirectoryServerSource(sourceDir) {
+ @Override
+ public Map
+ * 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.freon;
+
+import com.codahale.metrics.Timer;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.ozone.container.stream.DirectoryServerDestination;
+import org.apache.hadoop.ozone.container.stream.DirectoryServerSource;
+import org.apache.hadoop.ozone.container.stream.StreamingClient;
+import org.apache.hadoop.ozone.container.stream.StreamingServer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.Callable;
+
+@CommandLine.Command(name = "strmg",
+ aliases = "streaming-generator",
+ description =
+ "Create directory structure and stream them multiple times.",
+ versionProvider = HddsVersionProvider.class,
+ mixinStandardHelpOptions = true,
+ showDefaultValues = true)
+public class StreamingGenerator extends BaseFreonGenerator
+ implements Callable