[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-29 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/incubator-carbondata/pull/310


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r94132843
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryKey.java
 ---
@@ -0,0 +1,92 @@
+/*
+ * 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.generator.key;
+
+import java.io.Serializable;
+
+/**
+ * Dictionary key to generate dictionary
+ */
+public class DictionaryKey implements Serializable {
+
+  /**
+   * tableUniqueName
+   */
+  private String tableUniqueName;
+
+  /**
+   * columnName
+   */
+  private String columnName;
+
+  /**
+   * message data
+   */
+  private Object data;
+
+  /**
+   * message type
+   */
+  private String type;
--- End diff --

please use a Enum to represent message type


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r94132811
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryKey.java
 ---
@@ -0,0 +1,92 @@
+/*
+ * 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.generator.key;
+
+import java.io.Serializable;
+
+/**
+ * Dictionary key to generate dictionary
+ */
+public class DictionaryKey implements Serializable {
+
+  /**
+   * tableUniqueName
+   */
+  private String tableUniqueName;
--- End diff --

please provide meaningful comment, is this string include database name or 
just table name?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-28 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r94093632
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClient.java
 ---
@@ -0,0 +1,87 @@
+/*
+ * 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.client;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
+
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.channel.*;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import org.jboss.netty.handler.codec.serialization.ClassResolvers;
+import org.jboss.netty.handler.codec.serialization.ObjectDecoder;
+import org.jboss.netty.handler.codec.serialization.ObjectEncoder;
+
+
+/**
+ * Dictionary client to connect to Dictionary server and generate 
dictionary values
+ */
+public class DictionaryClient {
+
+  private DictionaryClientHandler dictionaryClientHandler = new 
DictionaryClientHandler();
+
+  private ClientBootstrap clientBootstrap;
+
+  /**
+   * start dictionary client
+   *
+   * @param address
+   * @param port
+   */
+  public void startClient(String address, int port) {
+clientBootstrap = new ClientBootstrap();
+ExecutorService boss = Executors.newCachedThreadPool();
+ExecutorService worker = Executors.newCachedThreadPool();
+clientBootstrap.setFactory(new NioClientSocketChannelFactory(boss, 
worker));
+clientBootstrap.setPipelineFactory(new ChannelPipelineFactory() {
+  @Override
+  public ChannelPipeline getPipeline() throws Exception {
+ChannelPipeline pipeline = Channels.pipeline();
+pipeline.addLast("ObjectEncoder", new ObjectEncoder());
+pipeline.addLast("ObjectDecoder", new 
ObjectDecoder(ClassResolvers.cacheDisabled(
+getClass().getClassLoader(;
+pipeline.addLast("DictionaryClientHandler", 
dictionaryClientHandler);
+return pipeline;
+  }
+});
+clientBootstrap.connect(new InetSocketAddress(address, port));
+System.out.println("Client Start!");
--- End diff --

Please use LOGGER


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-28 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r94093526
  
--- Diff: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
 ---
@@ -402,16 +430,57 @@ case class LoadTable(
 carbonLoadModel.setColDictFilePath(columnDict)
 carbonLoadModel.setDirectLoad(true)
 GlobalDictionaryUtil.updateTableMetadataFunc = updateTableMetadata
-GlobalDictionaryUtil
-  .generateGlobalDictionary(sqlContext, carbonLoadModel, 
relation.tableMeta.storePath,
-dataFrame)
+
+if (carbonLoadModel.getUseOnePass) {
+  val colDictFilePath = carbonLoadModel.getColDictFilePath
+  if (colDictFilePath != null) {
+val storePath = relation.tableMeta.storePath
+val carbonTable = 
carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+val carbonTableIdentifier = 
carbonTable.getAbsoluteTableIdentifier
+  .getCarbonTableIdentifier
+val carbonTablePath = CarbonStorePath
+  .getCarbonTablePath(storePath, carbonTableIdentifier)
+val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
+val dimensions = carbonTable.getDimensionByTableName(
+  carbonTable.getFactTableName).asScala.toArray
+carbonLoadModel.initPredefDictMap()
+// generate predefined dictionary
+GlobalDictionaryUtil
+  .generatePredefinedColDictionary(colDictFilePath, 
carbonTableIdentifier,
+dimensions, carbonLoadModel, sqlContext, storePath, 
dictFolderPath)
+  }
+  // dictionaryServerClient dictionary generator
+  val dictionaryServerPort = CarbonProperties.getInstance()
+.getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,
+  CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT)
+  
carbonLoadModel.setDictionaryServerPort(Integer.parseInt(dictionaryServerPort))
+  val sparkDriverHost = 
sqlContext.sparkContext.getConf.get("spark.driver.host")
+  carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
+  // start dictionary server when use one pass load.
+  executorService = Executors.newFixedThreadPool(1)
+  result = executorService.submit(new Callable[DictionaryServer]() 
{
+@throws[Exception]
--- End diff --

why this is added?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-18 Thread foryou2030
Github user foryou2030 commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r92961704
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
 ---
@@ -0,0 +1,110 @@
+/*
+ * 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.generator;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.*;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.carbon.metadata.CarbonMetadata;
+import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable;
+import 
org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.devapi.BiDictionary;
+import org.apache.carbondata.core.devapi.DictionaryGenerationException;
+import org.apache.carbondata.core.devapi.DictionaryGenerator;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
+import org.apache.carbondata.core.util.CarbonProperties;
+
+/**
+ * Dictionary generation for table.
+ */
+public class TableDictionaryGenerator
+implements DictionaryGenerator, 
DictionaryWriter {
+
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(TableDictionaryGenerator.class.getName());
+
+  private Map> columnMap = 
new ConcurrentHashMap<>();
--- End diff --

fixed


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-18 Thread foryou2030
Github user foryou2030 commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r92959768
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/CarbonTable.java
 ---
@@ -504,4 +521,19 @@ public void setBlockSizeInMB(int blockSize) {
 this.blockSize = blockSize;
   }
 
+  /**
+   * to get the all dimension of a table
+   *
+   * @param tableName
+   * @return all dimension of a table
+   */
+  public CarbonDimension getAllDimensionByName(String tableName, String 
columnName) {
--- End diff --

it is used to get normal dimension and the primitive dimenson of the 
complex. 
I will rename it to a proper name


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-15 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r92760541
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
 ---
@@ -73,10 +79,39 @@ public void initialize() {
 
 long lruCacheStartTime = System.currentTimeMillis();
 
+// for one pass load, start the dictionary client
+if (configuration.getUseOnePass()) {
+  executorService = Executors.newFixedThreadPool(1);
+  Future result = executorService.submit(new 
Callable() {
+@Override
+public DictionaryClient call() throws Exception {
+  Thread.currentThread().setName("Dictionary client");
+  DictionaryClient dictionaryClient = new DictionaryClient();
+  
dictionaryClient.startClient(configuration.getDictionaryServerHost(),
+  configuration.getDictionaryServerPort());
+  return dictionaryClient;
+}
+  });
+
+  try {
+Thread.sleep(1000);
--- End diff --

Why sleep 1 second? please add comment


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-15 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r92760381
  
--- Diff: 
integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseOnePass.scala
 ---
@@ -0,0 +1,75 @@
+/*
+ * 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.testsuite.dataload
+
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+/**
+  * Test Class for data loading use one pass
+  *
+  */
+class TestLoadDataUseOnePass extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+sql("DROP TABLE IF EXISTS table_two_pass")
+sql("DROP TABLE IF EXISTS table_one_pass")
+
+sql(
+  """
+|CREATE TABLE table_two_pass (ID int, date Timestamp, country 
String,
+|name String, phonetype String, serialname String, salary int)
+|STORED BY 'org.apache.carbondata.format'
+  """.stripMargin)
+
+sql(
+  """
+|LOAD DATA local inpath './src/test/resources/dataDiff.csv' INTO 
TABLE table_two_pass
+|OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 
'USE_ONEPASS'='false')
--- End diff --

change option name from USE_ONEPASS to SINGLE_PASS


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-15 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r92760320
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java
 ---
@@ -0,0 +1,75 @@
+/*
+ * 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 
org.apache.carbondata.core.dictionary.generator.DictionaryGeneratorForServer;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
+
+import com.alibaba.fastjson.JSON;
+
+import org.jboss.netty.channel.*;
+
+
+/**
+ * Handler for Dictionary server.
+ */
+public class DictionaryServerHandler extends SimpleChannelHandler {
+
+  private DictionaryGeneratorForServer generatorForServer = new 
DictionaryGeneratorForServer();
+
+  public void channelConnected(ChannelHandlerContext ctx, 
ChannelStateEvent e) throws Exception {
--- End diff --

Add description to all functions in this 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-15 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r92760025
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
 ---
@@ -0,0 +1,110 @@
+/*
+ * 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.generator;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.*;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.carbon.metadata.CarbonMetadata;
+import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable;
+import 
org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.devapi.BiDictionary;
+import org.apache.carbondata.core.devapi.DictionaryGenerationException;
+import org.apache.carbondata.core.devapi.DictionaryGenerator;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
+import org.apache.carbondata.core.util.CarbonProperties;
+
+/**
+ * Dictionary generation for table.
+ */
+public class TableDictionaryGenerator
+implements DictionaryGenerator, 
DictionaryWriter {
+
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(TableDictionaryGenerator.class.getName());
+
+  private Map> columnMap = 
new ConcurrentHashMap<>();
--- End diff --

What is the key and value of this map, please add description.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-15 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r92760062
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryKey.java
 ---
@@ -0,0 +1,77 @@
+/*
+ * 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.generator.key;
+
+
+/**
+ * Dictionary key to generate dictionary
+ */
+public class DictionaryKey {
+
+  private String tableUniqueName;
+
+  private String columnName;
+
+  private Object data;
+
+  private String type;
+
+  private String threadNo;
+
+
--- End diff --

remove empty 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-15 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r92759176
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClientHandler.java
 ---
@@ -0,0 +1,103 @@
+/*
+ * 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.client;
+
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
+
+import com.alibaba.fastjson.JSON;
+
+import org.jboss.netty.channel.*;
+
+/**
+ * Client handler to get data.
+ */
+public class DictionaryClientHandler extends SimpleChannelHandler {
+
+  final Map dictKeyQueueMap = new 
ConcurrentHashMap<>();
+
+  private ChannelHandlerContext ctx;
+
+  private Object lock = new Object();
+
+  @Override
+  public void channelConnected(ChannelHandlerContext ctx, 
ChannelStateEvent e) throws Exception {
+this.ctx = ctx;
+System.out.println("Connected " + ctx.getHandler());
+super.channelConnected(ctx, e);
+  }
+
+  @Override
+  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) 
throws Exception {
+String backkeyString = (String) e.getMessage();
+DictionaryKey key = JSON.parseObject(backkeyString, 
DictionaryKey.class);
+BlockingQueue dictKeyQueue = 
dictKeyQueueMap.get(key.getThreadNo());
+dictKeyQueue.offer(key);
+super.messageReceived(ctx, e);
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) 
throws Exception {
+System.out.println("exceptionCaught");
+ctx.getChannel().close();
+  }
+
+  /**
+   * client send request to server
+   *
+   * @param key
+   * @return
+   */
+  public DictionaryKey getDictionary(DictionaryKey key) {
+DictionaryKey dictionaryKey;
+BlockingQueue dictKeyQueue = null;
+try {
+  synchronized (lock) {
+dictKeyQueue = dictKeyQueueMap.get(key.getThreadNo());
+if (dictKeyQueue == null) {
+  dictKeyQueue = new LinkedBlockingQueue();
+  dictKeyQueueMap.put(key.getThreadNo(), dictKeyQueue);
+}
+  }
+  String keyString = JSON.toJSONString(key);
+  ctx.getChannel().write(keyString);
+} catch (Exception e) {
+  e.printStackTrace();
+}
+boolean interrupted = false;
+try {
+  for (; ; ) {
+try {
+  dictionaryKey = dictKeyQueue.take();
--- End diff --

Is there timeout option here, it is not good to do dead loop here


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-15 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r92759097
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClientHandler.java
 ---
@@ -0,0 +1,103 @@
+/*
+ * 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.client;
+
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
+
+import com.alibaba.fastjson.JSON;
+
+import org.jboss.netty.channel.*;
+
+/**
+ * Client handler to get data.
+ */
+public class DictionaryClientHandler extends SimpleChannelHandler {
+
+  final Map dictKeyQueueMap = new 
ConcurrentHashMap<>();
+
+  private ChannelHandlerContext ctx;
+
+  private Object lock = new Object();
+
+  @Override
+  public void channelConnected(ChannelHandlerContext ctx, 
ChannelStateEvent e) throws Exception {
+this.ctx = ctx;
+System.out.println("Connected " + ctx.getHandler());
+super.channelConnected(ctx, e);
+  }
+
+  @Override
+  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) 
throws Exception {
+String backkeyString = (String) e.getMessage();
+DictionaryKey key = JSON.parseObject(backkeyString, 
DictionaryKey.class);
+BlockingQueue dictKeyQueue = 
dictKeyQueueMap.get(key.getThreadNo());
+dictKeyQueue.offer(key);
+super.messageReceived(ctx, e);
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) 
throws Exception {
+System.out.println("exceptionCaught");
+ctx.getChannel().close();
+  }
+
+  /**
+   * client send request to server
+   *
+   * @param key
+   * @return
+   */
+  public DictionaryKey getDictionary(DictionaryKey key) {
+DictionaryKey dictionaryKey;
+BlockingQueue dictKeyQueue = null;
+try {
+  synchronized (lock) {
+dictKeyQueue = dictKeyQueueMap.get(key.getThreadNo());
+if (dictKeyQueue == null) {
+  dictKeyQueue = new LinkedBlockingQueue();
+  dictKeyQueueMap.put(key.getThreadNo(), dictKeyQueue);
+}
+  }
+  String keyString = JSON.toJSONString(key);
+  ctx.getChannel().write(keyString);
+} catch (Exception e) {
+  e.printStackTrace();
--- End diff --

should use LOGGER


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-carbondata pull request #310: [CARBONDATA-401] One Pass Load

2016-12-15 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/incubator-carbondata/pull/310#discussion_r92758681
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/CarbonTable.java
 ---
@@ -504,4 +521,19 @@ public void setBlockSizeInMB(int blockSize) {
 this.blockSize = blockSize;
   }
 
+  /**
+   * to get the all dimension of a table
+   *
+   * @param tableName
+   * @return all dimension of a table
+   */
+  public CarbonDimension getAllDimensionByName(String tableName, String 
columnName) {
--- End diff --

This function is not getting all dimensions, it is used for finding the 
specified column, right? I think there is already a function called 
`getDimensionByName` to provide this functionality


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---