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


[GitHub] incubator-carbondata pull request #412: [CARBONDATA-519]Added vector reader ...

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

https://github.com/apache/incubator-carbondata/pull/412#discussion_r92744134
  
--- Diff: 
core/src/main/java/org/apache/carbondata/scan/executor/QueryExecutorFactory.java
 ---
@@ -18,15 +18,69 @@
  */
 package org.apache.carbondata.scan.executor;
 
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.apache.carbondata.scan.executor.impl.DetailQueryExecutor;
+import org.apache.carbondata.scan.executor.impl.VectorDetailQueryExecutor;
+import org.apache.carbondata.scan.model.QueryDimension;
+import org.apache.carbondata.scan.model.QueryMeasure;
+import org.apache.carbondata.scan.model.QueryModel;
+import org.apache.carbondata.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.scan.result.vector.CarbonColumnarBatch;
+import 
org.apache.carbondata.scan.result.vector.impl.CarbonColumnVectorImpl;
 
 /**
  * Factory class to get the query executor from RDD
  * This will return the executor based on query type
  */
 public class QueryExecutorFactory {
 
-  public static QueryExecutor getQueryExecutor() {
-return new DetailQueryExecutor();
+  public static QueryExecutor getQueryExecutor(QueryModel queryModel) {
+if (queryModel.isVectorReader()) {
+  return new VectorDetailQueryExecutor();
+} else {
+  return new DetailQueryExecutor();
+}
+  }
+
+  public static CarbonColumnarBatch createColuminarBatch(QueryModel 
queryModel) {
+int batchSize = 1;
+List queryDimension = queryModel.getQueryDimension();
+List queryMeasures = queryModel.getQueryMeasures();
+CarbonColumnVector[] vectors =
+new CarbonColumnVector[queryDimension.size() + 
queryMeasures.size()];
+for (int i = 0; i < queryDimension.size(); i++) {
+  QueryDimension dim = queryDimension.get(i);
+  if (dim.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+vectors[dim.getQueryOrder()] = new 
CarbonColumnVectorImpl(batchSize, DataType.LONG);
+  } else if (!dim.getDimension().hasEncoding(Encoding.DICTIONARY)) {
+vectors[dim.getQueryOrder()] =
+new CarbonColumnVectorImpl(batchSize, 
dim.getDimension().getDataType());
+  } else if (dim.getDimension().isComplex()) {
+vectors[dim.getQueryOrder()] = new 
CarbonColumnVectorImpl(batchSize, DataType.STRUCT);
+  } else {
+vectors[dim.getQueryOrder()] = new 
CarbonColumnVectorImpl(batchSize, DataType.INT);
+  }
+}
+
+for (int i = 0; i < queryMeasures.size(); i++) {
+  QueryMeasure msr = queryMeasures.get(i);
+  switch (msr.getMeasure().getDataType()) {
+case SHORT:
+case INT:
+case LONG:
+  vectors[msr.getQueryOrder()] =
+  new CarbonColumnVectorImpl(batchSize, 
msr.getMeasure().getDataType());
+  break;
+case DECIMAL:
+  vectors[msr.getQueryOrder()] = new 
CarbonColumnVectorImpl(batchSize, DataType.DECIMAL);
+  break;
+default:
+  vectors[msr.getQueryOrder()] = new 
CarbonColumnVectorImpl(batchSize, DataType.DOUBLE);
+  }
--- End diff --

No we can't use as we support few datatypes while storing reading measure 
data.
Anyway this method is not used now, I am removing it


---
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 #345: [CARBONDATA-443]Nosort dataloading

2016-12-15 Thread QiangCai
Github user QiangCai closed the pull request at:

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


---
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 issue #345: [CARBONDATA-443]Nosort dataloading

2016-12-15 Thread QiangCai
Github user QiangCai commented on the issue:

https://github.com/apache/incubator-carbondata/pull/345
  
Close this PR. In the future, I will raise another PR to support mixed data 
format table.


---
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 #412: [CARBONDATA-519]Added vector reader ...

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

https://github.com/apache/incubator-carbondata/pull/412#discussion_r92743691
  
--- Diff: 
core/src/main/java/org/apache/carbondata/scan/executor/QueryExecutorFactory.java
 ---
@@ -18,15 +18,69 @@
  */
 package org.apache.carbondata.scan.executor;
 
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.apache.carbondata.scan.executor.impl.DetailQueryExecutor;
+import org.apache.carbondata.scan.executor.impl.VectorDetailQueryExecutor;
+import org.apache.carbondata.scan.model.QueryDimension;
+import org.apache.carbondata.scan.model.QueryMeasure;
+import org.apache.carbondata.scan.model.QueryModel;
+import org.apache.carbondata.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.scan.result.vector.CarbonColumnarBatch;
+import 
org.apache.carbondata.scan.result.vector.impl.CarbonColumnVectorImpl;
 
 /**
  * Factory class to get the query executor from RDD
  * This will return the executor based on query type
  */
 public class QueryExecutorFactory {
 
-  public static QueryExecutor getQueryExecutor() {
-return new DetailQueryExecutor();
+  public static QueryExecutor getQueryExecutor(QueryModel queryModel) {
+if (queryModel.isVectorReader()) {
+  return new VectorDetailQueryExecutor();
+} else {
+  return new DetailQueryExecutor();
+}
+  }
+
+  public static CarbonColumnarBatch createColuminarBatch(QueryModel 
queryModel) {
--- End diff --

This method is not used now, I am removing it


---
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 #412: [CARBONDATA-519]Added vector reader ...

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

https://github.com/apache/incubator-carbondata/pull/412#discussion_r92743655
  
--- Diff: 
core/src/main/java/org/apache/carbondata/scan/executor/QueryExecutorFactory.java
 ---
@@ -18,15 +18,69 @@
  */
 package org.apache.carbondata.scan.executor;
 
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.apache.carbondata.scan.executor.impl.DetailQueryExecutor;
+import org.apache.carbondata.scan.executor.impl.VectorDetailQueryExecutor;
+import org.apache.carbondata.scan.model.QueryDimension;
+import org.apache.carbondata.scan.model.QueryMeasure;
+import org.apache.carbondata.scan.model.QueryModel;
+import org.apache.carbondata.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.scan.result.vector.CarbonColumnarBatch;
+import 
org.apache.carbondata.scan.result.vector.impl.CarbonColumnVectorImpl;
 
 /**
  * Factory class to get the query executor from RDD
  * This will return the executor based on query type
  */
 public class QueryExecutorFactory {
 
-  public static QueryExecutor getQueryExecutor() {
-return new DetailQueryExecutor();
+  public static QueryExecutor getQueryExecutor(QueryModel queryModel) {
+if (queryModel.isVectorReader()) {
+  return new VectorDetailQueryExecutor();
+} else {
+  return new DetailQueryExecutor();
+}
+  }
+
+  public static CarbonColumnarBatch createColuminarBatch(QueryModel 
queryModel) {
+int batchSize = 1;
--- End diff --

This method is not used now, I am removing it


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


[jira] [Resolved] (CARBONDATA-516) [SPARK2]update union class in CarbonLateDecoderRule for Spark 2.x integration

2016-12-15 Thread Jacky Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jacky Li resolved CARBONDATA-516.
-
   Resolution: Fixed
Fix Version/s: 1.0.0-incubating

> [SPARK2]update union class in CarbonLateDecoderRule for Spark 2.x integration
> -
>
> Key: CARBONDATA-516
> URL: https://issues.apache.org/jira/browse/CARBONDATA-516
> Project: CarbonData
>  Issue Type: New Feature
>Reporter: QiangCai
>Assignee: QiangCai
> Fix For: 1.0.0-incubating
>
>  Time Spent: 3h
>  Remaining Estimate: 0h
>
> In spark2, Union class is no longer sub-class of BinaryNode. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] incubator-carbondata pull request #412: [CARBONDATA-519]Added vector reader ...

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

https://github.com/apache/incubator-carbondata/pull/412#discussion_r92742973
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
 ---
@@ -87,19 +90,17 @@ private[sql] class CarbonLateDecodeStrategy extends 
SparkStrategy {
   private[this] def toCatalystRDD(
   relation: LogicalRelation,
   output: Seq[Attribute],
-  rdd: RDD[Row],
+  rdd: RDD[InternalRow],
   needDecode: ArrayBuffer[AttributeReference]):
   RDD[InternalRow] = {
-val newRdd = if (needDecode.size > 0) {
+if (needDecode.size > 0) {
+  rdd.asInstanceOf[CarbonScanRDD].setVectorReaderSupport(false)
   getDecoderRDD(relation, needDecode, rdd, output)
--- End diff --

if vector reader is true and `needDecode.size > 0` then it uses dictionary 
decoder rdd in its parent.But decoder rdd is not capable of handling columnar 
batches. 
I will raise another PR to move the decoder RDD logic to carbon layer. 


---
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 #413: [CARBONDATA-516][SPARK2]fix union is...

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

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


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


[jira] [Updated] (CARBONDATA-516) [SPARK2]update union class in CarbonLateDecoderRule for Spark 2.x integration

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-516:
-
Summary: [SPARK2]update union class in CarbonLateDecoderRule for Spark 2.x 
integration  (was: [SPARK2]update union issue in CarbonLateDecoderRule for 
Spark 2.x integration)

> [SPARK2]update union class in CarbonLateDecoderRule for Spark 2.x integration
> -
>
> Key: CARBONDATA-516
> URL: https://issues.apache.org/jira/browse/CARBONDATA-516
> Project: CarbonData
>  Issue Type: New Feature
>Reporter: QiangCai
>Assignee: QiangCai
>  Time Spent: 2h 50m
>  Remaining Estimate: 0h
>
> In spark2, Union class is no longer sub-class of BinaryNode. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-464) Frequent GC incurs when Carbon's blocklet size is enlarged from the default

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-464:
-
Description: 
other columnar file format fetch 1 million(a row group) at a time, its data is 
divided into column chunks in columnar format, and each column trunk consists 
of many pages, the page(default size 1 MB) can be independently uncompressed 
and processed.
In case of current carbon,  since we use larger blocklet, it requires larger 
processing memory because it decompresses all projected column chunks within a 
blocklet all at once, which consumes big amount of memory in total. Maybe we 
should consider to come up with an alternative approach to balance I/O and 
processing, in order to reduce GC pressure.

  was:
parquet might fetch from i/o 1 million(a row group) at one time, its data is 
divided into column chunks in columnar format, and each column trunk consists 
of many pages, the page(default size 1 MB) can be independently uncompressed 
and processed.
In case of current carbon since we use larger blocklet, it requires larger 
processing memory as well, as it decompresses all projected column chunks 
within a blocklet, which consumes big amount of memory. Maybe we should 
consider to come up with similar approach to balance I/O and processing, but 
such a change requires carbon format level changes.

Summary: Frequent GC incurs when Carbon's blocklet size is enlarged 
from the default  (was: Big GC occurs frequently when Carbon's blocklet size is 
enlarged from the default)

> Frequent GC incurs when Carbon's blocklet size is enlarged from the default
> ---
>
> Key: CARBONDATA-464
> URL: https://issues.apache.org/jira/browse/CARBONDATA-464
> Project: CarbonData
>  Issue Type: Sub-task
>Reporter: suo tong
>
> other columnar file format fetch 1 million(a row group) at a time, its data 
> is divided into column chunks in columnar format, and each column trunk 
> consists of many pages, the page(default size 1 MB) can be independently 
> uncompressed and processed.
> In case of current carbon,  since we use larger blocklet, it requires larger 
> processing memory because it decompresses all projected column chunks within 
> a blocklet all at once, which consumes big amount of memory in total. Maybe 
> we should consider to come up with an alternative approach to balance I/O and 
> processing, in order to reduce GC pressure.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-436) Make blocklet size configuration respect to the actual size (in terms of byte) of the blocklet

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-436?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-436:
-
Description: Currently, the blocklet size is based on the row counts within 
the blocklet. The default value(12) is small for hdfs io. If we increase 
the value, which may cause too many Young-GC when we scan many columns, 
instead, we can extend the configuration with respect to the actual size of the 
blocklet.  (was: Currently, the blocklet size is the row counts in the 
blocklet. The default value(12) is small for hdfs io. If we increase the 
value, which may cause too many Young-GC when we scan many columns. Like 
parquet, its row group size can be configed, and using hdfs block size as its 
default value.)
Summary: Make blocklet size configuration respect to the actual size 
(in terms of byte) of the blocklet  (was: change blocklet size related to the 
raw size of data )

> Make blocklet size configuration respect to the actual size (in terms of 
> byte) of the blocklet
> --
>
> Key: CARBONDATA-436
> URL: https://issues.apache.org/jira/browse/CARBONDATA-436
> Project: CarbonData
>  Issue Type: Sub-task
>Reporter: suo tong
>Assignee: QiangCai
>
> Currently, the blocklet size is based on the row counts within the blocklet. 
> The default value(12) is small for hdfs io. If we increase the value, 
> which may cause too many Young-GC when we scan many columns, instead, we can 
> extend the configuration with respect to the actual size of the blocklet.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] incubator-carbondata issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread jackylk
Github user jackylk commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
Thanks for adding test case to spark2 integration, but it still needs 
improvement, I have created a JIRA for it. (CARBONDATA-538)


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


[jira] [Created] (CARBONDATA-538) Add test case to spark2 integration

2016-12-15 Thread Jacky Li (JIRA)
Jacky Li created CARBONDATA-538:
---

 Summary: Add test case to spark2 integration
 Key: CARBONDATA-538
 URL: https://issues.apache.org/jira/browse/CARBONDATA-538
 Project: CarbonData
  Issue Type: Improvement
Reporter: Jacky Li
 Fix For: 1.0.0-incubating


Currently spark2 integration has very few test case, it should be improved



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] incubator-carbondata issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread jackylk
Github user jackylk commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
LGTM


---
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 #439: [CARBONDATA-536]initialize updateTab...

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

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


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


[jira] [Updated] (CARBONDATA-432) Feed Carbon task‘s input size to Spark

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-432?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-432:
-
Description: Currently, the input size of task/stage couldn't be displayed 
properly in the spark web UI  (was: Currently, the input size of task/stage 
does not display in the spark job UI)
Summary: Feed Carbon task‘s input size to Spark  (was: Adaptation  
task‘s input size to spark job UI)

> Feed Carbon task‘s input size to Spark
> --
>
> Key: CARBONDATA-432
> URL: https://issues.apache.org/jira/browse/CARBONDATA-432
> Project: CarbonData
>  Issue Type: Sub-task
>Reporter: suo tong
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Currently, the input size of task/stage couldn't be displayed properly in the 
> spark web UI



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] incubator-carbondata pull request #440: [CARBONDATA-537][SPARK2] Fix bug for...

2016-12-15 Thread jackylk
GitHub user jackylk opened a pull request:

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

[CARBONDATA-537][SPARK2] Fix bug for DICTIONARY_EXCLUDE option in spark2

1. Fix bug for dictionary_exclude option in spark2 integration. In spark2, 
datat type name is changed from "string" to "stringtype", but 
`isStringAndTimestampColDictionaryExclude` is not modified.
2. Fix bug for data loading with no-kettle. In no-kettle loading, should 
not ask user to set kettle home environment variable.
3. clean up scala code style in `GlobalDictionaryUtil`

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

$ git pull https://github.com/jackylk/incubator-carbondata exclude

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

https://github.com/apache/incubator-carbondata/pull/440.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 #440


commit 0a8c098eeaa4cdb9a5b7bc361e0abb1e4bff
Author: jackylk 
Date:   2016-12-15T16:37:55Z

fix dictionary exclude

commit 8434b2bdb91e6e0f6f93702bd6044696d559e812
Author: jackylk 
Date:   2016-12-16T00:46:34Z

add testcase




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


[jira] [Created] (CARBONDATA-537) Bug fix for DICTIONARY_EXCLUDE option in spark2 integration

2016-12-15 Thread Jacky Li (JIRA)
Jacky Li created CARBONDATA-537:
---

 Summary: Bug fix for DICTIONARY_EXCLUDE option in spark2 
integration
 Key: CARBONDATA-537
 URL: https://issues.apache.org/jira/browse/CARBONDATA-537
 Project: CarbonData
  Issue Type: Bug
Reporter: Jacky Li
 Fix For: 1.0.0-incubating


1. Fix bug for dictionary_exclude option in spark2 integration. In spark2, 
datat type name is changed from "string" to "stringtype", but 
`isStringAndTimestampColDictionaryExclude` is not modified.
2. Fix bug for data loading with no-kettle. In no-kettle loading, should not 
ask user to set kettle home environment variable.
3. clean up scala code style in `GlobalDictionaryUtil`



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-442) Query result mismatching with Hive

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-442:
-
Summary: Query result mismatching with Hive  (was: SELECT querry result 
mismatched with hive result)

> Query result mismatching with Hive
> --
>
> Key: CARBONDATA-442
> URL: https://issues.apache.org/jira/browse/CARBONDATA-442
> Project: CarbonData
>  Issue Type: Bug
>Reporter: SWATI RAO
>
> => I created table using following command : 
> create table Carbon_automation_test5 (imei string,deviceInformationId int,MAC 
> string,deviceColor string,device_backColor string,modelId string,marketName 
> string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series 
> string,productionDate string,bomCode string,internalModels string, 
> deliveryTime string, channelsId string,channelsName string , deliveryAreaId 
> string, deliveryCountry string, deliveryProvince string, deliveryCity 
> string,deliveryDistrict string, deliveryStreet string,oxSingleNumber string, 
> ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, 
> ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet 
> string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion 
> string,Active_operaSysVersion string, Active_BacVerNumber string, 
> Active_BacFlashVer string,Active_webUIVersion string, Active_webUITypeCarrVer 
> string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, 
> Active_phonePADPartitionedVersions string,Latest_YEAR int, Latest_MONTH int, 
> Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country 
> string, Latest_province string, Latest_city string,Latest_district string, 
> Latest_street string, Latest_releaseId string,Latest_EMUIVersion string, 
> Latest_operaSysVersion string, Latest_BacVerNumber string,Latest_BacFlashVer 
> string, Latest_webUIVersion string, Latest_webUITypeCarrVer 
> string,Latest_webTypeDataVerNumber string, Latest_operatorsVersion 
> string,Latest_phonePADPartitionedVersions string, Latest_operatorId 
> string,gamePointDescription string, gamePointId int,contractNumber int) 
> stored by 'org.apache.carbondata.format' 
> => Load csv to table : 
> LOAD DATA INPATH 'hdfs://localhost:54310/user/hduser/100_olap.csv' INTO table 
> Carbon_automation_test5 OPTIONS('DELIMITER'= ',' ,'QUOTECHAR'= '"', 
> 'FILEHEADER'= 
> 'imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')
> =>now executed SELECT querry : 
> SELECT Carbon_automation_test5.AMSize AS AMSize, 
> Carbon_automation_test5.ActiveCountry AS ActiveCountry, 
> Carbon_automation_test5.Activecity AS Activecity , 
> SUM(Carbon_automation_test5.gamePointId) AS Sum_gamePointId FROM ( SELECT 
> AMSize,ActiveCountry,gamePointId, Activecity FROM (select * from 
> Carbon_automation_test5) SUB_QRY ) Carbon_automation_test5 INNER JOIN ( 
> SELECT ActiveCountry, Activecity, AMSize FROM (select * from 
> Carbon_automation_test5) SUB_QRY ) Carbon_automation_vmall_test1 ON 
> Carbon_automation_test5.AMSize = Carbon_automation_vmall_test1.AMSize WHERE 
> NOT(Carbon_automation_test5.AMSize <= '3RAM size') GROUP BY 
> Carbon_automation_test5.AMSize, Carbon_automation_test5.ActiveCountry, 
> Carbon_automation_test5.Activecity ORDER BY Carbon_automation_test5.AMSize 
> ASC, Carbon_automation_test5.ActiveCountry ASC, 
> Carbon_automation_test5.Activecity ASC;
> +++-+--+--+
> |   AMSize   | ActiveCountry  | Activecity  | Sum_gamePointId  |
> +++-+--+--+
> | 4RAM size  | Chinese| changsha| 200860   |
> | 4RAM size  | Chinese| guangzhou   | 38016|
> | 4RAM size  | Chinese| shenzhen| 49610|
> | 4RAM size  | 

[jira] [Updated] (CARBONDATA-478) Separate SparkRowReadSupportImpl implementation for integrating with Spark1.x vs. Spark 2.x

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-478?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-478:
-
Issue Type: New Feature  (was: Bug)
   Summary: Separate SparkRowReadSupportImpl implementation for integrating 
with Spark1.x vs. Spark 2.x  (was: Spark2 module should have different 
SparkRowReadSupportImpl with spark1)

> Separate SparkRowReadSupportImpl implementation for integrating with Spark1.x 
> vs. Spark 2.x
> ---
>
> Key: CARBONDATA-478
> URL: https://issues.apache.org/jira/browse/CARBONDATA-478
> Project: CarbonData
>  Issue Type: New Feature
>  Components: data-query
>Affects Versions: 0.2.0-incubating
>Reporter: QiangCai
>Assignee: QiangCai
> Fix For: 1.0.0-incubating
>
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-484) Implement LRU cache for B-Tree

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-484?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-484:
-
Description: 
LRU Cache for B-Tree is proposed  to ensure to avoid out memory, when too many 
number of tables exits and all are not frequently used.

Problem:

CarbonData is maintaining two level of B-Tree cache, one at the driver level 
and another at executor level.  Currently CarbonData has the mechanism to 
invalidate the segments and blocks cache for the invalid table segments, but 
there is no eviction policy for the unused cached object. So the instance at 
which complete memory is utilized then the system will not be able to process 
any new requests.

Solution:

In the cache maintained at the driver level and at the executor there must be 
objects in cache currently not in use. Therefore system should have the 
mechanism to below mechanism.

1.   Set the max memory limit till which objects could be hold in the 
memory.

2.   When configured memory limit reached then identify the cached objects 
currently not in use so that the required memory could be freed without 
impacting the existing process.

3.   Eviction should be done only till the required memory is not meet.

For details please refer to attachments.

  was:
*LRU Cache for B-Tree*
Problem:

CarbonData is maintaining two level of B-Tree cache, one at the driver level 
and another at executor level.  Currently CarbonData has the mechanism to 
invalidate the segments and blocks cache for the invalid table segments, but 
there is no eviction policy for the unused cached object. So the instance at 
which complete memory is utilized then the system will not be able to process 
any new requests.

Solution:

In the cache maintained at the driver level and at the executor there must be 
objects in cache currently not in use. Therefore system should have the 
mechanism to below mechanism.

1.   Set the max memory limit till which objects could be hold in the 
memory.

2.   When configured memory limit reached then identify the cached objects 
currently not in use so that the required memory could be freed without 
impacting the existing process.

3.   Eviction should be done only till the required memory is not meet.

For details please refer to attachments.

Summary: Implement LRU cache for B-Tree   (was: Implement LRU cache for 
B-Tree to ensure to avoid out memory, when too many number of tables exits and 
all are not frequently used.)

> Implement LRU cache for B-Tree 
> ---
>
> Key: CARBONDATA-484
> URL: https://issues.apache.org/jira/browse/CARBONDATA-484
> Project: CarbonData
>  Issue Type: New Feature
>Reporter: Mohammad Shahid Khan
>Assignee: Mohammad Shahid Khan
> Attachments: B-Tree LRU Cache.pdf
>
>
> LRU Cache for B-Tree is proposed  to ensure to avoid out memory, when too 
> many number of tables exits and all are not frequently used.
> Problem:
> CarbonData is maintaining two level of B-Tree cache, one at the driver level 
> and another at executor level.  Currently CarbonData has the mechanism to 
> invalidate the segments and blocks cache for the invalid table segments, but 
> there is no eviction policy for the unused cached object. So the instance at 
> which complete memory is utilized then the system will not be able to process 
> any new requests.
> Solution:
> In the cache maintained at the driver level and at the executor there must be 
> objects in cache currently not in use. Therefore system should have the 
> mechanism to below mechanism.
> 1.   Set the max memory limit till which objects could be hold in the 
> memory.
> 2.   When configured memory limit reached then identify the cached 
> objects currently not in use so that the required memory could be freed 
> without impacting the existing process.
> 3.   Eviction should be done only till the required memory is not meet.
> For details please refer to attachments.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-495) Unify compressor interface

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-495?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-495:
-
Description: Use compressor factory to unify the interface and eliminate 
small objects  (was: Use factory for compressor to unify the interface and 
reduce small objects)
 Issue Type: Improvement  (was: Bug)

> Unify compressor interface
> --
>
> Key: CARBONDATA-495
> URL: https://issues.apache.org/jira/browse/CARBONDATA-495
> Project: CarbonData
>  Issue Type: Improvement
>Affects Versions: 0.2.0-incubating
>Reporter: Jacky Li
>Assignee: Jacky Li
> Fix For: 1.0.0-incubating
>
>  Time Spent: 40m
>  Remaining Estimate: 0h
>
> Use compressor factory to unify the interface and eliminate small objects



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-516) [SPARK2]update union issue in CarbonLateDecoderRule for Spark 2.x integration

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-516:
-
Description: In spark2, Union class is no longer sub-class of BinaryNode.   
(was: In spark2, Union class is no longer the sub-class of BinaryNode. We need 
fix union issue in CarbonLateDecoderRule for spark2.)
 Issue Type: New Feature  (was: Bug)
Summary: [SPARK2]update union issue in CarbonLateDecoderRule for Spark 
2.x integration  (was: [SPARK2]fix union issue in CarbonLateDecoderRule)

> [SPARK2]update union issue in CarbonLateDecoderRule for Spark 2.x integration
> -
>
> Key: CARBONDATA-516
> URL: https://issues.apache.org/jira/browse/CARBONDATA-516
> Project: CarbonData
>  Issue Type: New Feature
>Reporter: QiangCai
>Assignee: QiangCai
>  Time Spent: 2.5h
>  Remaining Estimate: 0h
>
> In spark2, Union class is no longer sub-class of BinaryNode. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-519) Enable vector reader in Carbon-Spark 2.0 integration and Carbon layer

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-519?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-519:
-
Description: Spark 2.0 supports vectorized reader and uses whole codegen to 
improve performance, Carbon will enable vectorized reader integrating with 
Spark to take advantage of new features of Spark2.x  (was: Spark 2.0 supports 
batch reader and uses whole codegen to improve performance, so carbon also can 
implement vector reader and leverage the features of Spark2.0)
 Issue Type: New Feature  (was: Improvement)

> Enable vector reader in Carbon-Spark 2.0 integration and Carbon layer
> -
>
> Key: CARBONDATA-519
> URL: https://issues.apache.org/jira/browse/CARBONDATA-519
> Project: CarbonData
>  Issue Type: New Feature
>Reporter: Ravindra Pesala
>Assignee: Ravindra Pesala
>  Time Spent: 2h 50m
>  Remaining Estimate: 0h
>
> Spark 2.0 supports vectorized reader and uses whole codegen to improve 
> performance, Carbon will enable vectorized reader integrating with Spark to 
> take advantage of new features of Spark2.x



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-522) New data loading flowcauses testcase failures like big decimal etc

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-522?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-522:
-
Description: 
Pls check http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/105/.

new data flow cause test regressions.

  was:
Pls check http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/105/.

I suggest that we should test new data loading flow when adding new feature.

Summary: New data loading flowcauses testcase failures like big decimal 
etc  (was: New data loading flow can not pass some testcase like big decimal 
etc)

> New data loading flowcauses testcase failures like big decimal etc
> --
>
> Key: CARBONDATA-522
> URL: https://issues.apache.org/jira/browse/CARBONDATA-522
> Project: CarbonData
>  Issue Type: Bug
>Reporter: Lionx
>Assignee: Ravindra Pesala
>
> Pls check http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/105/.
> new data flow cause test regressions.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-527) Greater than/less-than/Like filters optimization for dictionary encoded columns

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-527?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-527:
-
Issue Type: New Feature  (was: Improvement)
   Summary: Greater than/less-than/Like filters optimization for dictionary 
encoded columns  (was: Greater than/less-than/Like filters optimization for 
dictionary columns)

> Greater than/less-than/Like filters optimization for dictionary encoded 
> columns
> ---
>
> Key: CARBONDATA-527
> URL: https://issues.apache.org/jira/browse/CARBONDATA-527
> Project: CarbonData
>  Issue Type: New Feature
>Reporter: Sujith
>  Time Spent: 40m
>  Remaining Estimate: 0h
>
> Current design 
> In greater than/less-than/Like filters, system first iterates each row 
> present in the dictionary cache for identifying valid filter actual members  
> by applying the filter expression , once evaluation done system will hold the 
> list of identified valid filter actual member values(String), now in next 
> step again  system will look up the dictionary cache in order to identify the 
> dictionary surrogate values of the identified members. this look up is an 
> additional cost to our system even though the look up methodology is an 
> binary search in dictionary cache.
>  
> Proposed design/solution:
> Identify the dictionary surrogate values in filter expression evaluation step 
> itself  when actual dictionary values will be scanned for identifying valid 
> filter members .
> Keep a dictionary counter variable which will be increased  when system 
> iterates through  the dictionary cache in order to retrieve each actual 
> member stored in dictionary cache , after this system will evaluate each row 
> against the filter expression to identify whether its a valid filter member 
> or not, while doing this process itself counter value can be taken as valid 
> selected dictionary value since the actual member values and its  dictionary 
> values will be kept in same order in dictionary cache as the iteration order.
> thus it will eliminate the further dictionary look up step which is required  
> to retrieve the dictionary surrogate value against identified actual valid 
> filter member. this can also increase significantly the filter query 
> performance of such filter queries which require expression evaluation to 
> identify it the filter members by looking up dictionary cache, like greater 
> than/less-than/Like filters .
> Note : this optimization is applicable for dictionary columns.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-531) Eliminate spark dependency in carbon core

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-531?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-531:
-
Description: Clean up the interface and take out Spark dependency on 
Carbon-core module.  (was: Carbon-core module should not depend on spark )
Summary: Eliminate spark dependency in carbon core  (was: Remove spark 
dependency in carbon core)

