[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3773: [CARBONDATA-3830]Presto array columns read support

2020-08-04 Thread GitBox


ajantha-bhat commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r465477662



##
File path: 
integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/GenerateFiles.scala
##
@@ -0,0 +1,650 @@
+package org.apache.carbondata.presto.integrationtest
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, 
File, InputStream}
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.avro
+import org.apache.avro.file.DataFileWriter
+import org.apache.avro.generic.{GenericDatumReader, GenericDatumWriter, 
GenericRecord}
+import org.apache.avro.io.{DecoderFactory, Encoder}
+import org.junit.Assert
+
+import org.apache.carbondata.core.cache.dictionary.DictionaryByteArrayWrapper
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.block.TableBlockInfo
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk
+import 
org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory
+import 
org.apache.carbondata.core.datastore.chunk.reader.dimension.v3.DimensionChunkReaderV3
+import org.apache.carbondata.core.datastore.compression.CompressorFactory
+import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, 
CarbonFileFilter}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import 
org.apache.carbondata.core.datastore.page.encoding.DefaultEncodingFactory
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion
+import org.apache.carbondata.core.util.{CarbonMetadataUtil, 
DataFileFooterConverterV3}
+import org.apache.carbondata.sdk.file.CarbonWriter
+
+class GenerateFiles {
+
+  def singleLevelArrayFile() = {
+val json1: String =
+  """ {"stringCol": "bob","intCol": 14,"doubleCol": 10.5,"realCol": 12.7,
+|"boolCol": true,"arrayStringCol1":["Street1"],"arrayStringCol2": 
["India", "Egypt"],
+|"arrayIntCol": 
[1,2,3],"arrayBigIntCol":[7,6],"arrayRealCol":[1.111,2.2],
+|"arrayDoubleCol":[1.1,2.2,3.3], "arrayBooleanCol": [true, false, 
true]} """.stripMargin
+val json2: String =
+  """ {"stringCol": "Alex","intCol": 15,"doubleCol": 11.5,"realCol": 13.7,
+|"boolCol": true, "arrayStringCol1": ["Street1", 
"Street2"],"arrayStringCol2": ["Japan",
+|"China", "India"],"arrayIntCol": 
[1,2,3,4],"arrayBigIntCol":[7,6,8000],
+|"arrayRealCol":[1.1,2.2,3.3],"arrayDoubleCol":[1.1,2.2,4.45,3.3],
+|"arrayBooleanCol": [true, true, true]} """.stripMargin
+val json3: String =
+  """ {"stringCol": "Rio","intCol": 16,"doubleCol": 12.5,"realCol": 14.7,
+|"boolCol": true, "arrayStringCol1": ["Street1", "Street2","Street3"],
+|"arrayStringCol2": ["China", "Brazil", "Paris", 
"France"],"arrayIntCol": [1,2,3,4,5],
+
|"arrayBigIntCol":[7,6,8000,91],"arrayRealCol":[1.1,2.2,3.3,4.45],
+|"arrayDoubleCol":[1.1,2.2,4.45,5.5,3.3], "arrayBooleanCol": [true, 
false, true]} """
+.stripMargin
+val json4: String =
+  """ {"stringCol": "bob","intCol": 14,"doubleCol": 10.5,"realCol": 12.7,
+|"boolCol": true, "arrayStringCol1":["Street1"],"arrayStringCol2": 
["India", "Egypt"],
+|"arrayIntCol": 
[1,2,3],"arrayBigIntCol":[7,6],"arrayRealCol":[1.1,2.2],
+|"arrayDoubleCol":[1.1,2.2,3.3], "arrayBooleanCol": [true, false, 
true]} """.stripMargin
+val json5: String =
+  """ {"stringCol": "Alex","intCol": 15,"doubleCol": 11.5,"realCol": 13.7,
+|"boolCol": true, "arrayStringCol1": ["Street1", 
"Street2"],"arrayStringCol2": ["Japan",
+|"China", "India"],"arrayIntCol": 
[1,2,3,4],"arrayBigIntCol":[7,6,8000],
+|"arrayRealCol":[1.1,2.2,3.3],"arrayDoubleCol":[4,1,21.222,15.231],
+|"arrayBooleanCol": [false, false, false]} """.stripMargin
+
+
+val mySchema =

Review comment:
   These code is similar to what present in TestNonTransactional carbon 
table, better to extract it to common module [sdk] and reuse it instead of 
duplicating  





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] kunal642 commented on a change in pull request #3855: [CARBONDATA-3863], after using index service clean the temp data

2020-08-04 Thread GitBox


kunal642 commented on a change in pull request #3855:
URL: https://github.com/apache/carbondata/pull/3855#discussion_r465477710



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/indexserver/IndexServer.scala
##
@@ -316,4 +324,17 @@ object IndexServer extends ServerInterface {
   Array(new Service("security.indexserver.protocol.acl", 
classOf[ServerInterface]))
 }
   }
