You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/05/19 05:42:00 UTC

[GitHub] [ozone] bshashikant commented on a change in pull request #2256: HDDS-5142. Make generic streaming client/service for container re-replication, data read, scm/om snapshot download

bshashikant commented on a change in pull request #2256:
URL: https://github.com/apache/ozone/pull/2256#discussion_r634912997



##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/DirectoryServerSource.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.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

Review comment:
       Can we add comment on what this "id" field signify?

##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/DirstreamServerHandler.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.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";
+
+  public static final ByteBuf END_MARKER_BUF =
+      Unpooled.wrappedBuffer(END_MARKER.getBytes(StandardCharsets.UTF_8));
+
+  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) {
+      ChannelFuture lastFuture = null;

Review comment:
       is lastFuture field required?

##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/StreamingClient.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.container.stream;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.*;

Review comment:
       Remove * imports.

##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/StreamingClient.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.container.stream;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.*;
+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;
+
+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)

Review comment:
       should we use SO_KEEP_ALIVE option as well?

##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/StreamingClient.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.container.stream;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.*;
+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;
+
+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);

Review comment:
       Just curious, why the event loop group is created with 100 threads? any rationale behind it?

##########
File path: hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/stream/DirectoryServerSource.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.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

Review comment:
       I think, it will be good to add "list of files" as a directory server source as well ? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org