> Eliminate spark dependency in carbon core
> -
>
> Key: CARBONDATA-531
> URL: https://issues.apache.org/jira/browse/CARBONDATA-531
> Project: CarbonData
>  Issue Type: Improvement
>Affects Versions: 0.2.0-incubating
>Reporter: Jacky Li
> Fix For: 1.0.0-incubating
>
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Clean up the interface and take out Spark dependency on Carbon-core module.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-536) Initialize GlobalDictionaryUtil.updateTableMetadataFunc for Spark 2.x

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-536?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-536:
-
Description: GlobalDictionaryUtil.updateTableMetadataFunc needs to be 
initialized.  (was: For spark2, GlobalDictionaryUtil.updateTableMetadataFunc 
should been initialized)
Summary: Initialize GlobalDictionaryUtil.updateTableMetadataFunc for 
Spark 2.x  (was: For spark2, GlobalDictionaryUtil.updateTableMetadataFunc 
should been initialized)

> Initialize GlobalDictionaryUtil.updateTableMetadataFunc for Spark 2.x
> -
>
> Key: CARBONDATA-536
> URL: https://issues.apache.org/jira/browse/CARBONDATA-536
> Project: CarbonData
>  Issue Type: Bug
>  Components: data-load
>Affects Versions: 1.0.0-incubating
>Reporter: QiangCai
>Assignee: QiangCai
> Fix For: 1.0.0-incubating
>
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> GlobalDictionaryUtil.updateTableMetadataFunc needs to be initialized.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-322) Integration with spark 2.x

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-322:
-
Issue Type: New Feature  (was: Improvement)

