You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@tajo.apache.org by hyunsik <gi...@git.apache.org> on 2015/09/16 10:25:04 UTC

[GitHub] tajo pull request: TAJO-1860: Refactor Rpc clients to take Connect...

GitHub user hyunsik opened a pull request:

    https://github.com/apache/tajo/pull/763

    TAJO-1860: Refactor Rpc clients to take Connection Parameters.

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/hyunsik/tajo TAJO-1860

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/tajo/pull/763.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #763
    
----
commit bfe5d87a788220fd3978616120bfbca32d37d28a
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-09-14T21:47:41Z

    initial work

commit 3ce490862315b783e1c9c2552473eba32061afd7
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-09-16T00:03:22Z

    Refined APIs.

commit d54347f64c81e7e6b2df08e4c564ad4cd5d78ab4
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-09-16T00:04:37Z

    Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo into TAJO-1847
    
    Conflicts:
    	tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/NettyClientBase.java

commit 598267556fa7819748661a32959c5ca65c3f20d9
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-09-16T07:49:16Z

    Completed.

commit 53f61ed998398dfbc110f14ee69772601d325e8d
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-09-16T08:06:20Z

    Add unit tests.

commit c96626cea85246246d7b3aaa52edab1aacf85ac6
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-09-16T08:06:27Z

    Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo into TAJO-1847