+
+  def startAgingFolders(): Unit = {
+val runnable = new Runnable() {
+  def run() {
+val age = System.currentTimeMillis() - agePeriod.toLong
+CarbonUtil.agingTempFolderForIndexServer(age)
+LOGGER.info(s"Complete age temp folder 
${CarbonUtil.getIndexServerTempPath}")
+  }
+}
+val ags: ScheduledExecutorService = 
Executors.newSingleThreadScheduledExecutor
+ags.scheduleAtFixedRate(runnable, 1000, 360, TimeUnit.MICROSECONDS)

Review comment:
   the unit you used is microseconds not milliseconds, better to keep it as 
5 min. 3s very less duration





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3773: [CARBONDATA-3830]Presto array columns read support

2020-08-04 Thread GitBox


ajantha-bhat commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r465477333



##
File path: 
integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/GenerateFiles.scala
##
@@ -0,0 +1,650 @@
+package org.apache.carbondata.presto.integrationtest

Review comment:
   add license header





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3773: [CARBONDATA-3830]Presto array columns read support

2020-08-04 Thread GitBox


ajantha-bhat commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r465477228



##
File path: 
integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoReadTableFilesTest.scala
##
@@ -0,0 +1,398 @@
+package org.apache.carbondata.presto.integrationtest
+
+import java.io.File
+import java.util
+import java.util.Arrays.asList
+
+import io.prestosql.jdbc.PrestoArray
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.presto.server.PrestoServer
+import org.apache.commons.io.FileUtils
+import org.scalatest.{BeforeAndAfterAll, FunSuiteLike, BeforeAndAfterEach}
+
+class PrestoReadTableFilesTest extends FunSuiteLike with BeforeAndAfterAll 
with BeforeAndAfterEach{
+  private val logger = LogServiceFactory
+
.getLogService(classOf[PrestoTestNonTransactionalTableFiles].getCanonicalName)

Review comment:
   class name is wrong in logger class





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] kunal642 commented on a change in pull request #3855: [CARBONDATA-3863], after using index service clean the temp data

2020-08-04 Thread GitBox


kunal642 commented on a change in pull request #3855:
URL: https://github.com/apache/carbondata/pull/3855#discussion_r465476995



##
File path: 
core/src/main/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFile.java
##
@@ -485,4 +486,30 @@ public boolean equals(Object o) {
   public int hashCode() {
 return Objects.hash(file.getAbsolutePath());
   }
+
+  @Override
+  public List listDirs() throws IOException {
+if (!file.isDirectory()) {
+  return new ArrayList();
+}
+Collection fileCollection = FileUtils.listFilesAndDirs(file,
+DirectoryFileFilter.DIRECTORY, null);
+if (fileCollection.isEmpty()) {
+  return new ArrayList();
+}
+List carbonFiles = new ArrayList();
+for (File file : fileCollection) {
+  if (file.isDirectory()) {
+File[] files = file.listFiles();

Review comment:
   This is a generic API.. i think the output for this should be 
"fileCollection" i.e listing only once...if you want to list the subfolder then 
better to add a recursive parameter or pass a pattern like /tmp/*/* so that it 
can list the folders with 2 depth.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3773: [CARBONDATA-3830]Presto array columns read support

2020-08-04 Thread GitBox


ajantha-bhat commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r465477005



##
File path: 
integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoReadTableFilesTest.scala
##
@@ -0,0 +1,398 @@
+package org.apache.carbondata.presto.integrationtest

Review comment:
   add license header 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] Karan-c980 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


Karan-c980 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668983403


   retest this please



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-668849764


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3607/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3773: [CARBONDATA-3830]Presto array columns read support

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#issuecomment-668849338


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1868/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668803681


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1867/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668802657


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3606/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] Karan980 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


Karan980 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668734484


   retest this please



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668715245


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1866/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668714775


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3605/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3862: [CARBONDATA-3933]Fix DDL/DML failures after table is created with column names having special characters like #,\,%

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3862:
URL: https://github.com/apache/carbondata/pull/3862#issuecomment-668676377


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1865/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3862: [CARBONDATA-3933]Fix DDL/DML failures after table is created with column names having special characters like #,\,%

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3862:
URL: https://github.com/apache/carbondata/pull/3862#issuecomment-668676112


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3604/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3773: [CARBONDATA-3830]Presto array columns read support

2020-08-04 Thread GitBox


ajantha-bhat commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r465142026



##
File path: 
integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ArrayStreamReader.java
##
@@ -0,0 +1,163 @@
+/*
+ * 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.presto.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import 
org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+
+/**
+ * Class to read the Array Stream
+ */
+
+public class ArrayStreamReader extends CarbonColumnVectorImpl implements 
PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+  Block childBlock = null;
+  private int index = 0;
+
+  public ArrayStreamReader(int batchSize, DataType dataType, StructField 
field) {
+super(batchSize, dataType);
+this.batchSize = batchSize;
+this.type = getArrayOfType(field, dataType);
+ArrayList childrenList= new ArrayList<>();
+
childrenList.add(CarbonVectorBatch.createDirectStreamReader(this.batchSize, 
field.getDataType(), field));
+setChildrenVector(childrenList);
+this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  public int getIndex() {
+return index;
+  }
+
+  public void setIndex(int index) {
+this.index = index;
+  }
+
+  public String getDataTypeName() {
+return "ARRAY";
+  }
+
+  Type getArrayOfType(StructField field, DataType dataType) {
+if (dataType == DataTypes.STRING) {
+  return new ArrayType(VarcharType.VARCHAR);
+} else if (dataType == DataTypes.BYTE) {
+  return new ArrayType(TinyintType.TINYINT);
+} else if (dataType == DataTypes.SHORT) {
+  return new ArrayType(SmallintType.SMALLINT);
+} else if (dataType == DataTypes.INT) {
+  return new ArrayType(IntegerType.INTEGER);
+} else if (dataType == DataTypes.LONG) {
+  return new ArrayType(BigintType.BIGINT);
+} else if (dataType == DataTypes.DOUBLE) {
+  return new ArrayType(DoubleType.DOUBLE);
+} else if (dataType == DataTypes.FLOAT) {
+  return new ArrayType(RealType.REAL);
+} else if (dataType == DataTypes.BOOLEAN) {
+  return new ArrayType(BooleanType.BOOLEAN);
+} else if (dataType == DataTypes.TIMESTAMP) {
+  return new ArrayType(TimestampType.TIMESTAMP);
+} else if (DataTypes.isArrayType(dataType)) {
+  StructField childField = field.getChildren().get(0);
+  return new ArrayType(getArrayOfType(childField, 
childField.getDataType()));
+} else {
+  throw new UnsupportedOperationException("Unsupported type: " + dataType);
+}
+  }
+
+  @Override
+  public Block buildBlock() {
+return builder.build();
+  }
+
+  public boolean isComplex() {
+return true;
+  }
+
+  @Override
+  public void setBatchSize(int batchSize) {
+this.batchSize = batchSize;
+  }
+
+  @Override
+  public void putObject(int rowId, Object value) {
+if (value == null) {

Review comment:
   putObject is never used ? Instead of putComplexObject, may be we need to 
use the same interface 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3773: [CARBONDATA-3830]Presto array columns read support

2020-08-04 Thread GitBox


ajantha-bhat commented on a change in pull request #3773:
URL: https://github.com/apache/carbondata/pull/3773#discussion_r465142026



##
File path: 
integration/presto/src/main/prestosql/org/apache/carbondata/presto/readers/ArrayStreamReader.java
##
@@ -0,0 +1,163 @@
+/*
+ * 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.presto.readers;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import io.prestosql.spi.type.*;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import 
org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+
+import org.apache.carbondata.presto.CarbonVectorBatch;
+
+/**
+ * Class to read the Array Stream
+ */
+
+public class ArrayStreamReader extends CarbonColumnVectorImpl implements 
PrestoVectorBlockBuilder {
+
+  protected int batchSize;
+
+  protected Type type;
+  protected BlockBuilder builder;
+  Block childBlock = null;
+  private int index = 0;
+
+  public ArrayStreamReader(int batchSize, DataType dataType, StructField 
field) {
+super(batchSize, dataType);
+this.batchSize = batchSize;
+this.type = getArrayOfType(field, dataType);
+ArrayList childrenList= new ArrayList<>();
+
childrenList.add(CarbonVectorBatch.createDirectStreamReader(this.batchSize, 
field.getDataType(), field));
+setChildrenVector(childrenList);
+this.builder = type.createBlockBuilder(null, batchSize);
+  }
+
+  public int getIndex() {
+return index;
+  }
+
+  public void setIndex(int index) {
+this.index = index;
+  }
+
+  public String getDataTypeName() {
+return "ARRAY";
+  }
+
+  Type getArrayOfType(StructField field, DataType dataType) {
+if (dataType == DataTypes.STRING) {
+  return new ArrayType(VarcharType.VARCHAR);
+} else if (dataType == DataTypes.BYTE) {
+  return new ArrayType(TinyintType.TINYINT);
+} else if (dataType == DataTypes.SHORT) {
+  return new ArrayType(SmallintType.SMALLINT);
+} else if (dataType == DataTypes.INT) {
+  return new ArrayType(IntegerType.INTEGER);
+} else if (dataType == DataTypes.LONG) {
+  return new ArrayType(BigintType.BIGINT);
+} else if (dataType == DataTypes.DOUBLE) {
+  return new ArrayType(DoubleType.DOUBLE);
+} else if (dataType == DataTypes.FLOAT) {
+  return new ArrayType(RealType.REAL);
+} else if (dataType == DataTypes.BOOLEAN) {
+  return new ArrayType(BooleanType.BOOLEAN);
+} else if (dataType == DataTypes.TIMESTAMP) {
+  return new ArrayType(TimestampType.TIMESTAMP);
+} else if (DataTypes.isArrayType(dataType)) {
+  StructField childField = field.getChildren().get(0);
+  return new ArrayType(getArrayOfType(childField, 
childField.getDataType()));
+} else {
+  throw new UnsupportedOperationException("Unsupported type: " + dataType);
+}
+  }
+
+  @Override
+  public Block buildBlock() {
+return builder.build();
+  }
+
+  public boolean isComplex() {
+return true;
+  }
+
+  @Override
+  public void setBatchSize(int batchSize) {
+this.batchSize = batchSize;
+  }
+
+  @Override
+  public void putObject(int rowId, Object value) {
+if (value == null) {

Review comment:
   putObject is never used ? Instead of putArrayObject, may be we need to 
use the same interface 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#issuecomment-668657094


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1864/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#issuecomment-668655526


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3603/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3872: [CARBONDATA-3889] Enable java check style for all java modules

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3872:
URL: https://github.com/apache/carbondata/pull/3872#issuecomment-668649437


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3602/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3872: [CARBONDATA-3889] Enable java check style for all java modules

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3872:
URL: https://github.com/apache/carbondata/pull/3872#issuecomment-668649274


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1863/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] Karan980 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


Karan980 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668634313


   retest this please



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668624924


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1862/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668622658


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3601/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3872: [CARBONDATA-3889] Enable java check style for all java modules

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3872:
URL: https://github.com/apache/carbondata/pull/3872#issuecomment-668610320


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3598/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3877: [CARBONDATA-3889] Cleanup duplicated code in carbondata-spark module

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3877:
URL: https://github.com/apache/carbondata/pull/3877#issuecomment-668586669


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1861/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3877: [CARBONDATA-3889] Cleanup duplicated code in carbondata-spark module

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3877:
URL: https://github.com/apache/carbondata/pull/3877#issuecomment-668585899


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3600/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[jira] [Resolved] (CARBONDATA-3841) Remove useless string in create and alter command

2020-08-04 Thread Akash R Nilugal (Jira)


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

Akash R Nilugal resolved CARBONDATA-3841.
-
Fix Version/s: 2.1.0
   Resolution: Fixed

> Remove useless string in create and alter command
> -
>
> Key: CARBONDATA-3841
> URL: https://issues.apache.org/jira/browse/CARBONDATA-3841
> Project: CarbonData
>  Issue Type: Improvement
>Reporter: Manhua Jiang
>Priority: Major
> Fix For: 2.1.0
>
>  Time Spent: 2h 40m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [carbondata] asfgit closed pull request #3784: [CARBONDATA-3841] Remove useless string in create and alter command

2020-08-04 Thread GitBox


asfgit closed pull request #3784:
URL: https://github.com/apache/carbondata/pull/3784


   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] akashrn5 commented on pull request #3784: [CARBONDATA-3841] Remove useless string in create and alter command

2020-08-04 Thread GitBox


akashrn5 commented on pull request #3784:
URL: https://github.com/apache/carbondata/pull/3784#issuecomment-668565516


   LGTM



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#issuecomment-668547165


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1858/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#issuecomment-668545715


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3597/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] QiangCai opened a new pull request #3877: [CARBONDATA-3889] Cleanup duplicated code in carbondata-spark module

2020-08-04 Thread GitBox


QiangCai opened a new pull request #3877:
URL: https://github.com/apache/carbondata/pull/3877


### Why is this PR needed?
There are many duplicated code in carbondata-spark module

### What changes were proposed in this PR?
Cleanup duplicated code in carbondata-spark module
   
### Does this PR introduce any user interface change?
- No
   
### Is any new testcase added?
- No
   
   
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668543909


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3599/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668543546


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1859/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668536385


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1855/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668534633


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3594/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3837: [CARBONDATA-3927]Remove compressor name from tupleID to make it short to improve store size and performance.

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3837:
URL: https://github.com/apache/carbondata/pull/3837#issuecomment-668490907


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1851/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3837: [CARBONDATA-3927]Remove compressor name from tupleID to make it short to improve store size and performance.

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3837:
URL: https://github.com/apache/carbondata/pull/3837#issuecomment-668490005


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3590/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r464911706



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -154,13 +154,18 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("CREATE TABLE load32000chardata(dim1 String, dim2 String, mes1 int) 
STORED AS carbondata")
 sql("CREATE TABLE load32000chardata_dup(dim1 String, dim2 String, mes1 
int) STORED AS carbondata")
 sql(s"LOAD DATA LOCAL INPATH '$testdata' into table load32000chardata 
OPTIONS('FILEHEADER'='dim1,dim2,mes1')")
-intercept[Exception] {
-  sql("insert into load32000chardata_dup select 
dim1,concat(load32000chardata.dim2,''),mes1 from load32000chardata").show()
-}
+checkAnswer(sql("select count(*) from load32000chardata"), Seq(Row(3)))

Review comment:
   It was handled for all types of action. In case of "FORCE" action, I am 
just converting long string to null. Now I have added testcases for all those 
types.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#issuecomment-668480465


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3595/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r464910778



##
File path: 
processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
##
@@ -82,21 +83,25 @@ public Object convert(Object value, BadRecordLogHolder 
logHolder)
   .getBytesBasedOnDataTypeForNoDictionaryColumn(dimensionValue, 
dataType, dateFormat);
   if (dataType == DataTypes.STRING
   && parsedValue.length > 
CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-throw new CarbonDataLoadingException(String.format(
-"Dataload failed, String size cannot exceed %d bytes,"
-+ " please consider long string data type",
-CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT));
+
logHolder.setReason(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE);
+String badRecordAction = CarbonProperties.getInstance()
+.getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION);
+if (badRecordAction.equalsIgnoreCase("FORCE")) {

Review comment:
   done.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] nihal0107 commented on a change in pull request #3865: [CARBONDATA-3928] Handled the Strings which length is greater than 32000 as a bad record.

2020-08-04 Thread GitBox


nihal0107 commented on a change in pull request #3865:
URL: https://github.com/apache/carbondata/pull/3865#discussion_r464908207



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
##
@@ -75,11 +75,10 @@ object CarbonScalaUtil {
 carbonLoadModel.getBinaryDecoder)
 } catch {
   case e: Exception =>
-if 
(e.getMessage.startsWith(FieldConverter.stringLengthExceedErrorMsg)) {
-  val msg = s"Column 
${carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-.getCreateOrderColumn.get(idx).getColName} is too long," +
-s" consider to use 'long_string_columns' table property."
-  LOGGER.error(msg, e)
+if 
(e.getMessage.startsWith(CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE)) 
{
+  val msg = 
CarbonCommonConstants.STRING_LENGTH_EXCEEDED_MESSAGE.format(row,
+  
carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getCreateOrderColumn
+.get(idx).getColName)

Review comment:
   done.

##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
##
@@ -170,21 +175,22 @@ class TestLoadDataGeneral extends QueryTest with 
BeforeAndAfterEach {
 sql("drop table if exists load32000bytes")
 sql("create table load32000bytes(name string) STORED AS carbondata")
 sql("insert into table load32000bytes select 'aaa'")
+checkAnswer(sql("select count(*) from load32000bytes"), Seq(Row(1)))
 
-assert(intercept[Exception] {
-  sql(s"load data local inpath '$testdata' into table load32000bytes 
OPTIONS ('FILEHEADER'='name')")
-}.getMessage.contains("DataLoad failure: Dataload failed, String size 
cannot exceed 32000 bytes"))
+// Below load will be inserted as null because Strings greater than 32000 
is bad record.
+sql(s"load data local inpath '$testdata' into table load32000bytes OPTIONS 
('FILEHEADER'='name')")
+checkAnswer(sql("select count(*) from load32000bytes"), Seq(Row(2)))
+checkAnswer(sql("select * from load32000bytes"), Seq(Row("aaa"), 
Row(null)))
 
 val source = scala.io.Source.fromFile(testdata, 
CarbonCommonConstants.DEFAULT_CHARSET)
 val data = source.mkString
 
+// Insert will throw exception as it is without converter step.
 intercept[Exception] {
   sql(s"insert into load32000bytes values('$data')")
 }
 
-intercept[Exception] {
-  sql(s"update load32000bytes set(name)= ('$data')").show()
-}
+sql(s"update load32000bytes set(name)= ('$data')").show()

Review comment:
   done.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[jira] [Resolved] (CARBONDATA-3939) Exception added for index creation on long string columns

2020-08-04 Thread Akash R Nilugal (Jira)


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

Akash R Nilugal resolved CARBONDATA-3939.
-
Fix Version/s: 2.1.0
   Resolution: Fixed

> Exception added for index creation on long string columns
> -
>
> Key: CARBONDATA-3939
> URL: https://issues.apache.org/jira/browse/CARBONDATA-3939
> Project: CarbonData
>  Issue Type: Bug
>Reporter: Akshay
>Priority: Major
> Fix For: 2.1.0
>
>  Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> Index creation for long string columns are not yet supported.
> User understandable exceptions are thrown if user tries to create the same.
> https://github.com/apache/carbondata/pull/3869



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [carbondata] asfgit closed pull request #3869: [CARBONDATA-3939]Exception added for index creation on long string columns

2020-08-04 Thread GitBox


asfgit closed pull request #3869:
URL: https://github.com/apache/carbondata/pull/3869


   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] akashrn5 commented on pull request #3869: [CARBONDATA-3939]Exception added for index creation on long string columns

2020-08-04 Thread GitBox


akashrn5 commented on pull request #3869:
URL: https://github.com/apache/carbondata/pull/3869#issuecomment-668474607


   LGTM



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668443444


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3593/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668442971


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1854/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668429141


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3592/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668428636


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1853/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668424168


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1852/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668422443


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3591/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] kunal642 commented on a change in pull request #3866: [CARBONDATA-3915] Correction in the documentation for spark-shell

2020-08-04 Thread GitBox


kunal642 commented on a change in pull request #3866:
URL: https://github.com/apache/carbondata/pull/3866#discussion_r464844335



##
File path: docs/hive-guide.md
##
@@ -52,16 +52,11 @@ $HADOOP_HOME/bin/hadoop fs -put sample.csv /sample.csv
 ```
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.CarbonSession._
-val rootPath = "hdfs:///user/hadoop/carbon"
-val storeLocation = s"$rootPath/store"
-val warehouse = s"$rootPath/warehouse"
-val metaStoreDB = s"$rootPath/metastore_db"
-
-val carbon = 
SparkSession.builder().enableHiveSupport().config("spark.sql.warehouse.dir", 
warehouse).config(org.apache.carbondata.core.constants.CarbonCommonConstants.STORE_LOCATION,
 storeLocation).getOrCreateCarbonSession(storeLocation, metaStoreDB)
-
-carbon.sql("create table hive_carbon(id int, name string, scale decimal, 
country string, salary double) STORED AS carbondata")
-carbon.sql("LOAD DATA INPATH '/sample.csv' INTO TABLE 
hive_carbon")
-scala>carbon.sql("SELECT * FROM hive_carbon").show()
+val newSpark = 
SparkSession.builder().config(sc.getConf).enableHiveSupport.config("spark.sql.extensions","org.apache.spark.sql.CarbonExtensions").getOrCreate()
+newSpark.sql("drop table if exists hive_carbon").show
+newSpark.sql("create table hive_carbon(id int, name string, scale decimal, 
country string, salary double) STORED AS carbondata").show
+newSpark.sql("/sample.csv INTO TABLE hive_carbon").show

Review comment:
   Please fix the load query, many keywords including load data are missing





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] kunal642 commented on a change in pull request #3837: [CARBONDATA-3927]Remove compressor name from tupleID to make it short to improve store size and performance.

2020-08-04 Thread GitBox


kunal642 commented on a change in pull request #3837:
URL: https://github.com/apache/carbondata/pull/3837#discussion_r464843011



##
File path: 
core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
##
@@ -952,10 +952,22 @@ public static String getSegmentBlockNameKey(String segID, 
String blockName,
 String blockNameWithOutPart = blockName
 .substring(blockName.indexOf(CarbonCommonConstants.HYPHEN) + 1,
 blockName.lastIndexOf(CarbonTablePath.getCarbonDataExtension()));
+// to remove compressor name
+int index = blockNameWithOutPart.lastIndexOf(CarbonCommonConstants.POINT);
 if (isPartitionTable) {
-  return blockNameWithOutPart;
+  if (index != -1) {
+return 
blockNameWithOutPart.replace(blockNameWithOutPart.substring(index), "");
+  } else {
+return blockNameWithOutPart;
+  }
+}
+if (index != -1) {
+  String blockNameWithoutCompressorName =
+  blockNameWithOutPart.replace(blockNameWithOutPart.substring(index), 
"");
+  return segID + CarbonCommonConstants.FILE_SEPARATOR + 
blockNameWithoutCompressorName;
+} else {
+  return segID + CarbonCommonConstants.FILE_SEPARATOR + 
blockNameWithOutPart;

Review comment:
   Please add a unit test to validate this functionality





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] kunal642 commented on pull request #3837: [CARBONDATA-3927]Remove compressor name from tupleID to make it short to improve store size and performance.

2020-08-04 Thread GitBox


kunal642 commented on pull request #3837:
URL: https://github.com/apache/carbondata/pull/3837#issuecomment-668418007


   retest this please



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668410305


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3589/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3876: TestingCI

2020-08-04 Thread GitBox


CarbonDataQA1 commented on pull request #3876:
URL: https://github.com/apache/carbondata/pull/3876#issuecomment-668409887


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1850/
   



This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org