> Integration with  spark 2.x 
> 
>
> Key: CARBONDATA-322
> URL: https://issues.apache.org/jira/browse/CARBONDATA-322
> Project: CarbonData
>  Issue Type: New Feature
>  Components: spark-integration
>Affects Versions: 0.2.0-incubating
>Reporter: Fei Wang
>Assignee: Fei Wang
> Fix For: 1.0.0-incubating
>
>
> Since spark 2.0 released. there are many nice features such as more efficient 
> parser, vectorized execution, adaptive execution. 
> It is good to integrate with spark 2.x
> current integration up to Spark v1.6 is tightly coupled with spark, we would 
> like to cleanup the interface with following design points in mind: 
> 1. decoupled with Spark, integration based on Spark's v2 datasource API
> 2. Enable vectorized carbon reader
> 3. Support saving DataFrame to Carbondata file through Carbondata's output 
> format.
> ...



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-322) Integration with spark 2.x

2016-12-15 Thread Jihong MA (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jihong MA updated CARBONDATA-322:
-
Description: 
Since spark 2.0 released. there are many nice features such as more efficient 
parser, vectorized execution, adaptive execution. 
It is good to integrate with spark 2.x

current integration up to Spark v1.6 is tightly coupled with spark, we would 
like to cleanup the interface with following design points in mind: 

1. decoupled with Spark, integration based on Spark's v2 datasource API
2. Enable vectorized carbon reader
3. Support saving DataFrame to Carbondata file through Carbondata's output 
format.
...


  was:
As spark 2.0 released. there are many nice features such as more efficient 
parser, vectorized execution, adaptive execution. 
It is good to integrate with spark 2.x

Another side now in carbondata, spark integration is heavy coupling with spark 
code and the code need clean, we should redesign the spark integration, it 
should satisfy flowing requirement:

1. decoupled with spark, integrate according to spark datasource API(V2)
2. This integration should support vectorized carbon reader
3. Supoort write to carbondata from dadatrame
...


 Issue Type: Improvement  (was: Bug)
Summary: Integration with  spark 2.x   (was: integrate spark 2.x )

> Integration with  spark 2.x 
> 
>
> Key: CARBONDATA-322
> URL: https://issues.apache.org/jira/browse/CARBONDATA-322
> Project: CarbonData
>  Issue Type: Improvement
>  Components: spark-integration
>Affects Versions: 0.2.0-incubating
>Reporter: Fei Wang
>Assignee: Fei Wang
> Fix For: 1.0.0-incubating
>
>
> Since spark 2.0 released. there are many nice features such as more efficient 
> parser, vectorized execution, adaptive execution. 
> It is good to integrate with spark 2.x
> current integration up to Spark v1.6 is tightly coupled with spark, we would 
> like to cleanup the interface with following design points in mind: 
> 1. decoupled with Spark, integration based on Spark's v2 datasource API
> 2. Enable vectorized carbon reader
> 3. Support saving DataFrame to Carbondata file through Carbondata's output 
> format.
> ...



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] incubator-carbondata issue #439: [CARBONDATA-536]initialize updateTableMetad...

