You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by sounakr <gi...@git.apache.org> on 2017/07/10 04:34:07 UTC

[GitHub] carbondata pull request #1152: [WIP] Secure Dictionary Server Implementation

GitHub user sounakr opened a pull request:

    https://github.com/apache/carbondata/pull/1152

    [WIP] Secure Dictionary Server Implementation

    Secure Dictionary Implementation Along with Non Secure.
    


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

    $ git pull https://github.com/sounakr/incubator-carbondata Secure_Dictionry

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

    https://github.com/apache/carbondata/pull/1152.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 #1152
    
----
commit d0aca3a94f07e3dc4d456e7e263dfe905dcafd8b
Author: sounak <so...@huawei.com>
Date:   2017-07-06T15:18:54Z

    Secure Dictionary Server Implementation
    
    Secure Dictionary Implementation Along with Non Secure.

----


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126879400
  
    --- Diff: integration/spark2/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClientHandler.java ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.carbondata.spark.dictionary.client;
    +
    +import java.nio.ByteBuffer;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.buffer.ByteBufAllocator;
    +import io.netty.buffer.Unpooled;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import io.netty.channel.ChannelHandlerContext;
    +import org.apache.spark.network.client.RpcResponseCallback;
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.server.OneForOneStreamManager;
    +import org.apache.spark.network.server.RpcHandler;
    +import org.apache.spark.network.server.StreamManager;
    +
    +/**
    + * Client handler to get data.
    + */
    +public class SecureDictionaryClientHandler extends RpcHandler {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SecureDictionaryClientHandler.class.getName());
    +
    +  private final BlockingQueue<DictionaryMessage> responseMsgQueue = new LinkedBlockingQueue<>();
    +
    +  private ChannelHandlerContext ctx;
    +
    +  private DictionaryChannelFutureListener channelFutureListener;
    +
    +  /**
    +   * client send request to server
    +   *
    +   * @param key DictionaryMessage
    +   * @return DictionaryMessage
    +   */
    +  public DictionaryMessage getDictionary(DictionaryMessage key, TransportClient client) {
    +    DictionaryMessage dictionaryMessage;
    +    ByteBuffer resp = null;
    +    try {
    +
    +      ByteBuf buffer = ByteBufAllocator.DEFAULT.heapBuffer();
    +      key.writeData(buffer);
    +      resp = client.sendRpcSync(buffer.nioBuffer(), 100000);
    +    } catch (Exception e) {
    --- End diff --
    
    resp failure should be handled


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126870083
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/client/NonSecureDictionaryClientHandler.java ---
    @@ -57,7 +57,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
         try {
           ByteBuf data = (ByteBuf) msg;
           DictionaryMessage key = new DictionaryMessage();
    -      key.readData(data);
    +      key.readNonSecureData(data);
    --- End diff --
    
    function can be named readskiplength


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/424/



---
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] carbondata issue #1152: [CARBONDATA-1288] Secure Dictionary Server Implement...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/459/



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126900258
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/client/NonSecureDictionaryClientHandler.java ---
    @@ -57,7 +57,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
         try {
           ByteBuf data = (ByteBuf) msg;
           DictionaryMessage key = new DictionaryMessage();
    -      key.readData(data);
    +      key.readNonSecureData(data);
    --- End diff --
    
    Done



---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Can one of the admins verify this patch?


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126935953
  
    --- Diff: integration/spark2/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClientHandler.java ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.carbondata.spark.dictionary.client;
    +
    +import java.nio.ByteBuffer;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.buffer.ByteBufAllocator;
    +import io.netty.buffer.Unpooled;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import io.netty.channel.ChannelHandlerContext;
    +import org.apache.spark.network.client.RpcResponseCallback;
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.server.OneForOneStreamManager;
    +import org.apache.spark.network.server.RpcHandler;
    +import org.apache.spark.network.server.StreamManager;
    +
    +/**
    + * Client handler to get data.
    + */
    +public class SecureDictionaryClientHandler extends RpcHandler {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SecureDictionaryClientHandler.class.getName());
    +
    +  private final BlockingQueue<DictionaryMessage> responseMsgQueue = new LinkedBlockingQueue<>();
    +
    +  private ChannelHandlerContext ctx;
    +
    +  private DictionaryChannelFutureListener channelFutureListener;
    +
    +  /**
    +   * client send request to server
    +   *
    +   * @param key DictionaryMessage
    +   * @return DictionaryMessage
    +   */
    +  public DictionaryMessage getDictionary(DictionaryMessage key, TransportClient client) {
    +    DictionaryMessage dictionaryMessage;
    +    ByteBuffer resp = null;
    +    try {
    +
    --- End diff --
    
    Handles. If response is Null the data will be handled. 


---
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] carbondata issue #1152: [CARBONDATA-1288] Secure Dictionary Server Implement...

Posted by sounakr <gi...@git.apache.org>.
Github user sounakr commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    retest this please


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126900846
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessage.java ---
    @@ -50,7 +50,32 @@
        */
       private DictionaryMessageType type;
     
    -  public void readData(ByteBuf byteBuf) {
    +  public void readNonSecureData(ByteBuf byteBuf) {
    +    byteBuf.resetReaderIndex();
    --- End diff --
    
    Done


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by sounakr <gi...@git.apache.org>.
Github user sounakr commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    retest this please


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126870879
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java ---
    @@ -14,153 +14,28 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.carbondata.core.dictionary.server;
    -
    -import org.apache.carbondata.common.logging.LogService;
    -import org.apache.carbondata.common.logging.LogServiceFactory;
    -import org.apache.carbondata.core.constants.CarbonCommonConstants;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
    -import org.apache.carbondata.core.util.CarbonProperties;
    -
    -import io.netty.bootstrap.ServerBootstrap;
    -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.NioServerSocketChannel;
    -import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    -
    -/**
    - * Dictionary Server to generate dictionary keys.
    - */
    -public class DictionaryServer {
    -
    -  private static final LogService LOGGER =
    -          LogServiceFactory.getLogService(DictionaryServer.class.getName());
     
    -  private DictionaryServerHandler dictionaryServerHandler;
    -
    -  private EventLoopGroup boss;
    -  private EventLoopGroup worker;
    -  private int port;
    -  private static Object lock = new Object();
    -  private static DictionaryServer INSTANCE = null;
    +package org.apache.carbondata.core.dictionary.server;
     
    -  private DictionaryServer(int port) {
    -    startServer(port);
    -  }
    +import org.apache.spark.SparkConf;
     
    -  public static DictionaryServer getInstance(int port) {
    -    if (INSTANCE == null) {
    -      synchronized (lock) {
    -        if (INSTANCE == null) {
    -          INSTANCE = new DictionaryServer(port);
    -        }
    -      }
    -    }
    -    return INSTANCE;
    -  }
    +public interface DictionaryServer {
     
    -  /**
    -   * start dictionary server
    -   *
    -   * @param port
    -   */
    -  private void startServer(int port) {
    -    dictionaryServerHandler = new DictionaryServerHandler();
    -    String workerThreads = CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.DICTIONARY_WORKER_THREADS,
    -            CarbonCommonConstants.DICTIONARY_WORKER_THREADS_DEFAULT);
    -    boss = new NioEventLoopGroup(1);
    -    worker = new NioEventLoopGroup(Integer.parseInt(workerThreads));
    -    // Configure the server.
    -    bindToPort(port);
    -  }
    +  public void startServer(SparkConf conf, String host, int port);
     
    -  /**
    -   * Binds dictionary server to an available port.
    -   *
    -   * @param port
    -   */
    -  private void bindToPort(int port) {
    -    long start = System.currentTimeMillis();
    -    // Configure the server.
    -    int i = 0;
    -    while (i < 10) {
    -      int newPort = port + i;
    -      try {
    -        ServerBootstrap bootstrap = new ServerBootstrap();
    -        bootstrap.group(boss, worker);
    -        bootstrap.channel(NioServerSocketChannel.class);
    -        bootstrap.childHandler(new ChannelInitializer<SocketChannel>() {
    -          @Override public void initChannel(SocketChannel ch) throws Exception {
    -            ChannelPipeline pipeline = ch.pipeline();
    -            pipeline
    -                .addLast("LengthDecoder",
    -                    new LengthFieldBasedFrameDecoder(1048576, 0,
    -                        2, 0, 2));
    -            pipeline.addLast("DictionaryServerHandler", dictionaryServerHandler);
    -          }
    -        });
    -        bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true);
    -        bootstrap.bind(newPort).sync();
    -        LOGGER.audit("Dictionary Server started, Time spent " + (System.currentTimeMillis() - start)
    -            + " Listening on port " + newPort);
    -        this.port = newPort;
    -        break;
    -      } catch (Exception e) {
    -        LOGGER.error(e, "Dictionary Server Failed to bind to port:");
    -        if (i == 9) {
    -          throw new RuntimeException("Dictionary Server Could not bind to any port");
    -        }
    -      }
    -      i++;
    -    }
    -  }
    +  public void bindToPort(SparkConf orgConf, String host, int port);
     
    -  /**
    -   *
    -   * @return Port on which the DictionaryServer has started.
    -   */
    -  public int getPort() {
    -    return port;
    -  }
    +  public void shutdown()throws Exception;
     
    -  /**
    -   * shutdown dictionary server
    -   *
    -   * @throws Exception
    -   */
    -  public void shutdown() throws Exception {
    -    LOGGER.info("Shutting down dictionary server");
    -    worker.shutdownGracefully();
    -    boss.shutdownGracefully();
    -  }
    +  public String findLocalIpAddress();
     
    +  public String getHost();
     
    +  public int getPort();
     
    -  /**
    -   * Write dictionary to the store.
    -   * @throws Exception
    -   */
    -  public void writeDictionary() throws Exception {
    -    DictionaryMessage key = new DictionaryMessage();
    -    key.setType(DictionaryMessageType.WRITE_DICTIONARY);
    -    dictionaryServerHandler.processMessage(key);
    -  }
    +  public String getSecretKey();
    --- End diff --
    
    This is common interface remove getSecretKey


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2983/



---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2995/



---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2997/



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126880211
  
    --- Diff: integration/spark2/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClientHandler.java ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.carbondata.spark.dictionary.client;
    +
    +import java.nio.ByteBuffer;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.buffer.ByteBufAllocator;
    +import io.netty.buffer.Unpooled;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import io.netty.channel.ChannelHandlerContext;
    +import org.apache.spark.network.client.RpcResponseCallback;
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.server.OneForOneStreamManager;
    +import org.apache.spark.network.server.RpcHandler;
    +import org.apache.spark.network.server.StreamManager;
    +
    +/**
    + * Client handler to get data.
    + */
    +public class SecureDictionaryClientHandler extends RpcHandler {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SecureDictionaryClientHandler.class.getName());
    +
    +  private final BlockingQueue<DictionaryMessage> responseMsgQueue = new LinkedBlockingQueue<>();
    +
    +  private ChannelHandlerContext ctx;
    +
    +  private DictionaryChannelFutureListener channelFutureListener;
    +
    +  /**
    +   * client send request to server
    +   *
    +   * @param key DictionaryMessage
    +   * @return DictionaryMessage
    +   */
    +  public DictionaryMessage getDictionary(DictionaryMessage key, TransportClient client) {
    +    DictionaryMessage dictionaryMessage;
    +    ByteBuffer resp = null;
    +    try {
    +
    +      ByteBuf buffer = ByteBufAllocator.DEFAULT.heapBuffer();
    +      key.writeData(buffer);
    +      resp = client.sendRpcSync(buffer.nioBuffer(), 100000);
    +    } catch (Exception e) {
    +      LOGGER.error(e, "Error while send request to server ");
    +    }
    +    try {
    +      ByteBuf data = Unpooled.wrappedBuffer(resp);
    +      DictionaryMessage key1 = new DictionaryMessage();
    +      if (data == null) {
    +        StringBuilder message = new StringBuilder();
    +        message.append("DictionaryMessage { ColumnName: ").append(key1.getColumnName())
    +            .append(", DictionaryValue: ").append(key1.getDictionaryValue()).append(", type: ")
    +            .append(key1.getType());
    +        throw new RuntimeException("Request timed out for key : " + message);
    +      }
    +      key1.readSecureData(data);
    +      data.release();
    +      return key1;
    +    } catch (Exception e) {
    +      LOGGER.error(e);
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override public void receive(TransportClient transportClient, ByteBuffer byteBuffer,
    +      RpcResponseCallback rpcResponseCallback) {
    +    try {
    +      ByteBuf data = Unpooled.wrappedBuffer(byteBuffer);
    +      DictionaryMessage key = new DictionaryMessage();
    +      key.readSecureData(data);
    +      data.release();
    +      responseMsgQueue.offer(key);
    +    } catch (Exception e) {
    +      LOGGER.error(e);
    +      throw e;
    +    }
    +  }
    +
    +  @Override public StreamManager getStreamManager() {
    +    return new OneForOneStreamManager();
    +  }
    +
    +  private static class DictionaryChannelFutureListener implements ChannelFutureListener {
    --- End diff --
    
    remove this listener


---
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] carbondata issue #1152: [CARBONDATA-1288] Secure Dictionary Server Implement...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/460/



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126973284
  
    --- Diff: integration/spark2/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClientHandler.java ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.carbondata.spark.dictionary.client;
    +
    +import java.nio.ByteBuffer;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.buffer.ByteBufAllocator;
    +import io.netty.buffer.Unpooled;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import io.netty.channel.ChannelHandlerContext;
    +import org.apache.spark.network.client.RpcResponseCallback;
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.server.OneForOneStreamManager;
    +import org.apache.spark.network.server.RpcHandler;
    +import org.apache.spark.network.server.StreamManager;
    +
    +/**
    + * Client handler to get data.
    + */
    +public class SecureDictionaryClientHandler extends RpcHandler {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SecureDictionaryClientHandler.class.getName());
    +
    +  private final BlockingQueue<DictionaryMessage> responseMsgQueue = new LinkedBlockingQueue<>();
    +
    +  private ChannelHandlerContext ctx;
    +
    +  private DictionaryChannelFutureListener channelFutureListener;
    +
    +  /**
    +   * client send request to server
    +   *
    +   * @param key DictionaryMessage
    +   * @return DictionaryMessage
    +   */
    +  public DictionaryMessage getDictionary(DictionaryMessage key, TransportClient client) {
    +    DictionaryMessage dictionaryMessage;
    +    ByteBuffer resp = null;
    +    try {
    +
    +      ByteBuf buffer = ByteBufAllocator.DEFAULT.heapBuffer();
    +      key.writeData(buffer);
    +      resp = client.sendRpcSync(buffer.nioBuffer(), 100000);
    +    } catch (Exception e) {
    --- End diff --
    
    aLready Handled


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/402/



---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2992/



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126904475
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java ---
    @@ -14,153 +14,28 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.carbondata.core.dictionary.server;
    -
    -import org.apache.carbondata.common.logging.LogService;
    -import org.apache.carbondata.common.logging.LogServiceFactory;
    -import org.apache.carbondata.core.constants.CarbonCommonConstants;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
    -import org.apache.carbondata.core.util.CarbonProperties;
    -
    -import io.netty.bootstrap.ServerBootstrap;
    -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.NioServerSocketChannel;
    -import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    -
    -/**
    - * Dictionary Server to generate dictionary keys.
    - */
    -public class DictionaryServer {
    -
    -  private static final LogService LOGGER =
    -          LogServiceFactory.getLogService(DictionaryServer.class.getName());
     
    -  private DictionaryServerHandler dictionaryServerHandler;
    -
    -  private EventLoopGroup boss;
    -  private EventLoopGroup worker;
    -  private int port;
    -  private static Object lock = new Object();
    -  private static DictionaryServer INSTANCE = null;
    +package org.apache.carbondata.core.dictionary.server;
     
    -  private DictionaryServer(int port) {
    -    startServer(port);
    -  }
    +import org.apache.spark.SparkConf;
     
    -  public static DictionaryServer getInstance(int port) {
    -    if (INSTANCE == null) {
    -      synchronized (lock) {
    -        if (INSTANCE == null) {
    -          INSTANCE = new DictionaryServer(port);
    -        }
    -      }
    -    }
    -    return INSTANCE;
    -  }
    +public interface DictionaryServer {
     
    -  /**
    -   * start dictionary server
    -   *
    -   * @param port
    -   */
    -  private void startServer(int port) {
    -    dictionaryServerHandler = new DictionaryServerHandler();
    -    String workerThreads = CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.DICTIONARY_WORKER_THREADS,
    -            CarbonCommonConstants.DICTIONARY_WORKER_THREADS_DEFAULT);
    -    boss = new NioEventLoopGroup(1);
    -    worker = new NioEventLoopGroup(Integer.parseInt(workerThreads));
    -    // Configure the server.
    -    bindToPort(port);
    -  }
    +  public void startServer(SparkConf conf, String host, int port);
    --- End diff --
    
    Done


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by sounakr <gi...@git.apache.org>.
Github user sounakr commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    retest this please



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126906879
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServerHandler.java ---
    @@ -76,8 +75,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
        * @param ctx
        * @param cause
        */
    -  @Override
    -  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
    +  @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
    --- End diff --
    
    Done


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Can one of the admins verify this patch?


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/37/



---

[GitHub] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126900457
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java ---
    @@ -26,6 +26,7 @@
     import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
     import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
     
    +
    --- End diff --
    
    Done


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1215/



---

[GitHub] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/3012/



---
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] carbondata issue #1152: [CARBONDATA-1288] Secure Dictionary Server Implement...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/3048/



---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by sounakr <gi...@git.apache.org>.
Github user sounakr commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    retest this please



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126904515
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java ---
    @@ -14,153 +14,28 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.carbondata.core.dictionary.server;
    -
    -import org.apache.carbondata.common.logging.LogService;
    -import org.apache.carbondata.common.logging.LogServiceFactory;
    -import org.apache.carbondata.core.constants.CarbonCommonConstants;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
    -import org.apache.carbondata.core.util.CarbonProperties;
    -
    -import io.netty.bootstrap.ServerBootstrap;
    -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.NioServerSocketChannel;
    -import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    -
    -/**
    - * Dictionary Server to generate dictionary keys.
    - */
    -public class DictionaryServer {
    -
    -  private static final LogService LOGGER =
    -          LogServiceFactory.getLogService(DictionaryServer.class.getName());
     
    -  private DictionaryServerHandler dictionaryServerHandler;
    -
    -  private EventLoopGroup boss;
    -  private EventLoopGroup worker;
    -  private int port;
    -  private static Object lock = new Object();
    -  private static DictionaryServer INSTANCE = null;
    +package org.apache.carbondata.core.dictionary.server;
     
    -  private DictionaryServer(int port) {
    -    startServer(port);
    -  }
    +import org.apache.spark.SparkConf;
     
    -  public static DictionaryServer getInstance(int port) {
    -    if (INSTANCE == null) {
    -      synchronized (lock) {
    -        if (INSTANCE == null) {
    -          INSTANCE = new DictionaryServer(port);
    -        }
    -      }
    -    }
    -    return INSTANCE;
    -  }
    +public interface DictionaryServer {
     
    -  /**
    -   * start dictionary server
    -   *
    -   * @param port
    -   */
    -  private void startServer(int port) {
    -    dictionaryServerHandler = new DictionaryServerHandler();
    -    String workerThreads = CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.DICTIONARY_WORKER_THREADS,
    -            CarbonCommonConstants.DICTIONARY_WORKER_THREADS_DEFAULT);
    -    boss = new NioEventLoopGroup(1);
    -    worker = new NioEventLoopGroup(Integer.parseInt(workerThreads));
    -    // Configure the server.
    -    bindToPort(port);
    -  }
    +  public void startServer(SparkConf conf, String host, int port);
     
    -  /**
    -   * Binds dictionary server to an available port.
    -   *
    -   * @param port
    -   */
    -  private void bindToPort(int port) {
    -    long start = System.currentTimeMillis();
    -    // Configure the server.
    -    int i = 0;
    -    while (i < 10) {
    -      int newPort = port + i;
    -      try {
    -        ServerBootstrap bootstrap = new ServerBootstrap();
    -        bootstrap.group(boss, worker);
    -        bootstrap.channel(NioServerSocketChannel.class);
    -        bootstrap.childHandler(new ChannelInitializer<SocketChannel>() {
    -          @Override public void initChannel(SocketChannel ch) throws Exception {
    -            ChannelPipeline pipeline = ch.pipeline();
    -            pipeline
    -                .addLast("LengthDecoder",
    -                    new LengthFieldBasedFrameDecoder(1048576, 0,
    -                        2, 0, 2));
    -            pipeline.addLast("DictionaryServerHandler", dictionaryServerHandler);
    -          }
    -        });
    -        bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true);
    -        bootstrap.bind(newPort).sync();
    -        LOGGER.audit("Dictionary Server started, Time spent " + (System.currentTimeMillis() - start)
    -            + " Listening on port " + newPort);
    -        this.port = newPort;
    -        break;
    -      } catch (Exception e) {
    -        LOGGER.error(e, "Dictionary Server Failed to bind to port:");
    -        if (i == 9) {
    -          throw new RuntimeException("Dictionary Server Could not bind to any port");
    -        }
    -      }
    -      i++;
    -    }
    -  }
    +  public void bindToPort(SparkConf orgConf, String host, int port);
    --- End diff --
    
    Done


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126906057
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * 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.carbondata.core.dictionary.server;
    +
    +import java.net.Inet4Address;
    +import java.net.InetAddress;
    +import java.net.InetSocketAddress;
    +import java.net.NetworkInterface;
    +import java.net.SocketException;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Enumeration;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
    +import org.apache.carbondata.core.util.CarbonProperties;
    +
    +
    +import io.netty.bootstrap.ServerBootstrap;
    +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.NioServerSocketChannel;
    +import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    +import org.apache.commons.lang3.SystemUtils;
    +import org.apache.spark.SparkConf;
    +
    +/**
    + * Dictionary Server to generate dictionary keys.
    + */
    +public class NonSecureDictionaryServer implements DictionaryServer {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(NonSecureDictionaryServer.class.getName());
    +
    +  private NonSecureDictionaryServerHandler nonSecureDictionaryServerHandler;
    +
    +  private EventLoopGroup boss;
    +  private EventLoopGroup worker;
    +  private int port;
    +  private String host;
    +  private static Object lock = new Object();
    +  private static NonSecureDictionaryServer INSTANCE = null;
    +
    +  private NonSecureDictionaryServer(int port) {
    +    startServer(null, null, port);
    +  }
    +
    +  public static DictionaryServer getInstance(int port) {
    +    if (INSTANCE == null) {
    +      synchronized (lock) {
    +        if (INSTANCE == null) {
    +          INSTANCE = new NonSecureDictionaryServer(port);
    +        }
    +      }
    +    }
    +    return INSTANCE;
    +  }
    +
    +  /**
    +   * start dictionary server
    +   *
    +   * @param port
    +   */
    +  @Override public void startServer(SparkConf conf, String host, int port) {
    +    nonSecureDictionaryServerHandler = new NonSecureDictionaryServerHandler();
    +    String workerThreads = CarbonProperties.getInstance()
    +        .getProperty(CarbonCommonConstants.DICTIONARY_WORKER_THREADS,
    +            CarbonCommonConstants.DICTIONARY_WORKER_THREADS_DEFAULT);
    +    boss = new NioEventLoopGroup(1);
    +    worker = new NioEventLoopGroup(Integer.parseInt(workerThreads));
    +    // Configure the server.
    +    bindToPort(null, host, port);
    +  }
    +
    +  /**
    +   * Binds dictionary server to an available port.
    +   *
    +   * @param port
    +   */
    +  @Override public void bindToPort(SparkConf orgConf, String host, int port) {
    +    long start = System.currentTimeMillis();
    +    // Configure the server.
    +    int i = 0;
    +    while (i < 10) {
    +      int newPort = port + i;
    +      try {
    +        ServerBootstrap bootstrap = new ServerBootstrap();
    +        bootstrap.group(boss, worker);
    +        bootstrap.channel(NioServerSocketChannel.class);
    +        bootstrap.childHandler(new ChannelInitializer<SocketChannel>() {
    +          @Override public void initChannel(SocketChannel ch) throws Exception {
    +            ChannelPipeline pipeline = ch.pipeline();
    +            pipeline
    +                .addLast("LengthDecoder", new LengthFieldBasedFrameDecoder(1048576, 0, 2, 0, 2));
    +            pipeline.addLast("NonSecureDictionaryServerHandler", nonSecureDictionaryServerHandler);
    +          }
    +        });
    +        bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true);
    +        String hostToBind = findLocalIpAddress();
    +        InetSocketAddress address = hostToBind == null ?
    +            new InetSocketAddress(newPort) :
    +            new InetSocketAddress(hostToBind, newPort);
    +        bootstrap.bind(address).sync();
    +        LOGGER.audit("Dictionary Server started, Time spent " + (System.currentTimeMillis() - start)
    +            + " Listening on port " + newPort);
    +        this.port = newPort;
    +        this.host = hostToBind;
    +        break;
    +      } catch (Exception e) {
    +        LOGGER.error(e, "Dictionary Server Failed to bind to port:");
    +        if (i == 9) {
    +          throw new RuntimeException("Dictionary Server Could not bind to any port");
    +        }
    +      }
    +      i++;
    +    }
    +  }
    +
    +  /**
    +   * @return Port on which the NonSecureDictionaryServer has started.
    +   */
    +  @Override public int getPort() {
    +    return port;
    +  }
    +
    +  @Override public String getHost() {
    +    return host;
    +  }
    +
    +  @Override public String getSecretKey() {
    +    return null;
    +  }
    +
    +  /**
    +   * shutdown dictionary server
    +   *
    +   * @throws Exception
    +   */
    +  @Override public void shutdown() throws Exception {
    +    LOGGER.info("Shutting down dictionary server");
    +    worker.shutdownGracefully();
    +    boss.shutdownGracefully();
    +  }
    +
    +  /**
    +   * Write dictionary to the store.
    +   *
    +   * @throws Exception
    +   */
    +  @Override
    +  public void writeDictionary() throws Exception {
    +    DictionaryMessage key = new DictionaryMessage();
    +    key.setType(DictionaryMessageType.WRITE_DICTIONARY);
    +    nonSecureDictionaryServerHandler.processMessage(key);
    +  }
    +
    +  /**
    +   * Write Dictionary for one table.
    +   *
    +   * @throws Exception
    +   */
    +
    +  @Override
    +  public void writeTableDictionary(String uniqueTableName) throws Exception {
    +    DictionaryMessage key = new DictionaryMessage();
    +    key.setTableUniqueName(uniqueTableName);
    +    key.setType(DictionaryMessageType.WRITE_TABLE_DICTIONARY);
    +    nonSecureDictionaryServerHandler.processMessage(key);
    +  }
    +
    +  @Override public String findLocalIpAddress() {
    --- End diff --
    
    All cannot be moved as some of the class belongs to Spark2 and some in Core.


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/403/



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126878853
  
    --- Diff: integration/spark2/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClientHandler.java ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.carbondata.spark.dictionary.client;
    +
    +import java.nio.ByteBuffer;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.buffer.ByteBufAllocator;
    +import io.netty.buffer.Unpooled;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import io.netty.channel.ChannelHandlerContext;
    +import org.apache.spark.network.client.RpcResponseCallback;
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.server.OneForOneStreamManager;
    +import org.apache.spark.network.server.RpcHandler;
    +import org.apache.spark.network.server.StreamManager;
    +
    +/**
    + * Client handler to get data.
    + */
    +public class SecureDictionaryClientHandler extends RpcHandler {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SecureDictionaryClientHandler.class.getName());
    +
    +  private final BlockingQueue<DictionaryMessage> responseMsgQueue = new LinkedBlockingQueue<>();
    +
    +  private ChannelHandlerContext ctx;
    +
    +  private DictionaryChannelFutureListener channelFutureListener;
    +
    +  /**
    +   * client send request to server
    +   *
    +   * @param key DictionaryMessage
    +   * @return DictionaryMessage
    +   */
    +  public DictionaryMessage getDictionary(DictionaryMessage key, TransportClient client) {
    +    DictionaryMessage dictionaryMessage;
    +    ByteBuffer resp = null;
    +    try {
    +
    --- End diff --
    
    clean buffers and data in finally block


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126870510
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java ---
    @@ -14,153 +14,28 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.carbondata.core.dictionary.server;
    -
    -import org.apache.carbondata.common.logging.LogService;
    -import org.apache.carbondata.common.logging.LogServiceFactory;
    -import org.apache.carbondata.core.constants.CarbonCommonConstants;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
    -import org.apache.carbondata.core.util.CarbonProperties;
    -
    -import io.netty.bootstrap.ServerBootstrap;
    -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.NioServerSocketChannel;
    -import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    -
    -/**
    - * Dictionary Server to generate dictionary keys.
    - */
    -public class DictionaryServer {
    -
    -  private static final LogService LOGGER =
    -          LogServiceFactory.getLogService(DictionaryServer.class.getName());
     
    -  private DictionaryServerHandler dictionaryServerHandler;
    -
    -  private EventLoopGroup boss;
    -  private EventLoopGroup worker;
    -  private int port;
    -  private static Object lock = new Object();
    -  private static DictionaryServer INSTANCE = null;
    +package org.apache.carbondata.core.dictionary.server;
     
    -  private DictionaryServer(int port) {
    -    startServer(port);
    -  }
    +import org.apache.spark.SparkConf;
     
    -  public static DictionaryServer getInstance(int port) {
    -    if (INSTANCE == null) {
    -      synchronized (lock) {
    -        if (INSTANCE == null) {
    -          INSTANCE = new DictionaryServer(port);
    -        }
    -      }
    -    }
    -    return INSTANCE;
    -  }
    +public interface DictionaryServer {
     
    -  /**
    -   * start dictionary server
    -   *
    -   * @param port
    -   */
    -  private void startServer(int port) {
    -    dictionaryServerHandler = new DictionaryServerHandler();
    -    String workerThreads = CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.DICTIONARY_WORKER_THREADS,
    -            CarbonCommonConstants.DICTIONARY_WORKER_THREADS_DEFAULT);
    -    boss = new NioEventLoopGroup(1);
    -    worker = new NioEventLoopGroup(Integer.parseInt(workerThreads));
    -    // Configure the server.
    -    bindToPort(port);
    -  }
    +  public void startServer(SparkConf conf, String host, int port);
    --- End diff --
    
    This interface should not contain SparkConf


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126930980
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/service/NonSecureDictionaryServiceProvider.java ---
    @@ -0,0 +1,37 @@
    +/*
    + * 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.carbondata.core.dictionary.service;
    +
    +import org.apache.carbondata.core.dictionary.client.DictionaryClient;
    +import org.apache.carbondata.core.dictionary.client.NonSecureDictionaryClient;
    +
    +public class NonSecureDictionaryServiceProvider implements DictionaryServiceProvider {
    +  private int port = 0;
    +
    +  public NonSecureDictionaryServiceProvider(int port) {
    +    this.port = port;
    +  }
    +
    +  //  @Override public DictionaryServer getDictionaryServer() {
    --- End diff --
    
    Sending the server instance in provider is not needed.


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126871630
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala ---
    @@ -59,7 +59,7 @@ class SerializableConfiguration(@transient var value: Configuration) extends Ser
       @transient
       private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
     
    -  private def writeObject(out: ObjectOutputStream): Unit =
    +  private def writeObject(out: ObjectOutputStream): Unit = {
    --- End diff --
    
    revert this change


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126870279
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessage.java ---
    @@ -50,7 +50,32 @@
        */
       private DictionaryMessageType type;
     
    -  public void readData(ByteBuf byteBuf) {
    +  public void readNonSecureData(ByteBuf byteBuf) {
    +    byteBuf.resetReaderIndex();
    --- End diff --
    
    remove this


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/417/



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126871253
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/service/DictionaryOnePassService.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.carbondata.core.dictionary.service;
    +
    +public class DictionaryOnePassService {
    --- End diff --
    
    Add class headers explaining purpose 


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126870601
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java ---
    @@ -14,153 +14,28 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.carbondata.core.dictionary.server;
    -
    -import org.apache.carbondata.common.logging.LogService;
    -import org.apache.carbondata.common.logging.LogServiceFactory;
    -import org.apache.carbondata.core.constants.CarbonCommonConstants;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
    -import org.apache.carbondata.core.util.CarbonProperties;
    -
    -import io.netty.bootstrap.ServerBootstrap;
    -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.NioServerSocketChannel;
    -import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    -
    -/**
    - * Dictionary Server to generate dictionary keys.
    - */
    -public class DictionaryServer {
    -
    -  private static final LogService LOGGER =
    -          LogServiceFactory.getLogService(DictionaryServer.class.getName());
     
    -  private DictionaryServerHandler dictionaryServerHandler;
    -
    -  private EventLoopGroup boss;
    -  private EventLoopGroup worker;
    -  private int port;
    -  private static Object lock = new Object();
    -  private static DictionaryServer INSTANCE = null;
    +package org.apache.carbondata.core.dictionary.server;
     
    -  private DictionaryServer(int port) {
    -    startServer(port);
    -  }
    +import org.apache.spark.SparkConf;
     
    -  public static DictionaryServer getInstance(int port) {
    -    if (INSTANCE == null) {
    -      synchronized (lock) {
    -        if (INSTANCE == null) {
    -          INSTANCE = new DictionaryServer(port);
    -        }
    -      }
    -    }
    -    return INSTANCE;
    -  }
    +public interface DictionaryServer {
     
    -  /**
    -   * start dictionary server
    -   *
    -   * @param port
    -   */
    -  private void startServer(int port) {
    -    dictionaryServerHandler = new DictionaryServerHandler();
    -    String workerThreads = CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.DICTIONARY_WORKER_THREADS,
    -            CarbonCommonConstants.DICTIONARY_WORKER_THREADS_DEFAULT);
    -    boss = new NioEventLoopGroup(1);
    -    worker = new NioEventLoopGroup(Integer.parseInt(workerThreads));
    -    // Configure the server.
    -    bindToPort(port);
    -  }
    +  public void startServer(SparkConf conf, String host, int port);
     
    -  /**
    -   * Binds dictionary server to an available port.
    -   *
    -   * @param port
    -   */
    -  private void bindToPort(int port) {
    -    long start = System.currentTimeMillis();
    -    // Configure the server.
    -    int i = 0;
    -    while (i < 10) {
    -      int newPort = port + i;
    -      try {
    -        ServerBootstrap bootstrap = new ServerBootstrap();
    -        bootstrap.group(boss, worker);
    -        bootstrap.channel(NioServerSocketChannel.class);
    -        bootstrap.childHandler(new ChannelInitializer<SocketChannel>() {
    -          @Override public void initChannel(SocketChannel ch) throws Exception {
    -            ChannelPipeline pipeline = ch.pipeline();
    -            pipeline
    -                .addLast("LengthDecoder",
    -                    new LengthFieldBasedFrameDecoder(1048576, 0,
    -                        2, 0, 2));
    -            pipeline.addLast("DictionaryServerHandler", dictionaryServerHandler);
    -          }
    -        });
    -        bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true);
    -        bootstrap.bind(newPort).sync();
    -        LOGGER.audit("Dictionary Server started, Time spent " + (System.currentTimeMillis() - start)
    -            + " Listening on port " + newPort);
    -        this.port = newPort;
    -        break;
    -      } catch (Exception e) {
    -        LOGGER.error(e, "Dictionary Server Failed to bind to port:");
    -        if (i == 9) {
    -          throw new RuntimeException("Dictionary Server Could not bind to any port");
    -        }
    -      }
    -      i++;
    -    }
    -  }
    +  public void bindToPort(SparkConf orgConf, String host, int port);
    --- End diff --
    
    only port is sufficient


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/3006/



---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/584/



---

[GitHub] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126932824
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala ---
    @@ -59,7 +59,7 @@ class SerializableConfiguration(@transient var value: Configuration) extends Ser
       @transient
       private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
     
    -  private def writeObject(out: ObjectOutputStream): Unit =
    +  private def writeObject(out: ObjectOutputStream): Unit = {
    --- End diff --
    
    Done


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2991/



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126871296
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/service/NonSecureDictionaryServiceProvider.java ---
    @@ -0,0 +1,37 @@
    +/*
    + * 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.carbondata.core.dictionary.service;
    +
    +import org.apache.carbondata.core.dictionary.client.DictionaryClient;
    +import org.apache.carbondata.core.dictionary.client.NonSecureDictionaryClient;
    +
    +public class NonSecureDictionaryServiceProvider implements DictionaryServiceProvider {
    +  private int port = 0;
    +
    +  public NonSecureDictionaryServiceProvider(int port) {
    +    this.port = port;
    +  }
    +
    +  //  @Override public DictionaryServer getDictionaryServer() {
    --- End diff --
    
    remove commented code


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/406/



---
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] carbondata pull request #1152: [WIP] Secure Dictionary Server Implementation

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

    https://github.com/apache/carbondata/pull/1152


---

[GitHub] carbondata issue #1152: [CARBONDATA-1288] Secure Dictionary Server Implement...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/3047/



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126936641
  
    --- Diff: integration/spark2/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClientHandler.java ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.carbondata.spark.dictionary.client;
    +
    +import java.nio.ByteBuffer;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.buffer.ByteBufAllocator;
    +import io.netty.buffer.Unpooled;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import io.netty.channel.ChannelHandlerContext;
    +import org.apache.spark.network.client.RpcResponseCallback;
    +import org.apache.spark.network.client.TransportClient;
    +import org.apache.spark.network.server.OneForOneStreamManager;
    +import org.apache.spark.network.server.RpcHandler;
    +import org.apache.spark.network.server.StreamManager;
    +
    +/**
    + * Client handler to get data.
    + */
    +public class SecureDictionaryClientHandler extends RpcHandler {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SecureDictionaryClientHandler.class.getName());
    +
    +  private final BlockingQueue<DictionaryMessage> responseMsgQueue = new LinkedBlockingQueue<>();
    +
    +  private ChannelHandlerContext ctx;
    +
    +  private DictionaryChannelFutureListener channelFutureListener;
    +
    +  /**
    +   * client send request to server
    +   *
    +   * @param key DictionaryMessage
    +   * @return DictionaryMessage
    +   */
    +  public DictionaryMessage getDictionary(DictionaryMessage key, TransportClient client) {
    +    DictionaryMessage dictionaryMessage;
    +    ByteBuffer resp = null;
    +    try {
    +
    +      ByteBuf buffer = ByteBufAllocator.DEFAULT.heapBuffer();
    +      key.writeData(buffer);
    +      resp = client.sendRpcSync(buffer.nioBuffer(), 100000);
    +    } catch (Exception e) {
    +      LOGGER.error(e, "Error while send request to server ");
    +    }
    +    try {
    +      ByteBuf data = Unpooled.wrappedBuffer(resp);
    +      DictionaryMessage key1 = new DictionaryMessage();
    +      if (data == null) {
    +        StringBuilder message = new StringBuilder();
    +        message.append("DictionaryMessage { ColumnName: ").append(key1.getColumnName())
    +            .append(", DictionaryValue: ").append(key1.getDictionaryValue()).append(", type: ")
    +            .append(key1.getType());
    +        throw new RuntimeException("Request timed out for key : " + message);
    +      }
    +      key1.readSecureData(data);
    +      data.release();
    +      return key1;
    +    } catch (Exception e) {
    +      LOGGER.error(e);
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override public void receive(TransportClient transportClient, ByteBuffer byteBuffer,
    +      RpcResponseCallback rpcResponseCallback) {
    +    try {
    +      ByteBuf data = Unpooled.wrappedBuffer(byteBuffer);
    +      DictionaryMessage key = new DictionaryMessage();
    +      key.readSecureData(data);
    +      data.release();
    +      responseMsgQueue.offer(key);
    +    } catch (Exception e) {
    +      LOGGER.error(e);
    +      throw e;
    +    }
    +  }
    +
    +  @Override public StreamManager getStreamManager() {
    +    return new OneForOneStreamManager();
    +  }
    +
    +  private static class DictionaryChannelFutureListener implements ChannelFutureListener {
    --- End diff --
    
    Done


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126880767
  
    --- Diff: integration/spark2/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServer.java ---
    @@ -0,0 +1,271 @@
    +/*
    + * 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.carbondata.spark.dictionary.server;
    +
    +import java.net.Inet4Address;
    +import java.net.InetAddress;
    +import java.net.NetworkInterface;
    +import java.net.SocketException;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Enumeration;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
    +import org.apache.carbondata.core.dictionary.server.DictionaryServer;
    +import org.apache.carbondata.core.dictionary.server.SecureDictionaryServerHandler;
    +import org.apache.carbondata.core.util.CarbonProperties;
    +
    +import com.google.common.collect.Lists;
    +import io.netty.channel.EventLoopGroup;
    +import io.netty.channel.nio.NioEventLoopGroup;
    +import org.apache.commons.lang3.SystemUtils;
    +import org.apache.spark.SecurityManager;
    +import org.apache.spark.SparkConf;
    +import org.apache.spark.network.TransportContext;
    +import org.apache.spark.network.netty.SparkTransportConf;
    +import org.apache.spark.network.sasl.SaslServerBootstrap;
    +import org.apache.spark.network.server.TransportServer;
    +import org.apache.spark.network.server.TransportServerBootstrap;
    +import org.apache.spark.network.util.TransportConf;
    +import org.apache.spark.security.CryptoStreamUtils;
    +
    +/**
    + * Dictionary Server to generate dictionary keys.
    + */
    +public class SecureDictionaryServer implements DictionaryServer {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SecureDictionaryServer.class.getName());
    +
    +  private SecureDictionaryServerHandler secureDictionaryServerHandler;
    +
    +  private EventLoopGroup boss;
    +  private EventLoopGroup worker;
    +  private int port;
    +  private String host;
    +  private String secretKey;
    +  private static Object lock = new Object();
    +  private static SecureDictionaryServer INSTANCE = null;
    +
    +  private SecureDictionaryServer(SparkConf conf, String host, int port) {
    +    startServer(conf, host, port);
    +  }
    +
    +  public static DictionaryServer getInstance(SparkConf conf, String host, int port) {
    +    if (INSTANCE == null) {
    +      synchronized (lock) {
    +        if (INSTANCE == null) {
    +          INSTANCE = new SecureDictionaryServer(conf, host, port);
    +        }
    +      }
    +    }
    +    return INSTANCE;
    +  }
    +
    +  /**
    +   * start dictionary server
    +   *
    +   * @param port
    +   */
    +  @Override
    +  public void startServer(SparkConf conf, String host, int port) {
    +    secureDictionaryServerHandler = new SecureDictionaryServerHandler();
    --- End diff --
    
    this configurations should be converted to TransportConf configurations


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126870138
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java ---
    @@ -26,6 +26,7 @@
     import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
     import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
     
    +
    --- End diff --
    
    remove extra line


---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/394/



---
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] carbondata issue #1152: [WIP] Secure Dictionary Server Implementation

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Build Success with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/408/



---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126871009
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java ---
    @@ -0,0 +1,247 @@
    +/*
    + * 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.carbondata.core.dictionary.server;
    +
    +import java.net.Inet4Address;
    +import java.net.InetAddress;
    +import java.net.InetSocketAddress;
    +import java.net.NetworkInterface;
    +import java.net.SocketException;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Enumeration;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    +import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
    +import org.apache.carbondata.core.util.CarbonProperties;
    +
    +
    +import io.netty.bootstrap.ServerBootstrap;
    +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.NioServerSocketChannel;
    +import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    +import org.apache.commons.lang3.SystemUtils;
    +import org.apache.spark.SparkConf;
    +
    +/**
    + * Dictionary Server to generate dictionary keys.
    + */
    +public class NonSecureDictionaryServer implements DictionaryServer {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(NonSecureDictionaryServer.class.getName());
    +
    +  private NonSecureDictionaryServerHandler nonSecureDictionaryServerHandler;
    +
    +  private EventLoopGroup boss;
    +  private EventLoopGroup worker;
    +  private int port;
    +  private String host;
    +  private static Object lock = new Object();
    +  private static NonSecureDictionaryServer INSTANCE = null;
    +
    +  private NonSecureDictionaryServer(int port) {
    +    startServer(null, null, port);
    +  }
    +
    +  public static DictionaryServer getInstance(int port) {
    +    if (INSTANCE == null) {
    +      synchronized (lock) {
    +        if (INSTANCE == null) {
    +          INSTANCE = new NonSecureDictionaryServer(port);
    +        }
    +      }
    +    }
    +    return INSTANCE;
    +  }
    +
    +  /**
    +   * start dictionary server
    +   *
    +   * @param port
    +   */
    +  @Override public void startServer(SparkConf conf, String host, int port) {
    +    nonSecureDictionaryServerHandler = new NonSecureDictionaryServerHandler();
    +    String workerThreads = CarbonProperties.getInstance()
    +        .getProperty(CarbonCommonConstants.DICTIONARY_WORKER_THREADS,
    +            CarbonCommonConstants.DICTIONARY_WORKER_THREADS_DEFAULT);
    +    boss = new NioEventLoopGroup(1);
    +    worker = new NioEventLoopGroup(Integer.parseInt(workerThreads));
    +    // Configure the server.
    +    bindToPort(null, host, port);
    +  }
    +
    +  /**
    +   * Binds dictionary server to an available port.
    +   *
    +   * @param port
    +   */
    +  @Override public void bindToPort(SparkConf orgConf, String host, int port) {
    +    long start = System.currentTimeMillis();
    +    // Configure the server.
    +    int i = 0;
    +    while (i < 10) {
    +      int newPort = port + i;
    +      try {
    +        ServerBootstrap bootstrap = new ServerBootstrap();
    +        bootstrap.group(boss, worker);
    +        bootstrap.channel(NioServerSocketChannel.class);
    +        bootstrap.childHandler(new ChannelInitializer<SocketChannel>() {
    +          @Override public void initChannel(SocketChannel ch) throws Exception {
    +            ChannelPipeline pipeline = ch.pipeline();
    +            pipeline
    +                .addLast("LengthDecoder", new LengthFieldBasedFrameDecoder(1048576, 0, 2, 0, 2));
    +            pipeline.addLast("NonSecureDictionaryServerHandler", nonSecureDictionaryServerHandler);
    +          }
    +        });
    +        bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true);
    +        String hostToBind = findLocalIpAddress();
    +        InetSocketAddress address = hostToBind == null ?
    +            new InetSocketAddress(newPort) :
    +            new InetSocketAddress(hostToBind, newPort);
    +        bootstrap.bind(address).sync();
    +        LOGGER.audit("Dictionary Server started, Time spent " + (System.currentTimeMillis() - start)
    +            + " Listening on port " + newPort);
    +        this.port = newPort;
    +        this.host = hostToBind;
    +        break;
    +      } catch (Exception e) {
    +        LOGGER.error(e, "Dictionary Server Failed to bind to port:");
    +        if (i == 9) {
    +          throw new RuntimeException("Dictionary Server Could not bind to any port");
    +        }
    +      }
    +      i++;
    +    }
    +  }
    +
    +  /**
    +   * @return Port on which the NonSecureDictionaryServer has started.
    +   */
    +  @Override public int getPort() {
    +    return port;
    +  }
    +
    +  @Override public String getHost() {
    +    return host;
    +  }
    +
    +  @Override public String getSecretKey() {
    +    return null;
    +  }
    +
    +  /**
    +   * shutdown dictionary server
    +   *
    +   * @throws Exception
    +   */
    +  @Override public void shutdown() throws Exception {
    +    LOGGER.info("Shutting down dictionary server");
    +    worker.shutdownGracefully();
    +    boss.shutdownGracefully();
    +  }
    +
    +  /**
    +   * Write dictionary to the store.
    +   *
    +   * @throws Exception
    +   */
    +  @Override
    +  public void writeDictionary() throws Exception {
    +    DictionaryMessage key = new DictionaryMessage();
    +    key.setType(DictionaryMessageType.WRITE_DICTIONARY);
    +    nonSecureDictionaryServerHandler.processMessage(key);
    +  }
    +
    +  /**
    +   * Write Dictionary for one table.
    +   *
    +   * @throws Exception
    +   */
    +
    +  @Override
    +  public void writeTableDictionary(String uniqueTableName) throws Exception {
    +    DictionaryMessage key = new DictionaryMessage();
    +    key.setTableUniqueName(uniqueTableName);
    +    key.setType(DictionaryMessageType.WRITE_TABLE_DICTIONARY);
    +    nonSecureDictionaryServerHandler.processMessage(key);
    +  }
    +
    +  @Override public String findLocalIpAddress() {
    --- End diff --
    
    move all the common function to abstract class


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126904754
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java ---
    @@ -14,153 +14,28 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.carbondata.core.dictionary.server;
    -
    -import org.apache.carbondata.common.logging.LogService;
    -import org.apache.carbondata.common.logging.LogServiceFactory;
    -import org.apache.carbondata.core.constants.CarbonCommonConstants;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
    -import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
    -import org.apache.carbondata.core.util.CarbonProperties;
    -
    -import io.netty.bootstrap.ServerBootstrap;
    -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.NioServerSocketChannel;
    -import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
    -
    -/**
    - * Dictionary Server to generate dictionary keys.
    - */
    -public class DictionaryServer {
    -
    -  private static final LogService LOGGER =
    -          LogServiceFactory.getLogService(DictionaryServer.class.getName());
     
    -  private DictionaryServerHandler dictionaryServerHandler;
    -
    -  private EventLoopGroup boss;
    -  private EventLoopGroup worker;
    -  private int port;
    -  private static Object lock = new Object();
    -  private static DictionaryServer INSTANCE = null;
    +package org.apache.carbondata.core.dictionary.server;
     
    -  private DictionaryServer(int port) {
    -    startServer(port);
    -  }
    +import org.apache.spark.SparkConf;
     
    -  public static DictionaryServer getInstance(int port) {
    -    if (INSTANCE == null) {
    -      synchronized (lock) {
    -        if (INSTANCE == null) {
    -          INSTANCE = new DictionaryServer(port);
    -        }
    -      }
    -    }
    -    return INSTANCE;
    -  }
    +public interface DictionaryServer {
     
    -  /**
    -   * start dictionary server
    -   *
    -   * @param port
    -   */
    -  private void startServer(int port) {
    -    dictionaryServerHandler = new DictionaryServerHandler();
    -    String workerThreads = CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.DICTIONARY_WORKER_THREADS,
    -            CarbonCommonConstants.DICTIONARY_WORKER_THREADS_DEFAULT);
    -    boss = new NioEventLoopGroup(1);
    -    worker = new NioEventLoopGroup(Integer.parseInt(workerThreads));
    -    // Configure the server.
    -    bindToPort(port);
    -  }
    +  public void startServer(SparkConf conf, String host, int port);
     
    -  /**
    -   * Binds dictionary server to an available port.
    -   *
    -   * @param port
    -   */
    -  private void bindToPort(int port) {
    -    long start = System.currentTimeMillis();
    -    // Configure the server.
    -    int i = 0;
    -    while (i < 10) {
    -      int newPort = port + i;
    -      try {
    -        ServerBootstrap bootstrap = new ServerBootstrap();
    -        bootstrap.group(boss, worker);
    -        bootstrap.channel(NioServerSocketChannel.class);
    -        bootstrap.childHandler(new ChannelInitializer<SocketChannel>() {
    -          @Override public void initChannel(SocketChannel ch) throws Exception {
    -            ChannelPipeline pipeline = ch.pipeline();
    -            pipeline
    -                .addLast("LengthDecoder",
    -                    new LengthFieldBasedFrameDecoder(1048576, 0,
    -                        2, 0, 2));
    -            pipeline.addLast("DictionaryServerHandler", dictionaryServerHandler);
    -          }
    -        });
    -        bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true);
    -        bootstrap.bind(newPort).sync();
    -        LOGGER.audit("Dictionary Server started, Time spent " + (System.currentTimeMillis() - start)
    -            + " Listening on port " + newPort);
    -        this.port = newPort;
    -        break;
    -      } catch (Exception e) {
    -        LOGGER.error(e, "Dictionary Server Failed to bind to port:");
    -        if (i == 9) {
    -          throw new RuntimeException("Dictionary Server Could not bind to any port");
    -        }
    -      }
    -      i++;
    -    }
    -  }
    +  public void bindToPort(SparkConf orgConf, String host, int port);
     
    -  /**
    -   *
    -   * @return Port on which the DictionaryServer has started.
    -   */
    -  public int getPort() {
    -    return port;
    -  }
    +  public void shutdown()throws Exception;
     
    -  /**
    -   * shutdown dictionary server
    -   *
    -   * @throws Exception
    -   */
    -  public void shutdown() throws Exception {
    -    LOGGER.info("Shutting down dictionary server");
    -    worker.shutdownGracefully();
    -    boss.shutdownGracefully();
    -  }
    +  public String findLocalIpAddress();
     
    +  public String getHost();
     
    +  public int getPort();
     
    -  /**
    -   * Write dictionary to the store.
    -   * @throws Exception
    -   */
    -  public void writeDictionary() throws Exception {
    -    DictionaryMessage key = new DictionaryMessage();
    -    key.setType(DictionaryMessageType.WRITE_DICTIONARY);
    -    dictionaryServerHandler.processMessage(key);
    -  }
    +  public String getSecretKey();
    --- End diff --
    
    Done


---
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] carbondata pull request #1152: [CARBONDATA-1288] Secure Dictionary Server Im...

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

    https://github.com/apache/carbondata/pull/1152#discussion_r126871123
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServerHandler.java ---
    @@ -76,8 +75,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
        * @param ctx
        * @param cause
        */
    -  @Override
    -  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
    +  @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
    --- End diff --
    
    revert this change


---
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] carbondata issue #1152: [CARBONDATA-1288] Secure Dictionary Server Implement...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1152
  
    Can one of the admins verify this patch?


---
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.
---