commit 03bba84b3a2375137fbc936e6e02659f46613fe4
Author: Hyunsik Choi <hy...@apache.org>
Date:   2015-09-16T08:22:18Z

    Removed unnecessary routine.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-1860: Refactor Rpc clients to take Connect...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/763#issuecomment-142196401
  
    I fixed bugs and reflected your comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-1860: Refactor Rpc clients to take Connect...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/763#discussion_r39628859
  
    --- Diff: tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java ---
    @@ -18,56 +18,53 @@
     
     package org.apache.tajo.rpc;
     
    -import com.google.common.annotations.VisibleForTesting;
     import com.google.protobuf.Descriptors.MethodDescriptor;
     import com.google.protobuf.*;
     import io.netty.channel.ChannelHandler;
     import io.netty.channel.EventLoopGroup;
    -import org.apache.tajo.rpc.RpcClientManager.RpcConnectionKey;
     import org.apache.tajo.rpc.RpcProtos.RpcResponse;
     
     import java.lang.reflect.Method;
    +import java.util.Properties;
     import java.util.concurrent.TimeUnit;
     import java.util.concurrent.atomic.AtomicInteger;
     
    +import static org.apache.tajo.rpc.RpcConstants.CLIENT_SOCKET_TIMEOUT;
    +import static org.apache.tajo.rpc.RpcConstants.CLIENT_SOCKET_TIMEOUT_DEFAULT;
    +
     public class AsyncRpcClient extends NettyClientBase<AsyncRpcClient.ResponseCallback> {
     
       private final Method stubMethod;
       private final ProxyRpcChannel rpcChannel;
       private final NettyChannelInboundHandler handler;
     
    -  @VisibleForTesting
    -  AsyncRpcClient(RpcConnectionKey rpcConnectionKey, int retries)
    -      throws ClassNotFoundException, NoSuchMethodException {
    -    this(rpcConnectionKey, retries, 0, TimeUnit.NANOSECONDS, false, NettyUtils.getDefaultEventLoopGroup());
    -  }
     
       /**
        * Intentionally make this method package-private, avoiding user directly
        * new an instance through this constructor.
        *
    -   * @param rpcConnectionKey
    -   * @param retries          retry operation number of times
    -   * @param timeout          disable ping, it trigger timeout event on idle-state.
    -   *                         otherwise it is request timeout on active-state
    -   * @param timeUnit         TimeUnit
    -   * @param enablePing       enable to detect remote peer hangs
    -   * @param eventLoopGroup   thread pool of netty's
    +   * @param rpcConnectionKey  RpcConnectionKey
    +   * @param eventLoopGroup    Thread pool of netty's
    +   * @param connectionParameters Connection parameters (see RpcConstants)
    +   *
        * @throws ClassNotFoundException
        * @throws NoSuchMethodException
        */
    -  AsyncRpcClient(RpcConnectionKey rpcConnectionKey, int retries, long timeout, TimeUnit timeUnit, boolean enablePing,
    -                 EventLoopGroup eventLoopGroup)
    +  AsyncRpcClient(EventLoopGroup eventLoopGroup,
    +                 RpcConnectionKey rpcConnectionKey,
    +                 Properties connectionParameters)
           throws ClassNotFoundException, NoSuchMethodException {
    -    super(rpcConnectionKey, retries);
    +    super(rpcConnectionKey, connectionParameters);
     
         this.stubMethod = getServiceClass().getMethod("newStub", RpcChannel.class);
         this.rpcChannel = new ProxyRpcChannel();
         this.handler = new ClientChannelInboundHandler();
    -    init(new ProtoClientChannelInitializer(handler,
    -        RpcResponse.getDefaultInstance(),
    -        timeUnit.toNanos(timeout),
    -        enablePing), eventLoopGroup);
    +
    +    final long socketTimeoutMills = Long.parseLong(
    +        connectionParameters.getProperty(CLIENT_SOCKET_TIMEOUT, String.valueOf(CLIENT_SOCKET_TIMEOUT_DEFAULT)));
    +
    +    init(new ProtoClientChannelInitializer(handler, RpcResponse.getDefaultInstance(),
    +        TimeUnit.MILLISECONDS.toNanos(socketTimeoutMills)),eventLoopGroup);
    --- End diff --
    
    I missed them after the change ProtoClientChannelInitializer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-1860: Refactor Rpc clients to take Connect...

Posted by jinossy <gi...@git.apache.org>.
Github user jinossy commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/763#discussion_r39621292
  
    --- Diff: tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/AsyncRpcClient.java ---
    @@ -18,56 +18,53 @@
     
     package org.apache.tajo.rpc;
     
    -import com.google.common.annotations.VisibleForTesting;
     import com.google.protobuf.Descriptors.MethodDescriptor;
     import com.google.protobuf.*;
     import io.netty.channel.ChannelHandler;
     import io.netty.channel.EventLoopGroup;
    -import org.apache.tajo.rpc.RpcClientManager.RpcConnectionKey;
     import org.apache.tajo.rpc.RpcProtos.RpcResponse;
     
     import java.lang.reflect.Method;
    +import java.util.Properties;
     import java.util.concurrent.TimeUnit;
     import java.util.concurrent.atomic.AtomicInteger;
     
    +import static org.apache.tajo.rpc.RpcConstants.CLIENT_SOCKET_TIMEOUT;
    +import static org.apache.tajo.rpc.RpcConstants.CLIENT_SOCKET_TIMEOUT_DEFAULT;
    +
     public class AsyncRpcClient extends NettyClientBase<AsyncRpcClient.ResponseCallback> {
     
       private final Method stubMethod;
       private final ProxyRpcChannel rpcChannel;
       private final NettyChannelInboundHandler handler;
     
    -  @VisibleForTesting
    -  AsyncRpcClient(RpcConnectionKey rpcConnectionKey, int retries)
    -      throws ClassNotFoundException, NoSuchMethodException {
    -    this(rpcConnectionKey, retries, 0, TimeUnit.NANOSECONDS, false, NettyUtils.getDefaultEventLoopGroup());
    -  }
     
       /**
        * Intentionally make this method package-private, avoiding user directly
        * new an instance through this constructor.
        *
    -   * @param rpcConnectionKey
    -   * @param retries          retry operation number of times
    -   * @param timeout          disable ping, it trigger timeout event on idle-state.
    -   *                         otherwise it is request timeout on active-state
    -   * @param timeUnit         TimeUnit
    -   * @param enablePing       enable to detect remote peer hangs
    -   * @param eventLoopGroup   thread pool of netty's
    +   * @param rpcConnectionKey  RpcConnectionKey
    +   * @param eventLoopGroup    Thread pool of netty's
    +   * @param connectionParameters Connection parameters (see RpcConstants)
    +   *
        * @throws ClassNotFoundException
        * @throws NoSuchMethodException
        */
    -  AsyncRpcClient(RpcConnectionKey rpcConnectionKey, int retries, long timeout, TimeUnit timeUnit, boolean enablePing,
    -                 EventLoopGroup eventLoopGroup)
    +  AsyncRpcClient(EventLoopGroup eventLoopGroup,
    +                 RpcConnectionKey rpcConnectionKey,
    +                 Properties connectionParameters)
           throws ClassNotFoundException, NoSuchMethodException {
    -    super(rpcConnectionKey, retries);
    +    super(rpcConnectionKey, connectionParameters);
     
         this.stubMethod = getServiceClass().getMethod("newStub", RpcChannel.class);
         this.rpcChannel = new ProxyRpcChannel();
         this.handler = new ClientChannelInboundHandler();
    -    init(new ProtoClientChannelInitializer(handler,
    -        RpcResponse.getDefaultInstance(),
    -        timeUnit.toNanos(timeout),
    -        enablePing), eventLoopGroup);
    +
    +    final long socketTimeoutMills = Long.parseLong(
    +        connectionParameters.getProperty(CLIENT_SOCKET_TIMEOUT, String.valueOf(CLIENT_SOCKET_TIMEOUT_DEFAULT)));
    +
    +    init(new ProtoClientChannelInitializer(handler, RpcResponse.getDefaultInstance(),
    +        TimeUnit.MILLISECONDS.toNanos(socketTimeoutMills)),eventLoopGroup);
    --- End diff --
    
    You should remove toNanos()


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-1860: Refactor Rpc clients to take Connect...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/tajo/pull/763


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-1860: Refactor Rpc clients to take Connect...

Posted by jinossy <gi...@git.apache.org>.
Github user jinossy commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/763#discussion_r39618433
  
    --- Diff: tajo-core-tests/src/test/java/org/apache/tajo/util/TestRpcConnectionParamUtil.java ---
    @@ -0,0 +1,40 @@
    +package org.apache.tajo.util;
    --- End diff --
    
    you should add license


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-1860: Refactor Rpc clients to take Connect...

Posted by hyunsik <gi...@git.apache.org>.
Github user hyunsik commented on the pull request:

    https://github.com/apache/tajo/pull/763#issuecomment-141929448
  
    I've updated the jdbc documentation.
    http://people.apache.org/~hyunsik/TAJO-1860/jdbc_driver.html


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-1860: Refactor Rpc clients to take Connect...

Posted by jinossy <gi...@git.apache.org>.
Github user jinossy commented on the pull request:

    https://github.com/apache/tajo/pull/763#issuecomment-142466226
  
    +1 Looks good to me


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] tajo pull request: TAJO-1860: Refactor Rpc clients to take Connect...

Posted by jinossy <gi...@git.apache.org>.
Github user jinossy commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/763#discussion_r39621315
  
    --- Diff: tajo-rpc/tajo-rpc-protobuf/src/main/java/org/apache/tajo/rpc/BlockingRpcClient.java ---
    @@ -18,56 +18,54 @@
     
     package org.apache.tajo.rpc;
     
    -import com.google.common.annotations.VisibleForTesting;
     import com.google.protobuf.*;
     import com.google.protobuf.Descriptors.MethodDescriptor;
     import io.netty.channel.ChannelHandler;
     import io.netty.channel.EventLoopGroup;
    -import org.apache.tajo.rpc.RpcClientManager.RpcConnectionKey;
     import org.apache.tajo.rpc.RpcProtos.RpcResponse;
     
     import java.lang.reflect.Method;
     import java.net.InetSocketAddress;
    +import java.util.Properties;
     import java.util.concurrent.*;
     import java.util.concurrent.atomic.AtomicInteger;
     
    +import static org.apache.tajo.rpc.RpcConstants.CLIENT_SOCKET_TIMEOUT;
    +import static org.apache.tajo.rpc.RpcConstants.CLIENT_SOCKET_TIMEOUT_DEFAULT;
    +
     public class BlockingRpcClient extends NettyClientBase<BlockingRpcClient.ProtoCallFuture> {
     
       private final Method stubMethod;
       private final ProxyRpcChannel rpcChannel;
       private final NettyChannelInboundHandler handler;
     
    -  @VisibleForTesting
    -  BlockingRpcClient(RpcConnectionKey rpcConnectionKey, int retries)
    -      throws NoSuchMethodException, ClassNotFoundException {
    -    this(rpcConnectionKey, retries, 0, TimeUnit.NANOSECONDS, false, NettyUtils.getDefaultEventLoopGroup());
    -  }
    -
       /**
        * Intentionally make this method package-private, avoiding user directly
        * new an instance through this constructor.
        *
    -   * @param rpcConnectionKey
    -   * @param retries          retry operation number of times
    -   * @param timeout          disable ping, it trigger timeout event on idle-state.
    -   *                         otherwise it is request timeout on active-state
    -   * @param timeUnit         TimeUnit
    -   * @param enablePing       enable to detect remote peer hangs
    -   * @param eventLoopGroup   thread pool of netty's
    +   * @param rpcConnectionKey     RpcConnectionKey
    +   * @param eventLoopGroup       Thread pool of netty's
    +   * @param connectionParameters Connection parameters (see RpcConstants)
    +   *
        * @throws ClassNotFoundException
        * @throws NoSuchMethodException
    +   * @see RpcConstants
        */
    -  BlockingRpcClient(RpcConnectionKey rpcConnectionKey, int retries, long timeout, TimeUnit timeUnit, boolean enablePing,
    -                    EventLoopGroup eventLoopGroup) throws ClassNotFoundException, NoSuchMethodException {
    -    super(rpcConnectionKey, retries);
    +  public BlockingRpcClient(EventLoopGroup eventLoopGroup,
    +                           RpcConnectionKey rpcConnectionKey,
    +                           Properties connectionParameters)
    +      throws ClassNotFoundException, NoSuchMethodException {
    +    super(rpcConnectionKey, connectionParameters);
     
         this.stubMethod = getServiceClass().getMethod("newBlockingStub", BlockingRpcChannel.class);
         this.rpcChannel = new ProxyRpcChannel();
         this.handler = new ClientChannelInboundHandler();
    -    init(new ProtoClientChannelInitializer(handler,
    -        RpcResponse.getDefaultInstance(),
    -        timeUnit.toNanos(timeout),
    -        enablePing), eventLoopGroup);
    +
    +    long socketTimeoutMills = Long.parseLong(
    +        connectionParameters.getProperty(CLIENT_SOCKET_TIMEOUT, String.valueOf(CLIENT_SOCKET_TIMEOUT_DEFAULT)));
    +
    +    init(new ProtoClientChannelInitializer(handler, RpcResponse.getDefaultInstance(),
    +        TimeUnit.MILLISECONDS.toNanos(socketTimeoutMills)),eventLoopGroup);
    --- End diff --
    
    You should remove toNanos()


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---