2016-12-15 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/incubator-carbondata/pull/439
  
Build Success with Spark 1.5.2, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/203/



---
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 #439: [CARBONDATA-536]initialize updateTab...

2016-12-15 Thread QiangCai
GitHub user QiangCai opened a pull request:

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

[CARBONDATA-536]initialize updateTableMetadata method in LoadTable for 
Spark2

For spark2, GlobalDictionaryUtil.updateTableMetadataFunc should been 
initialized

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

$ git pull https://github.com/QiangCai/incubator-carbondata 
fixBugInLoadTable

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

https://github.com/apache/incubator-carbondata/pull/439.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 #439


commit 1f28ef9864e2d45807bb7c6bb1cbb51f65f423e3
Author: QiangCai 
Date:   2016-12-15T16:26:09Z

fixLoadTableForSpark2




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


[jira] [Created] (CARBONDATA-536) For spark2, GlobalDictionaryUtil.updateTableMetadataFunc should been initialized

2016-12-15 Thread QiangCai (JIRA)
QiangCai created CARBONDATA-536:
---

 Summary: For spark2, GlobalDictionaryUtil.updateTableMetadataFunc 
should been initialized
 Key: CARBONDATA-536
 URL: https://issues.apache.org/jira/browse/CARBONDATA-536
 Project: CarbonData
  Issue Type: Bug
  Components: data-load
