hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From cnaur...@apache.org
Subject [1/2] hadoop git commit: HDFS-10180. Ozone: Refactor container Namespace. Contributed by Anu Engineer.
Date Fri, 18 Mar 2016 20:54:35 GMT
Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 37e3a36a3 -> c73a32c21


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73a32c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/package-info.java
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/package-info.java
deleted file mode 100644
index fbdb396..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ozone.container.ozoneimpl;
-
-/**
- This package is contains Ozone container implementation.
-**/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73a32c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClient.java
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClient.java
deleted file mode 100644
index 0c2686d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClient.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.transport.client;
-
-import com.google.common.base.Preconditions;
-import io.netty.bootstrap.Bootstrap;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.handler.logging.LogLevel;
-import io.netty.handler.logging.LoggingHandler;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConfiguration;
-import org.apache.hadoop.ozone.container.helpers.Pipeline;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * A Client for the storageContainer protocol.
- */
-public class XceiverClient {
-  static final Logger LOG = LoggerFactory.getLogger(XceiverClient.class);
-  private final Pipeline pipeline;
-  private final OzoneConfiguration config;
-  private ChannelFuture channelFuture;
-  private Bootstrap b;
-  private EventLoopGroup group;
-
-  /**
-   * Constructs a client that can communicate with the Container framework on
-   * data nodes.
-   * @param pipeline - Pipeline that defines the machines.
-   * @param config -- Ozone Config
-   */
-  public XceiverClient(Pipeline pipeline, OzoneConfiguration config) {
-    Preconditions.checkNotNull(pipeline);
-    Preconditions.checkNotNull(config);
-    this.pipeline = pipeline;
-    this.config = config;
-  }
-
-  /**
-   * Connects to the leader in the pipeline.
-   */
-  public void connect() throws Exception {
-    if (channelFuture != null
-        && channelFuture.channel() != null
-        && channelFuture.channel().isActive()) {
-      throw new IOException("This client is already connected to a host.");
-    }
-
-    group = new NioEventLoopGroup();
-    b = new Bootstrap();
-    b.group(group)
-        .channel(NioSocketChannel.class)
-        .handler(new LoggingHandler(LogLevel.INFO))
-        .handler(new XceiverClientInitializer(this.pipeline));
-    DatanodeID leader = this.pipeline.getLeader();
-
-    // read port from the data node, on failure use default configured
-    // port.
-    int port = leader.getContainerPort();
-    if (port == 0) {
-      port = config.getInt(OzoneConfigKeys.DFS_OZONE_CONTAINER_IPC_PORT,
-          OzoneConfigKeys.DFS_OZONE_CONTAINER_IPC_PORT_DEFAULT);
-    }
-    LOG.debug("Connecting to server Port : " + port);
-    channelFuture = b.connect(leader.getHostName(), port).sync();
-  }
-
-  /**
-   * Close the client.
-   */
-  public void close() {
-    if(group != null) {
-      group.shutdownGracefully();
-    }
-
-    if (channelFuture != null) {
-      channelFuture.channel().close();
-    }
-  }
-
-  /**
-   * Sends a given command to server and gets the reply back.
-   * @param request Request
-   * @return Response to the command
-   * @throws IOException
-   */
-  public ContainerProtos.ContainerCommandResponseProto sendCommand(
-      ContainerProtos.ContainerCommandRequestProto request)
-      throws IOException {
-    if((channelFuture == null) || (!channelFuture.channel().isActive())) {
-      throw new IOException("This channel is not connected.");
-    }
-    XceiverClientHandler handler =
-        channelFuture.channel().pipeline().get(XceiverClientHandler.class);
-
-    return handler.sendCommand(request);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73a32c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClientHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClientHandler.java
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClientHandler.java
deleted file mode 100644
index 25624f4..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClientHandler.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.container.transport.client;
-
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
-import org.apache.hadoop.ozone.container.helpers.Pipeline;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-/**
- * Netty client handler.
- */
-public class XceiverClientHandler extends
-    SimpleChannelInboundHandler<ContainerProtos.ContainerCommandResponseProto> {
-
-  static final Logger LOG = LoggerFactory.getLogger(XceiverClientHandler.class);
-  private final BlockingQueue<ContainerProtos.ContainerCommandResponseProto>
-      responses = new LinkedBlockingQueue<>();
-  private final Pipeline pipeline;
-  private volatile Channel channel;
-
-  /**
-   * Constructs a client that can communicate to a container server.
-   */
-  public XceiverClientHandler(Pipeline pipeline) {
-    super(false);
-    this.pipeline = pipeline;
-  }
-
-  /**
-   * <strong>Please keep in mind that this method will be renamed to {@code
-   * messageReceived(ChannelHandlerContext, I)} in 5.0.</strong>
-   * <p>
-   * Is called for each message of type {@link ContainerProtos
-   * .ContainerCommandResponseProto}.
-   *
-   * @param ctx the {@link ChannelHandlerContext} which this {@link
-   *            SimpleChannelInboundHandler} belongs to
-   * @param msg the message to handle
-   * @throws Exception is thrown if an error occurred
-   */
-  @Override
-  public void channelRead0(ChannelHandlerContext ctx,
-                           ContainerProtos.ContainerCommandResponseProto msg)
-      throws Exception {
-    responses.add(msg);
-  }
-
-  @Override
-  public void channelRegistered(ChannelHandlerContext ctx) {
-    LOG.debug("channelRegistered: Connected to ctx");
-    channel = ctx.channel();
-  }
-
-  @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
-    LOG.info("Exception in client " + cause.toString());
-    ctx.close();
-  }
-
-  /**
-   * Since netty is async, we send a work request and then wait until a response
-   * appears in the reply queue. This is simple sync interface for clients. we
-   * should consider building async interfaces for client if this turns out to
-   * be a performance bottleneck.
-   *
-   * @param request - request.
-   * @return -- response
-   */
-  public ContainerProtos.ContainerCommandResponseProto
-      sendCommand(ContainerProtos.ContainerCommandRequestProto request) {
-
-    ContainerProtos.ContainerCommandResponseProto response;
-    channel.writeAndFlush(request);
-    boolean interrupted = false;
-    for (; ; ) {
-      try {
-        response = responses.take();
-        break;
-      } catch (InterruptedException ignore) {
-        interrupted = true;
-      }
-    }
-
-    if (interrupted) {
-      Thread.currentThread().interrupt();
-    }
-    return response;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73a32c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClientInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClientInitializer.java
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClientInitializer.java
deleted file mode 100644
index 6951f28..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/client/XceiverClientInitializer.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.container.transport.client;
-
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.handler.codec.protobuf.ProtobufDecoder;
-import io.netty.handler.codec.protobuf.ProtobufEncoder;
-import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
-import io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
-import org.apache.hadoop.ozone.container.helpers.Pipeline;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
-
-/**
- * Setup the netty pipeline.
- */
-public class XceiverClientInitializer extends
-    ChannelInitializer<SocketChannel> {
-  private final Pipeline pipeline;
-
-  /**
-   * Constructs an Initializer for the client pipeline.
-   * @param pipeline  - Pipeline.
-   */
-  public XceiverClientInitializer(Pipeline pipeline) {
-    this.pipeline = pipeline;
-  }
-
-  /**
-   * This method will be called once when the Channel is registered. After
-   * the method returns this instance will be removed from the
-   * ChannelPipeline of the Channel.
-   *
-   * @param ch   Channel which was registered.
-   * @throws Exception is thrown if an error occurs. In that case the
-   *                   Channel will be closed.
-   */
-  @Override
-  protected void initChannel(SocketChannel ch) throws Exception {
-    ChannelPipeline p = ch.pipeline();
-
-    p.addLast(new ProtobufVarint32FrameDecoder());
-    p.addLast(new ProtobufDecoder(ContainerProtos
-        .ContainerCommandResponseProto.getDefaultInstance()));
-
-    p.addLast(new ProtobufVarint32LengthFieldPrepender());
-    p.addLast(new ProtobufEncoder());
-
-    p.addLast(new XceiverClientHandler(this.pipeline));
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73a32c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServer.java
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServer.java
deleted file mode 100644
index 66ffa93..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServer.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.transport.server;
-
-import com.google.common.base.Preconditions;
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.Channel;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.handler.logging.LogLevel;
-import io.netty.handler.logging.LoggingHandler;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConfiguration;
-import org.apache.hadoop.ozone.container.interfaces.ContainerDispatcher;
-
-/**
- * Creates a netty server endpoint that acts as the communication layer for
- * Ozone containers.
- */
-public final class XceiverServer {
-  private final int port;
-  private final ContainerDispatcher storageContainer;
-
-  private EventLoopGroup bossGroup;
-  private EventLoopGroup workerGroup;
-  private Channel channel;
-
-  /**
-   * Constructs a netty server class.
-   *
-   * @param conf - Configuration
-   */
-  public XceiverServer(OzoneConfiguration conf,
-                       ContainerDispatcher dispatcher) {
-    Preconditions.checkNotNull(conf);
-    this.port = conf.getInt(OzoneConfigKeys.DFS_OZONE_CONTAINER_IPC_PORT,
-        OzoneConfigKeys.DFS_OZONE_CONTAINER_IPC_PORT_DEFAULT);
-    this.storageContainer = dispatcher;
-  }
-
-  /**
-   * Starts running the server.
-   *
-   * @throws Exception
-   */
-  public void start() throws Exception {
-    bossGroup = new NioEventLoopGroup();
-    workerGroup = new NioEventLoopGroup();
-    channel = new ServerBootstrap()
-        .group(bossGroup, workerGroup)
-        .channel(NioServerSocketChannel.class)
-        .handler(new LoggingHandler(LogLevel.INFO))
-        .childHandler(new XceiverServerInitializer(storageContainer))
-        .bind(port)
-        .syncUninterruptibly()
-        .channel();
-  }
-
-  /**
-   * Stops a running server.
-   *
-   * @throws Exception
-   */
-  public void stop() throws Exception {
-    if (bossGroup != null) {
-      bossGroup.shutdownGracefully();
-    }
-    if (workerGroup != null) {
-      workerGroup.shutdownGracefully();
-    }
-    if (channel != null) {
-      channel.close().awaitUninterruptibly();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73a32c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServerHandler.java
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServerHandler.java
deleted file mode 100644
index 887ad62..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServerHandler.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.transport.server;
-
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.SimpleChannelInboundHandler;
-import org.slf4j.LoggerFactory;
-import org.slf4j.Logger;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
-import org.apache.hadoop.ozone.container.interfaces.ContainerDispatcher;
-
-/**
- * Netty server handlers that respond to Network events.
- */
-public class XceiverServerHandler extends
-    SimpleChannelInboundHandler<ContainerCommandRequestProto> {
-
-  static final Logger LOG = LoggerFactory.getLogger(XceiverServerHandler.class);
-  private final ContainerDispatcher dispatcher;
-
-  /**
-   * Constructor for server handler.
-   * @param dispatcher - Dispatcher interface
-   */
-  public XceiverServerHandler(ContainerDispatcher dispatcher) {
-    this.dispatcher = dispatcher;
-  }
-
-  /**
-   * <strong>Please keep in mind that this method will be renamed to {@code
-   * messageReceived(ChannelHandlerContext, I)} in 5.0.</strong>
-   * <p>
-   * Is called for each message of type {@link ContainerCommandRequestProto}.
-   *
-   * @param ctx the {@link ChannelHandlerContext} which this {@link
-   *            SimpleChannelInboundHandler} belongs to
-   * @param msg the message to handle
-   * @throws Exception is thrown if an error occurred
-   */
-  @Override
-  public void channelRead0(ChannelHandlerContext ctx,
-                           ContainerCommandRequestProto msg) throws
-      Exception {
-    ContainerCommandResponseProto response = this.dispatcher.dispatch(msg);
-    LOG.debug("Writing the reponse back to client.");
-    ctx.writeAndFlush(response);
-
-  }
-
-  /**
-   * Calls {@link ChannelHandlerContext#fireExceptionCaught(Throwable)}
-   * Sub-classes may override this method to change behavior.
-   *
-   * @param ctx   - Channel Handler Context
-   * @param cause - Exception
-   */
-  @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
-      throws Exception {
-    LOG.error("An exception caught in the pipeline : " + cause.toString());
-    super.exceptionCaught(ctx, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73a32c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServerInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServerInitializer.java
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServerInitializer.java
deleted file mode 100644
index 0e57855..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/transport/server/XceiverServerInitializer.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.transport.server;
-
-import com.google.common.base.Preconditions;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.handler.codec.protobuf.ProtobufDecoder;
-import io.netty.handler.codec.protobuf.ProtobufEncoder;
-import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
-import io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
-import org.apache.hadoop.ozone.container.interfaces.ContainerDispatcher;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
-
-/**
- * Creates a channel for the XceiverServer.
- */
-public class XceiverServerInitializer extends ChannelInitializer<SocketChannel>{
-  private final ContainerDispatcher dispatcher;
-  public XceiverServerInitializer(ContainerDispatcher dispatcher) {
-    Preconditions.checkNotNull(dispatcher);
-    this.dispatcher = dispatcher;
-  }
-
-  /**
-   * This method will be called once the Channel is registered. After
-   * the method returns this instance will be removed from the {@link
-   * ChannelPipeline}
-   *
-   * @param ch the  which was registered.
-   * @throws Exception is thrown if an error occurs. In that case the channel
-   * will be closed.
-   */
-  @Override
-  protected void initChannel(SocketChannel ch) throws Exception {
-    ChannelPipeline pipeline = ch.pipeline();
-    pipeline.addLast(new ProtobufVarint32FrameDecoder());
-    pipeline.addLast(new ProtobufDecoder(ContainerCommandRequestProto
-        .getDefaultInstance()));
-    pipeline.addLast(new ProtobufVarint32LengthFieldPrepender());
-    pipeline.addLast(new ProtobufEncoder());
-    pipeline.addLast(new XceiverServerHandler(dispatcher));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73a32c2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
index 28e8afd..0d1b269 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
 import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.ozone.container.helpers.Pipeline;
+import org.apache.hadoop.ozone.container.common.helpers.Pipeline;
 
 import java.io.IOException;
 import java.net.ServerSocket;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73a32c2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java
index e49d1c4..6b134fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java
@@ -25,11 +25,14 @@ import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerComm
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
-import org.apache.hadoop.ozone.container.helpers.Pipeline;
-import org.apache.hadoop.ozone.container.interfaces.ContainerDispatcher;
-import org.apache.hadoop.ozone.container.interfaces.ContainerManager;
-import org.apache.hadoop.ozone.container.ozoneimpl.Dispatcher;
-import org.apache.hadoop.ozone.container.transport.client.XceiverClient;
+import org.apache.hadoop.ozone.container.common.helpers.Pipeline;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
+import org.apache.hadoop.ozone.container.common.impl.Dispatcher;
+import org.apache.hadoop.ozone.container.common.transport.client.XceiverClient;
+import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
+import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerHandler;
+
 import org.junit.Assert;
 import org.junit.Test;
 


Mime
View raw message