Affects Versions: 1.0.0-incubating
Reporter: QiangCai
Assignee: QiangCai
 Fix For: 1.0.0-incubating


For spark2, GlobalDictionaryUtil.updateTableMetadataFunc should been initialized



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] incubator-carbondata issue #433: [WIP] fix testcase failure for -Pno-kettle ...

2016-12-15 Thread littleJava
Github user littleJava commented on the issue:

https://github.com/apache/incubator-carbondata/pull/433
  
another question:
is hive optional ? I just test carbondata in spark2 local driver model.

and what is the role-playing of derby , how can i use derby ?

Thank you !

by the way : the new version of carbondata-format-1.0.0-incubating-SNAPSHOT 
should be uploaded 
😀




---
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 issue #433: [WIP] fix testcase failure for -Pno-kettle ...

2016-12-15 Thread littleJava
Github user littleJava commented on the issue:

https://github.com/apache/incubator-carbondata/pull/433
  
hi, i ran 'org.apache.spark.sql.examples.CarbonExample' with spark2.0.2 
submit , and got the exception:
```
AUDIT 15-12 16:04:24,691 - [spark-host][root][Thread-1]Table created with 
Database name [default] and Table name [carbon_table]
WARN  15-12 16:04:24,691 - Couldn't find corresponding Hive SerDe for data 
source provider org.apache.spark.sql.CarbonSource. Persisting data source 
relation `carbon_table` into Hive metastore in Spark SQL specific format, which 
is NOT compatible with Hive.
Exception in thread "main" java.lang.UnsupportedOperationException: 
loadTable is not implemented
at 
org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.loadTable(InMemoryCatalog.scala:290)
at 
org.apache.spark.sql.catalyst.catalog.SessionCatalog.loadTable(SessionCatalog.scala:297)
at 
org.apache.spark.sql.execution.command.LoadDataCommand.run(tables.scala:335)
at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136)
at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at 
org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:114)
at 
org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:86)
at 
org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:86)
at org.apache.spark.sql.Dataset.(Dataset.scala:186)
at org.apache.spark.sql.Dataset.(Dataset.scala:167)
at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:65)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:582)
at org.apache.spark.sql.examples.CarbonExample$.main(Carbon.scala:105)
at org.apache.spark.sql.examples.CarbonExample.main(Carbon.scala)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:736)
at 
org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:185)
at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:210)
at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:124)
```


---
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 issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
Build Success with Spark 1.5.2, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/201/



---
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 issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
Build Failed  with Spark 1.5.2, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/200/



---
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 issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
Build Success with Spark 1.5.2, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/198/



---
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 issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
Build Failed  with Spark 1.5.2, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/197/



---
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 issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread QiangCai
Github user QiangCai commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
@jackylk
Added test case 
Local test case pass for spark1.5 and spark2


---
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 issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
Build Failed  with Spark 1.5.2, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/195/



---
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 issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
Build Failed  with Spark 1.5.2, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/194/



---
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 issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
Build Failed  with Spark 1.5.2, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/193/



---
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 issue #413: [CARBONDATA-516][SPARK2]fix union issue in ...

2016-12-15 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/incubator-carbondata/pull/413
  
Build Failed  with Spark 1.5.2, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/192/



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


[jira] [Closed] (CARBONDATA-394) Carbon Loading data from files having invalid extensions or no extension

2016-12-15 Thread SWATI RAO (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-394?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

SWATI RAO closed CARBONDATA-394.

Resolution: Not A Problem

> Carbon Loading data from files having invalid extensions or no extension
> 
>
> Key: CARBONDATA-394
> URL: https://issues.apache.org/jira/browse/CARBONDATA-394
> Project: CarbonData
>  Issue Type: Bug
>Reporter: SWATI RAO
>Assignee: xbkaishui
>Priority: Trivial
>
> When I try to run the following queries :
> LOAD DATA inpath 'hdfs://localhost:54310/user/hive/warehouse/file1.csv.csv' 
> INTO table empdata options('DELIMITER'=',', 'FILEHEADER'='id, 
> name','QUOTECHAR'='"');
> LOAD DATA inpath 
> 'hdfs://localhost:54310/user/hive/warehouse/file2.csv.csv.csv.csv' INTO table 
> empdata options('DELIMITER'=',', 'FILEHEADER'='id, name','QUOTECHAR'='"');
>  LOAD DATA inpath 'hdfs://localhost:54310/user/hive/warehouse/file3.txttt' 
> INTO table empdata options('DELIMITER'=',', 'FILEHEADER'='id, 
> name','QUOTECHAR'='"');
> LOAD DATA inpath 'hdfs://localhost:54310/user/hive/warehouse/file4' INTO 
> table empdata options('DELIMITER'=',', 'FILEHEADER'='id, 
> name','QUOTECHAR'='"');
> LOAD DATA inpath 
> 'hdfs://localhost:54310/user/hive/warehouse/file5.txt.bat.csv' INTO table 
> empdata options('DELIMITER'=',', 'FILEHEADER'='id, name','QUOTECHAR'='"');
> We should get Input File Errors, but the data is loaded successfully into the 
> Carbon table.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CARBONDATA-394) Carbon Loading data from files having invalid extensions or no extension

2016-12-15 Thread SWATI RAO (JIRA)

[ 
https://issues.apache.org/jira/browse/CARBONDATA-394?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15750987#comment-15750987
 ] 

SWATI RAO commented on CARBONDATA-394:
--

So we are closing this issue.

> Carbon Loading data from files having invalid extensions or no extension
> 
>
> Key: CARBONDATA-394
> URL: https://issues.apache.org/jira/browse/CARBONDATA-394
> Project: CarbonData
>  Issue Type: Bug
>Reporter: SWATI RAO
>Assignee: xbkaishui
>Priority: Trivial
>
> When I try to run the following queries :
> LOAD DATA inpath 'hdfs://localhost:54310/user/hive/warehouse/file1.csv.csv' 
> INTO table empdata options('DELIMITER'=',', 'FILEHEADER'='id, 
> name','QUOTECHAR'='"');
> LOAD DATA inpath 
> 'hdfs://localhost:54310/user/hive/warehouse/file2.csv.csv.csv.csv' INTO table 
> empdata options('DELIMITER'=',', 'FILEHEADER'='id, name','QUOTECHAR'='"');
>  LOAD DATA inpath 'hdfs://localhost:54310/user/hive/warehouse/file3.txttt' 
> INTO table empdata options('DELIMITER'=',', 'FILEHEADER'='id, 
> name','QUOTECHAR'='"');
> LOAD DATA inpath 'hdfs://localhost:54310/user/hive/warehouse/file4' INTO 
> table empdata options('DELIMITER'=',', 'FILEHEADER'='id, 
> name','QUOTECHAR'='"');
> LOAD DATA inpath 
> 'hdfs://localhost:54310/user/hive/warehouse/file5.txt.bat.csv' INTO table 
> empdata options('DELIMITER'=',', 'FILEHEADER'='id, name','QUOTECHAR'='"');
> We should get Input File Errors, but the data is loaded successfully into the 
> Carbon table.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CARBONDATA-327) Drop Daabase unexpected behaviour.

2016-12-15 Thread anubhav tarar (JIRA)

[ 
https://issues.apache.org/jira/browse/CARBONDATA-327?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15750940#comment-15750940
 ] 

anubhav tarar commented on CARBONDATA-327:
--

i create a database in hive and carbondata then dropped both databases when i 
execute show table in hive it gives an exception that database does not exists 
but in carbon data it does not give any exception

in hive---
i got following output 
hive> CREATE DATABASE SAMPLE;
OK
hive (default)> use SAMPLE;
OK
hive (SAMPLE)> drop database SAMPLE;
OK
hive (SAMPLE)> use SAMPLE;
FAILED: SemanticException [Error 10072]: Database does not exist: SAMPLE
hive (SAMPLE)> show tables;
FAILED: SemanticException [Error 10072]: Database does not exist: SAMPLE

where as in carbondata i got folowing output

0: jdbc:hive2://localhost:1> CREATE DATABASE SAMPLE;
0: jdbc:hive2://localhost:1> USE SAMPLE;
0: jdbc:hive2://localhost:1> DROP DATABASE SAMPLE;
0: jdbc:hive2://localhost:1> USE SAMPLE;
Error: org.apache.spark.sql.execution.QueryExecutionException: FAILED: 
SemanticException [Error 10072]: Database does not exist: SAMPLE (state=,code=0)
0: jdbc:hive2:

0: jdbc:hive2://localhost:1>show tables;
++--+--+
| tableName  | isTemporary  |
++--+--+
++--+--+
No rows selected (0.066 seconds)


> Drop Daabase unexpected behaviour. 
> ---
>
> Key: CARBONDATA-327
> URL: https://issues.apache.org/jira/browse/CARBONDATA-327
> Project: CarbonData
>  Issue Type: Bug
>Reporter: Harmeet Singh
>
> Hey team, I am creating a database as below:
> 0: jdbc:hive2://127.0.0.1:1> create database Test;
> +-+--+
> | result  |
> +-+--+
> +-+--+
> After creating an database i am using that database using below command:
> 0: jdbc:hive2://127.0.0.1:1> use Test;
> +-+--+
> | result  |
> +-+--+
> +-+--+
> After That, i am drop the database as below:
> 0: jdbc:hive2://127.0.0.1:1> drop database test;
> +-+--+
> | result  |
> +-+--+
> +-+--+
> The database drop successfully. I am expecting, after that the carbon data 
> automatically switch to the "default" database. But when i trying to execute 
> command "show tables" the result return nothing as below :
> 0: jdbc:hive2://127.0.0.1:1> show tables;
> ++--+--+
> | tableName  | isTemporary  |
> ++--+--+
> ++--+--+
> No rows selected (0.019 seconds)
> But my default database contains some table as below:
> 0: jdbc:hive2://127.0.0.1:1> use default;
> +-+--+
> | result  |
> +-+--+
> +-+--+
> No rows selected (0.024 seconds)
> 0: jdbc:hive2://127.0.0.1:1> show tables;
> ++--+--+
> | tableName  | isTemporary  |
> ++--+--+
> | one| false|
> | two| false|
> ++--+--+
> 2 rows selected (0.013 seconds)
> If I am following all above step on Hive, Hive gave us an error on show 
> tables after drop the database as below:
> hive> drop database test;
> OK
> Time taken: 0.628 seconds
> hive> show databases;
> OK
> default
> Time taken: 0.022 seconds, Fetched: 1 row(s)
> hive> show tables;
> FAILED: SemanticException [Error 10072]: Database does not exist: test 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (CARBONDATA-535) carbondata should support datatype: Date and Char

2016-12-15 Thread Jacky Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-535?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jacky Li resolved CARBONDATA-535.
-
Resolution: Fixed

> carbondata should support datatype: Date and Char
> -
>
> Key: CARBONDATA-535
> URL: https://issues.apache.org/jira/browse/CARBONDATA-535
> Project: CarbonData
>  Issue Type: Improvement
>  Components: file-format
>Affects Versions: 1.0.0-incubating
>Reporter: QiangCai
>Assignee: QiangCai
> Fix For: 1.0.0-incubating
>
>
> carbondata should support datatype: Date and Char



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] incubator-carbondata pull request #411: [CARBONDATA-535]Support data type: d...

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

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


---
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 issue #310: [CARBONDATA-401] One Pass Load

2016-12-15 Thread foryou2030
Github user foryou2030 commented on the issue:

https://github.com/apache/incubator-carbondata/pull/310
  
@jackylk, pls review it, thanks


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


[jira] [Comment Edited] (CARBONDATA-324) Decimal and Bigint type columns contains Null, after load data

2016-12-15 Thread Himani Arora (JIRA)

[ 
https://issues.apache.org/jira/browse/CARBONDATA-324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15750738#comment-15750738
 ] 

Himani Arora edited comment on CARBONDATA-324 at 12/15/16 8:20 AM:
---

This seems no longer a bug to me.
I did the following steps:
create table test(id INT,name STRING,salary DECIMAL,data BIGINT) stored by 
'carbondata';

LOAD DATA INPATH "hdfs://localhost:54310/user/test1.csv" INTO TABLE test 
options('DELIMITER'=',', 'FILEHEADER'='id, name, salary, data');

test1.csv contains the following fields:
himani,1,90,292092

The output was fine and as expected:
!https://issues.apache.org/jira/secure/attachment/12843371/CARBONDATA-324.png!


was (Author: himani1):
This seems no longer a bug to me.
I did the following steps:
create table test(id INT,name STRING,salary DECIMAL,data BIGINT) stored by 
'carbondata';

LOAD DATA INPATH "hdfs://localhost:54310/user/test1.csv" INTO TABLE test 
options('DELIMITER'=',', 'FILEHEADER'='id, name, salary, data');

test1.csv contains the following fields:
himani,1,90,292092

The output was fine and as expected 

> Decimal and Bigint type columns contains Null, after load data
> --
>
> Key: CARBONDATA-324
> URL: https://issues.apache.org/jira/browse/CARBONDATA-324
> Project: CarbonData
>  Issue Type: Bug
>Reporter: Harmeet Singh
> Attachments: CARBONDATA-324.png
>
>
> Using Thrift server and Beeling client, i am trying to create a table and 
> load the data from CSV. My tables contains BigInt and Decimal Column types, 
> After load the data using Load Data command, The BigInt and Decimal Column 
> contains Null Value. Bellow are the steps:
> Step 1: 
> > create database wednesday;
> > use wednesday;
> > CREATE TABLE one (id int, age iNt, name String, salary decimal, data 
> > bigInt, weight double, dob timeStamp) STORED BY 'carbondata';
> Step 2: 
> Create a csv file which contains column values as below: 
> id, age, name, salary, data, weight, dob
> 1, 54, james, 90, 292092, 34.2, 2016-05-04 22:55:00
> Step 3: 
> Load the data from CSV file as below: 
> > LOAD DATA INPATH 'hdfs://localhost:54310/home/harmeet/sample3.csv' INTO 
> > TABLE one;
> Step 4: 
> Select the data from table one, and BigInt and Decimal column contains Null 
> value. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-324) Decimal and Bigint type columns contains Null, after load data

2016-12-15 Thread Himani Arora (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-324?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Himani Arora updated CARBONDATA-324:

Attachment: CARBONDATA-324.png

> Decimal and Bigint type columns contains Null, after load data
> --
>
> Key: CARBONDATA-324
> URL: https://issues.apache.org/jira/browse/CARBONDATA-324
> Project: CarbonData
>  Issue Type: Bug
>Reporter: Harmeet Singh
> Attachments: CARBONDATA-324.png
>
>
> Using Thrift server and Beeling client, i am trying to create a table and 
> load the data from CSV. My tables contains BigInt and Decimal Column types, 
> After load the data using Load Data command, The BigInt and Decimal Column 
> contains Null Value. Bellow are the steps:
> Step 1: 
> > create database wednesday;
> > use wednesday;
> > CREATE TABLE one (id int, age iNt, name String, salary decimal, data 
> > bigInt, weight double, dob timeStamp) STORED BY 'carbondata';
> Step 2: 
> Create a csv file which contains column values as below: 
> id, age, name, salary, data, weight, dob
> 1, 54, james, 90, 292092, 34.2, 2016-05-04 22:55:00
> Step 3: 
> Load the data from CSV file as below: 
> > LOAD DATA INPATH 'hdfs://localhost:54310/home/harmeet/sample3.csv' INTO 
> > TABLE one;
> Step 4: 
> Select the data from table one, and BigInt and Decimal column contains Null 
> value. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CARBONDATA-324) Decimal and Bigint type columns contains Null, after load data

2016-12-15 Thread Himani Arora (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-324?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Himani Arora updated CARBONDATA-324:

Attachment: (was: Screenshot from 2016-10-19 10-54-06.png)

> Decimal and Bigint type columns contains Null, after load data
> --
>
> Key: CARBONDATA-324
> URL: https://issues.apache.org/jira/browse/CARBONDATA-324
> Project: CarbonData
>  Issue Type: Bug
>Reporter: Harmeet Singh
>
> Using Thrift server and Beeling client, i am trying to create a table and 
> load the data from CSV. My tables contains BigInt and Decimal Column types, 
> After load the data using Load Data command, The BigInt and Decimal Column 
> contains Null Value. Bellow are the steps:
> Step 1: 
> > create database wednesday;
> > use wednesday;
> > CREATE TABLE one (id int, age iNt, name String, salary decimal, data 
> > bigInt, weight double, dob timeStamp) STORED BY 'carbondata';
> Step 2: 
> Create a csv file which contains column values as below: 
> id, age, name, salary, data, weight, dob
> 1, 54, james, 90, 292092, 34.2, 2016-05-04 22:55:00
> Step 3: 
> Load the data from CSV file as below: 
> > LOAD DATA INPATH 'hdfs://localhost:54310/home/harmeet/sample3.csv' INTO 
> > TABLE one;
> Step 4: 
> Select the data from table one, and BigInt and Decimal column contains Null 
> value. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CARBONDATA-324) Decimal and Bigint type columns contains Null, after load data

2016-12-15 Thread Anurag Srivastava (JIRA)

[ 
https://issues.apache.org/jira/browse/CARBONDATA-324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15750746#comment-15750746
 ] 

Anurag Srivastava commented on CARBONDATA-324:
--

!https://issues.apache.org/jira/secure/attachment/12843370/Screenshot%20from%202016-10-19%2010-54-06.png!

> Decimal and Bigint type columns contains Null, after load data
> --
>
> Key: CARBONDATA-324
> URL: https://issues.apache.org/jira/browse/CARBONDATA-324
> Project: CarbonData
>  Issue Type: Bug
>Reporter: Harmeet Singh
> Attachments: Screenshot from 2016-10-19 10-54-06.png
>
>
> Using Thrift server and Beeling client, i am trying to create a table and 
> load the data from CSV. My tables contains BigInt and Decimal Column types, 
> After load the data using Load Data command, The BigInt and Decimal Column 
> contains Null Value. Bellow are the steps:
> Step 1: 
> > create database wednesday;
> > use wednesday;
> > CREATE TABLE one (id int, age iNt, name String, salary decimal, data 
> > bigInt, weight double, dob timeStamp) STORED BY 'carbondata';
> Step 2: 
> Create a csv file which contains column values as below: 
> id, age, name, salary, data, weight, dob
> 1, 54, james, 90, 292092, 34.2, 2016-05-04 22:55:00
> Step 3: 
> Load the data from CSV file as below: 
> > LOAD DATA INPATH 'hdfs://localhost:54310/home/harmeet/sample3.csv' INTO 
> > TABLE one;
> Step 4: 
> Select the data from table one, and BigInt and Decimal column contains Null 
> value. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] incubator-carbondata issue #339: [CARBONDATA-429] Remove unnecessary file na...

2016-12-15 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/incubator-carbondata/pull/339
  
Build Failed  with Spark 1.5.2, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/191/



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