[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-31 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r479973113



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -660,13 +895,39 @@ public CarbonWriter build() throws IOException, 
InvalidLoadOptionException {
   // removed from the load. LoadWithoutConverter flag is going to point to 
the Loader Builder
   // which will skip Conversion Step.
   loadModel.setLoadWithoutConverterStep(true);
-  return new AvroCarbonWriter(loadModel, hadoopConf, this.avroSchema);
+  AvroCarbonWriter avroCarbonWriter = new AvroCarbonWriter(loadModel,

Review comment:
   We have some code duplications for each type of writer. Suggest to 
refactor it. Something like this - 
   ```suggestion
   CarbonWriter carbonWriter;
   if (this.writerType == WRITER_TYPE.AVRO) {
 // AVRO records are pushed to Carbon as Object not as Strings. This 
was done in order to
 // handle multi level complex type support. As there are no conversion 
converter step is
 // removed from the load. LoadWithoutConverter flag is going to point 
to the Loader Builder
 // which will skip Conversion Step.
 loadModel.setLoadWithoutConverterStep(true);
 carbonWriter = new AvroCarbonWriter(loadModel, hadoopConf, 
this.avroSchema);
   } else if (this.writerType == WRITER_TYPE.JSON) {
 loadModel.setJsonFileLoad(true);
 carbonWriter = new JsonCarbonWriter(loadModel, hadoopConf);
   } else if (this.writerType == WRITER_TYPE.PARQUET) {
 loadModel.setLoadWithoutConverterStep(true);
 carbonWriter = new ParquetCarbonWriter(loadModel, hadoopConf, 
this.avroSchema);
   } else if (this.writerType == WRITER_TYPE.ORC) {
 carbonWriter = new ORCCarbonWriter(loadModel, hadoopConf);
   } else {
 // CSV
 CSVCarbonWriter csvCarbonWriter = new CSVCarbonWriter(loadModel, 
hadoopConf);
 if (!this.options.containsKey(CarbonCommonConstants.FILE_HEADER)) {
   csvCarbonWriter.setSkipHeader(true);
 }
 carbonWriter = csvCarbonWriter;
   }
   if (!StringUtils.isEmpty(filePath)) {
 carbonWriter.validateAndSetDataFiles(this.dataFiles);
   }
   return carbonWriter;
   ```





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-31 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r479962896



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -594,6 +608,227 @@ public CarbonWriterBuilder withJsonInput(Schema 
carbonSchema) {
 return this;
   }
 
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading CSV files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withCsvInput();
+this.dataFiles = 
this.extractDataFiles(CarbonCommonConstants.CSV_FILE_EXTENSION);
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts CSV files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the CSV file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withCsvPath(filePath);
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading JSON files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withJsonInput();
+this.dataFiles = 
this.extractDataFiles(CarbonCommonConstants.JSON_FILE_EXTENSION);
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts JSON file directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the json file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   * @throws IOException
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withJsonPath(filePath);
+return this;
+  }
+
+  private void validateFilePath(String filePath) {
+if (StringUtils.isEmpty(filePath)) {
+  throw new IllegalArgumentException("filePath can not be empty");
+}
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading Parquet files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withParquetPath(String filePath) throws 
IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.writerType = WRITER_TYPE.PARQUET;
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.PARQUET_FILE_EXT);
+org.apache.avro.Schema parquetSchema = ParquetCarbonWriter
+.extractParquetSchema(dataFiles[0], this.hadoopConf);
+this.dataFiles = dataFiles;
+this.avroSchema = parquetSchema;
+this.schema = 
AvroCarbonWriter.getCarbonSchemaFromAvroSchema(this.avroSchema);
+return this;
+  }
+
+  private void setIsDirectory(String filePath) {
+if (this.hadoopConf == null) {
+  this.hadoopConf = new Configuration(FileFactory.getConfiguration());
+}
+CarbonFile carbonFile = FileFactory.getCarbonFile(filePath, hadoopConf);
+this.isDirectory = carbonFile.isDirectory();
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts parquet files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the parquet file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   * @throws IOException
+   */
+  public CarbonWriterBuilder withParquetPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withParquetPath(filePath);
+return this;
+  }
+
+  private CarbonFile[] extractDataFiles(String suf) {
+List dataFiles;
+if (this.isDirectory) {
+  if (CollectionUtils.isEmpty(this.fileList)) {
+dataFiles = SDKUtil.extractFilesFromFolder(this.filePath, suf, 
this.hadoopConf);
+  } else {
+dataFiles = this.appendFileListWithPath();
+  }
+} else {
+  dataFiles = new ArrayList<>();
+  dataFiles.add(FileFactory.getCarbonFile(this.filePath, this.hadoopConf));
+}
+if (CollectionUtils.isEmpty(dataFiles)) {
+  throw new RuntimeException("Data files can't be empty.");
+}
+return dataFiles.toArray(new CarbonFile[0]);
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading ORC files.
+   *
+   * @param 

[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-31 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r479961626



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java
##
@@ -72,6 +90,72 @@ public void write(Object object) throws IOException {
 }
   }
 
+  public static CsvParser buildCsvParser(Configuration conf) {

Review comment:
   can it be private and non-static method?





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-31 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r479961626



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java
##
@@ -72,6 +90,72 @@ public void write(Object object) throws IOException {
 }
   }
 
+  public static CsvParser buildCsvParser(Configuration conf) {

Review comment:
   It can be private and non-static method?





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-28 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r479000935



##
File path: 
sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/ParquetCarbonWriterTest.java
##
@@ -0,0 +1,179 @@
+/*
+ * 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.sdk.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test suite for {@link ParquetCarbonWriter}
+ */
+public class ParquetCarbonWriterTest {
+  String DATA_PATH = "./src/test/resources/file/";
+  String outputPath = "./testWriteFiles";
+
+  @Before
+  @After
+  public void cleanTestData() {
+try {
+  FileUtils.deleteDirectory(new File(outputPath));
+} catch (Exception e) {
+  e.printStackTrace();
+  Assert.fail(e.getMessage());

Review comment:
   Same as above. Assert.fail from here.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-28 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478989788



##
File path: 
sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/ORCCarbonWriterTest.java
##
@@ -0,0 +1,165 @@
+/*
+ * 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.sdk.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test suite for {@link ORCCarbonWriter}
+ */
+public class ORCCarbonWriterTest {
+
+  String DATA_PATH = "./src/test/resources/file/";
+  String outputPath = "./testloadORCFiles";
+
+  @Before
+  @After
+  public void cleanTestData() {
+try {
+  FileUtils.deleteDirectory(new File(outputPath));
+} catch (Exception e) {
+  e.printStackTrace();
+  Assert.fail(e.getMessage());

Review comment:
   Assert.fail from here.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-28 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r47920



##
File path: 
sdk/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java
##
@@ -603,4 +605,161 @@ public void testWriteBasicForFloat() throws IOException {
 }
   }
 
+  @Test
+  public void testAvroFileLoadWithPrimitiveSchema() throws IOException {

Review comment:
   This testcase is redundant. It is covered in 
testLoadingAvroFileAndReadingCarbonFile





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-28 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478869005



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/utils/SDKUtil.java
##
@@ -79,4 +98,75 @@ public static ArrayList listFiles(String sourceImageFolder,
 return (Object[]) input[i];
   }
 
+  public static List extractFilesFromFolder(String path,
+  String suf, Configuration hadoopConf) {
+List dataFiles = listFiles(path, suf, hadoopConf);
+List carbonFiles = new ArrayList<>();
+for (Object dataFile: dataFiles) {
+  carbonFiles.add(FileFactory.getCarbonFile(dataFile.toString(), 
hadoopConf));
+}
+if (CollectionUtils.isEmpty(dataFiles)) {
+  throw new RuntimeException("No file found at given location. Please 
provide" +
+  "the correct folder location.");
+}
+return carbonFiles;
+  }
+
+  public static DataFileStream buildAvroReader(CarbonFile 
carbonFile,

Review comment:
   If validate Files methods are refactored to respecitive type 
carbonwriter as given in another review comment, no longer require these type 
specific buildreaders in this file(SDKUtil.java). Same for other readers 
below(buildOrcReader, buildParquetReader, buildCsvParser, buildJsonReader)





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-28 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478852900



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/ParquetCarbonWriter.java
##
@@ -0,0 +1,89 @@
+/*
+ * 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.sdk.file;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.sdk.file.utils.SDKUtil;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.hadoop.ParquetReader;
+
+/**
+ * Implementation to write parquet rows in avro format to carbondata file.
+ */
+public class ParquetCarbonWriter extends AvroCarbonWriter {

Review comment:
   Why extending `AvroCarbonWriter` and also having it as field in this 
class? Both inheritence & composition at the same time ? This case need just 
one of them. probably you need to relook into this once again.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-28 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478853321



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/ORCCarbonWriter.java
##
@@ -0,0 +1,173 @@
+/*
+ * 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.sdk.file;
+
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.sdk.file.utils.SDKUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
+import org.apache.hadoop.hive.ql.io.orc.Reader;
+import org.apache.hadoop.hive.ql.io.orc.RecordReader;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.Text;
+
+
+/**
+ * Implementation to write ORC rows in CSV format to carbondata file.
+ */
+public class ORCCarbonWriter extends CSVCarbonWriter {

Review comment:
   Why extending CSVCarbonWriter and also having it as field in this class? 
Both inheritence & composition at the same time ? This case need just one of 
them. probably you need to relook into this once again.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-28 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478853321



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/ORCCarbonWriter.java
##
@@ -0,0 +1,173 @@
+/*
+ * 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.sdk.file;
+
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.sdk.file.utils.SDKUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
+import org.apache.hadoop.hive.ql.io.orc.Reader;
+import org.apache.hadoop.hive.ql.io.orc.RecordReader;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.Text;
+
+
+/**
+ * Implementation to write ORC rows in CSV format to carbondata file.
+ */
+public class ORCCarbonWriter extends CSVCarbonWriter {

Review comment:
   Why extending CSVCarbonWriter and also having it as field in this class? 
Both inheritence & composition at the same time ? Just need one of them. I 
think, you need to relook into this once again.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-28 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478852900



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/ParquetCarbonWriter.java
##
@@ -0,0 +1,89 @@
+/*
+ * 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.sdk.file;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.sdk.file.utils.SDKUtil;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.hadoop.ParquetReader;
+
+/**
+ * Implementation to write parquet rows in avro format to carbondata file.
+ */
+public class ParquetCarbonWriter extends AvroCarbonWriter {

Review comment:
   Why extending `AvroCarbonWriter` and also having it as field in this 
class? Both inheritence & composition at the same time ? Just need one of them. 
I think, you need to relook into this once again. 





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478824388



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -594,6 +607,446 @@ public CarbonWriterBuilder withJsonInput(Schema 
carbonSchema) {
 return this;
   }
 
+  private void validateCsvFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.CSV_FILE_EXTENSION);
+if (CollectionUtils.isEmpty(Arrays.asList(dataFiles))) {
+  throw new RuntimeException("CSV files can't be empty.");
+}
+for (CarbonFile dataFile : dataFiles) {
+  try {
+CsvParser csvParser = SDKUtil.buildCsvParser(this.hadoopConf);
+
csvParser.beginParsing(FileFactory.getDataInputStream(dataFile.getPath(),
+-1, this.hadoopConf));
+  } catch (IllegalArgumentException ex) {
+if (ex.getCause() instanceof FileNotFoundException) {
+  throw new FileNotFoundException("File " + dataFile +
+  " not found to build carbon writer.");
+}
+throw ex;
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading CSV files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withCsvInput();
+this.validateCsvFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts CSV files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the CSV file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withCsvPath(filePath);
+return this;
+  }
+
+  private void validateJsonFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.JSON_FILE_EXTENSION);
+for (CarbonFile dataFile : dataFiles) {
+  try {
+new JSONParser().parse(SDKUtil.buildJsonReader(dataFile, 
this.hadoopConf));
+  } catch (FileNotFoundException ex) {
+throw new FileNotFoundException("File " + dataFile + " not found to 
build carbon writer.");
+  } catch (ParseException ex) {
+throw new RuntimeException("File " + dataFile + " is not in json 
format.");
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading JSON files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withJsonInput();
+this.validateJsonFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts JSON file directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the json file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   * @throws IOException
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withJsonPath(filePath);
+return this;
+  }
+
+  private void validateFilePath(String filePath) {
+if (StringUtils.isEmpty(filePath)) {
+  throw new IllegalArgumentException("filePath can not be empty");
+}
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading Parquet files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withParquetPath(String filePath) throws 
IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.writerType = WRITER_TYPE.PARQUET;
+this.validateParquetFiles();
+return this;
+  }
+
+  private void setIsDirectory(String filePath) {
+if (this.hadoopConf == null) {
+  this.hadoopConf = new Configuration(FileFactory.getConfiguration());
+}
+CarbonFile carbonFile = FileFactory.getCarbonFile(filePath, hadoopConf);
+this.isDirectory = carbonFile.isDirectory();
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts parquet files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the parquet file exists.
+   * @param fileList list of files which has to be 

[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478824388



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -594,6 +607,446 @@ public CarbonWriterBuilder withJsonInput(Schema 
carbonSchema) {
 return this;
   }
 
+  private void validateCsvFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.CSV_FILE_EXTENSION);
+if (CollectionUtils.isEmpty(Arrays.asList(dataFiles))) {
+  throw new RuntimeException("CSV files can't be empty.");
+}
+for (CarbonFile dataFile : dataFiles) {
+  try {
+CsvParser csvParser = SDKUtil.buildCsvParser(this.hadoopConf);
+
csvParser.beginParsing(FileFactory.getDataInputStream(dataFile.getPath(),
+-1, this.hadoopConf));
+  } catch (IllegalArgumentException ex) {
+if (ex.getCause() instanceof FileNotFoundException) {
+  throw new FileNotFoundException("File " + dataFile +
+  " not found to build carbon writer.");
+}
+throw ex;
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading CSV files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withCsvInput();
+this.validateCsvFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts CSV files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the CSV file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withCsvPath(filePath);
+return this;
+  }
+
+  private void validateJsonFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.JSON_FILE_EXTENSION);
+for (CarbonFile dataFile : dataFiles) {
+  try {
+new JSONParser().parse(SDKUtil.buildJsonReader(dataFile, 
this.hadoopConf));
+  } catch (FileNotFoundException ex) {
+throw new FileNotFoundException("File " + dataFile + " not found to 
build carbon writer.");
+  } catch (ParseException ex) {
+throw new RuntimeException("File " + dataFile + " is not in json 
format.");
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading JSON files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withJsonInput();
+this.validateJsonFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts JSON file directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the json file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   * @throws IOException
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withJsonPath(filePath);
+return this;
+  }
+
+  private void validateFilePath(String filePath) {
+if (StringUtils.isEmpty(filePath)) {
+  throw new IllegalArgumentException("filePath can not be empty");
+}
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading Parquet files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withParquetPath(String filePath) throws 
IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.writerType = WRITER_TYPE.PARQUET;
+this.validateParquetFiles();
+return this;
+  }
+
+  private void setIsDirectory(String filePath) {
+if (this.hadoopConf == null) {
+  this.hadoopConf = new Configuration(FileFactory.getConfiguration());
+}
+CarbonFile carbonFile = FileFactory.getCarbonFile(filePath, hadoopConf);
+this.isDirectory = carbonFile.isDirectory();
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts parquet files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the parquet file exists.
+   * @param fileList list of files which has to be 

[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478545280



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/ORCCarbonWriter.java
##
@@ -0,0 +1,173 @@
+/*
+ * 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.sdk.file;
+
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.sdk.file.utils.SDKUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
+import org.apache.hadoop.hive.ql.io.orc.Reader;
+import org.apache.hadoop.hive.ql.io.orc.RecordReader;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.Text;
+
+
+/**
+ * Implementation to write ORC rows in CSV format to carbondata file.
+ */
+public class ORCCarbonWriter extends CSVCarbonWriter {
+  private Configuration configuration;
+  private CSVCarbonWriter csvCarbonWriter = null;
+  private Reader orcReader = null;
+  private CarbonFile[] dataFiles;
+
+  ORCCarbonWriter(CSVCarbonWriter csvCarbonWriter, Configuration 
configuration) {
+this.csvCarbonWriter = csvCarbonWriter;
+this.configuration = configuration;
+  }
+
+  @Override
+  public void setDataFiles(CarbonFile[] dataFiles) {
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * Load ORC file in iterative way.
+   */
+  @Override
+  public void write() throws IOException {
+if (this.dataFiles == null || this.dataFiles.length == 0) {
+  throw new RuntimeException("'withOrcPath()' must be called to support 
loading ORC files");
+}
+if (this.csvCarbonWriter == null) {
+  throw new RuntimeException("csv carbon writer can not be null");
+}
+Arrays.sort(this.dataFiles, Comparator.comparing(CarbonFile::getPath));
+for (CarbonFile dataFile : this.dataFiles) {
+  this.loadSingleFile(dataFile);
+}
+  }
+
+  private void loadSingleFile(CarbonFile file) throws IOException {
+orcReader = SDKUtil.buildOrcReader(file.getPath(), this.configuration);
+ObjectInspector objectInspector = orcReader.getObjectInspector();
+RecordReader recordReader = orcReader.rows();
+if (objectInspector instanceof StructObjectInspector) {
+  StructObjectInspector structObjectInspector =
+  (StructObjectInspector) orcReader.getObjectInspector();
+  while (recordReader.hasNext()) {
+Object record = recordReader.next(null); // to remove duplicacy.
+List valueList = 
structObjectInspector.getStructFieldsDataAsList(record);
+for (int i = 0; i < valueList.size(); i++) {
+  valueList.set(i, parseOrcObject(valueList.get(i), 0));
+}
+this.csvCarbonWriter.write(valueList.toArray());
+  }
+} else {
+  while (recordReader.hasNext()) {

Review comment:
   Curious to know when does this else case hit ? You testcase do not seem 
to cover it





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478495670



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/ORCCarbonWriter.java
##
@@ -0,0 +1,173 @@
+/*
+ * 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.sdk.file;
+
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.sdk.file.utils.SDKUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
+import org.apache.hadoop.hive.ql.io.orc.Reader;
+import org.apache.hadoop.hive.ql.io.orc.RecordReader;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.Text;
+
+
+/**
+ * Implementation to write ORC rows in CSV format to carbondata file.
+ */
+public class ORCCarbonWriter extends CSVCarbonWriter {
+  private Configuration configuration;
+  private CSVCarbonWriter csvCarbonWriter = null;
+  private Reader orcReader = null;
+  private CarbonFile[] dataFiles;
+
+  ORCCarbonWriter(CSVCarbonWriter csvCarbonWriter, Configuration 
configuration) {
+this.csvCarbonWriter = csvCarbonWriter;
+this.configuration = configuration;
+  }
+
+  @Override
+  public void setDataFiles(CarbonFile[] dataFiles) {
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * Load ORC file in iterative way.
+   */
+  @Override
+  public void write() throws IOException {
+if (this.dataFiles == null || this.dataFiles.length == 0) {
+  throw new RuntimeException("'withOrcPath()' must be called to support 
loading ORC files");
+}
+if (this.csvCarbonWriter == null) {
+  throw new RuntimeException("csv carbon writer can not be null");
+}
+Arrays.sort(this.dataFiles, Comparator.comparing(CarbonFile::getPath));
+for (CarbonFile dataFile : this.dataFiles) {
+  this.loadSingleFile(dataFile);
+}
+  }
+
+  private void loadSingleFile(CarbonFile file) throws IOException {
+orcReader = SDKUtil.buildOrcReader(file.getPath(), this.configuration);
+ObjectInspector objectInspector = orcReader.getObjectInspector();
+RecordReader recordReader = orcReader.rows();
+if (objectInspector instanceof StructObjectInspector) {
+  StructObjectInspector structObjectInspector =
+  (StructObjectInspector) orcReader.getObjectInspector();
+  while (recordReader.hasNext()) {
+Object record = recordReader.next(null); // to remove duplicacy.

Review comment:
   was looking at API documentation and references from internet. Info in 
the documentation is limited though. `recordReader.next()` takes previous 
record as arg. But we are always passing null ? Same applies to below else case 
too





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478495670



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/ORCCarbonWriter.java
##
@@ -0,0 +1,173 @@
+/*
+ * 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.sdk.file;
+
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.sdk.file.utils.SDKUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
+import org.apache.hadoop.hive.ql.io.orc.Reader;
+import org.apache.hadoop.hive.ql.io.orc.RecordReader;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.Text;
+
+
+/**
+ * Implementation to write ORC rows in CSV format to carbondata file.
+ */
+public class ORCCarbonWriter extends CSVCarbonWriter {
+  private Configuration configuration;
+  private CSVCarbonWriter csvCarbonWriter = null;
+  private Reader orcReader = null;
+  private CarbonFile[] dataFiles;
+
+  ORCCarbonWriter(CSVCarbonWriter csvCarbonWriter, Configuration 
configuration) {
+this.csvCarbonWriter = csvCarbonWriter;
+this.configuration = configuration;
+  }
+
+  @Override
+  public void setDataFiles(CarbonFile[] dataFiles) {
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * Load ORC file in iterative way.
+   */
+  @Override
+  public void write() throws IOException {
+if (this.dataFiles == null || this.dataFiles.length == 0) {
+  throw new RuntimeException("'withOrcPath()' must be called to support 
loading ORC files");
+}
+if (this.csvCarbonWriter == null) {
+  throw new RuntimeException("csv carbon writer can not be null");
+}
+Arrays.sort(this.dataFiles, Comparator.comparing(CarbonFile::getPath));
+for (CarbonFile dataFile : this.dataFiles) {
+  this.loadSingleFile(dataFile);
+}
+  }
+
+  private void loadSingleFile(CarbonFile file) throws IOException {
+orcReader = SDKUtil.buildOrcReader(file.getPath(), this.configuration);
+ObjectInspector objectInspector = orcReader.getObjectInspector();
+RecordReader recordReader = orcReader.rows();
+if (objectInspector instanceof StructObjectInspector) {
+  StructObjectInspector structObjectInspector =
+  (StructObjectInspector) orcReader.getObjectInspector();
+  while (recordReader.hasNext()) {
+Object record = recordReader.next(null); // to remove duplicacy.

Review comment:
   was looking at API documentation and references from internet. Info in 
the documentation is limited though. `recordReader.next()` takes previous 
record as arg. But we are always passing null ?





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478408315



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/ORCCarbonWriter.java
##
@@ -0,0 +1,173 @@
+/*
+ * 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.sdk.file;
+
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.sdk.file.utils.SDKUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
+import org.apache.hadoop.hive.ql.io.orc.Reader;
+import org.apache.hadoop.hive.ql.io.orc.RecordReader;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.Text;
+
+
+/**
+ * Implementation to write ORC rows in CSV format to carbondata file.
+ */
+public class ORCCarbonWriter extends CSVCarbonWriter {
+  private Configuration configuration;
+  private CSVCarbonWriter csvCarbonWriter = null;
+  private Reader orcReader = null;
+  private CarbonFile[] dataFiles;
+
+  ORCCarbonWriter(CSVCarbonWriter csvCarbonWriter, Configuration 
configuration) {
+this.csvCarbonWriter = csvCarbonWriter;
+this.configuration = configuration;
+  }
+
+  @Override
+  public void setDataFiles(CarbonFile[] dataFiles) {
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * Load ORC file in iterative way.
+   */
+  @Override
+  public void write() throws IOException {
+if (this.dataFiles == null || this.dataFiles.length == 0) {
+  throw new RuntimeException("'withOrcPath()' must be called to support 
loading ORC files");
+}
+if (this.csvCarbonWriter == null) {

Review comment:
   We shouldn't have created the writer instance at first place if 
`csvCarbonWriter` was null. 





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478395584



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/JsonCarbonWriter.java
##
@@ -91,4 +106,44 @@ public void close() throws IOException {
   throw new IOException(e);
 }
   }
+
+  private void loadSingleFile(CarbonFile file) throws IOException {
+Reader reader = null;
+try {
+  reader = SDKUtil.buildJsonReader(file, configuration);
+  JSONParser jsonParser = new JSONParser();
+  Object jsonRecord = jsonParser.parse(reader);
+  if (jsonRecord instanceof JSONArray) {
+JSONArray jsonArray = (JSONArray) jsonRecord;
+for (Object record : jsonArray) {
+  this.write(record.toString());
+}
+  } else {
+this.write(jsonRecord.toString());
+  }
+} catch (Exception e) {

Review comment:
   Good to use specific exceptions wherever possible instead of generic 
exception





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478395584



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/JsonCarbonWriter.java
##
@@ -91,4 +106,44 @@ public void close() throws IOException {
   throw new IOException(e);
 }
   }
+
+  private void loadSingleFile(CarbonFile file) throws IOException {
+Reader reader = null;
+try {
+  reader = SDKUtil.buildJsonReader(file, configuration);
+  JSONParser jsonParser = new JSONParser();
+  Object jsonRecord = jsonParser.parse(reader);
+  if (jsonRecord instanceof JSONArray) {
+JSONArray jsonArray = (JSONArray) jsonRecord;
+for (Object record : jsonArray) {
+  this.write(record.toString());
+}
+  } else {
+this.write(jsonRecord.toString());
+  }
+} catch (Exception e) {

Review comment:
   Use specific exceptions instead of generic exception





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478387596



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/JsonCarbonWriter.java
##
@@ -91,4 +102,39 @@ public void close() throws IOException {
   throw new IOException(e);
 }
   }
+
+  private void loadSingleFile(File file) throws IOException {
+try {
+  Reader reader = SDKUtil.buildJsonReader(file);
+  JSONParser jsonParser = new JSONParser();
+  Object jsonRecord = jsonParser.parse(reader);
+  if (jsonRecord instanceof JSONArray) {
+JSONArray jsonArray = (JSONArray) jsonRecord;
+for (Object record : jsonArray) {
+  this.write(record.toString());
+}
+  } else {
+this.write(jsonRecord.toString());
+  }
+} catch (Exception e) {
+  e.printStackTrace();
+  throw new IOException(e.getMessage());

Review comment:
   Need to close stream in validateJsonFiles as well.  Please check for all 
the stream cases in this PR.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478378839



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -594,6 +607,446 @@ public CarbonWriterBuilder withJsonInput(Schema 
carbonSchema) {
 return this;
   }
 
+  private void validateCsvFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.CSV_FILE_EXTENSION);
+if (CollectionUtils.isEmpty(Arrays.asList(dataFiles))) {
+  throw new RuntimeException("CSV files can't be empty.");
+}
+for (CarbonFile dataFile : dataFiles) {
+  try {
+CsvParser csvParser = SDKUtil.buildCsvParser(this.hadoopConf);
+
csvParser.beginParsing(FileFactory.getDataInputStream(dataFile.getPath(),
+-1, this.hadoopConf));
+  } catch (IllegalArgumentException ex) {
+if (ex.getCause() instanceof FileNotFoundException) {
+  throw new FileNotFoundException("File " + dataFile +
+  " not found to build carbon writer.");
+}
+throw ex;
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading CSV files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withCsvInput();
+this.validateCsvFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts CSV files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the CSV file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withCsvPath(filePath);
+return this;
+  }
+
+  private void validateJsonFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.JSON_FILE_EXTENSION);
+for (CarbonFile dataFile : dataFiles) {
+  try {
+new JSONParser().parse(SDKUtil.buildJsonReader(dataFile, 
this.hadoopConf));
+  } catch (FileNotFoundException ex) {
+throw new FileNotFoundException("File " + dataFile + " not found to 
build carbon writer.");
+  } catch (ParseException ex) {
+throw new RuntimeException("File " + dataFile + " is not in json 
format.");
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading JSON files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withJsonInput();
+this.validateJsonFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts JSON file directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the json file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   * @throws IOException
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withJsonPath(filePath);
+return this;
+  }
+
+  private void validateFilePath(String filePath) {
+if (StringUtils.isEmpty(filePath)) {
+  throw new IllegalArgumentException("filePath can not be empty");
+}
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading Parquet files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withParquetPath(String filePath) throws 
IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.writerType = WRITER_TYPE.PARQUET;
+this.validateParquetFiles();
+return this;
+  }
+
+  private void setIsDirectory(String filePath) {
+if (this.hadoopConf == null) {
+  this.hadoopConf = new Configuration(FileFactory.getConfiguration());
+}
+CarbonFile carbonFile = FileFactory.getCarbonFile(filePath, hadoopConf);
+this.isDirectory = carbonFile.isDirectory();
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts parquet files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the parquet file exists.
+   * @param fileList list of files which has to be 

[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478375043



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -594,6 +607,446 @@ public CarbonWriterBuilder withJsonInput(Schema 
carbonSchema) {
 return this;
   }
 
+  private void validateCsvFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.CSV_FILE_EXTENSION);
+if (CollectionUtils.isEmpty(Arrays.asList(dataFiles))) {
+  throw new RuntimeException("CSV files can't be empty.");
+}
+for (CarbonFile dataFile : dataFiles) {
+  try {
+CsvParser csvParser = SDKUtil.buildCsvParser(this.hadoopConf);
+
csvParser.beginParsing(FileFactory.getDataInputStream(dataFile.getPath(),
+-1, this.hadoopConf));
+  } catch (IllegalArgumentException ex) {
+if (ex.getCause() instanceof FileNotFoundException) {
+  throw new FileNotFoundException("File " + dataFile +
+  " not found to build carbon writer.");
+}
+throw ex;
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading CSV files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withCsvInput();
+this.validateCsvFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts CSV files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the CSV file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withCsvPath(filePath);
+return this;
+  }
+
+  private void validateJsonFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.JSON_FILE_EXTENSION);
+for (CarbonFile dataFile : dataFiles) {
+  try {
+new JSONParser().parse(SDKUtil.buildJsonReader(dataFile, 
this.hadoopConf));
+  } catch (FileNotFoundException ex) {
+throw new FileNotFoundException("File " + dataFile + " not found to 
build carbon writer.");
+  } catch (ParseException ex) {
+throw new RuntimeException("File " + dataFile + " is not in json 
format.");
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading JSON files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withJsonInput();
+this.validateJsonFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts JSON file directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the json file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   * @throws IOException
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withJsonPath(filePath);
+return this;
+  }
+
+  private void validateFilePath(String filePath) {
+if (StringUtils.isEmpty(filePath)) {
+  throw new IllegalArgumentException("filePath can not be empty");
+}
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading Parquet files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withParquetPath(String filePath) throws 
IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.writerType = WRITER_TYPE.PARQUET;
+this.validateParquetFiles();
+return this;
+  }
+
+  private void setIsDirectory(String filePath) {
+if (this.hadoopConf == null) {
+  this.hadoopConf = new Configuration(FileFactory.getConfiguration());
+}
+CarbonFile carbonFile = FileFactory.getCarbonFile(filePath, hadoopConf);
+this.isDirectory = carbonFile.isDirectory();
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts parquet files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the parquet file exists.
+   * @param fileList list of files which has to be 

[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478354030



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -594,6 +607,446 @@ public CarbonWriterBuilder withJsonInput(Schema 
carbonSchema) {
 return this;
   }
 
+  private void validateCsvFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.CSV_FILE_EXTENSION);
+if (CollectionUtils.isEmpty(Arrays.asList(dataFiles))) {
+  throw new RuntimeException("CSV files can't be empty.");
+}
+for (CarbonFile dataFile : dataFiles) {
+  try {
+CsvParser csvParser = SDKUtil.buildCsvParser(this.hadoopConf);
+
csvParser.beginParsing(FileFactory.getDataInputStream(dataFile.getPath(),
+-1, this.hadoopConf));
+  } catch (IllegalArgumentException ex) {
+if (ex.getCause() instanceof FileNotFoundException) {
+  throw new FileNotFoundException("File " + dataFile +
+  " not found to build carbon writer.");
+}
+throw ex;
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading CSV files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withCsvInput();
+this.validateCsvFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts CSV files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the CSV file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withCsvPath(filePath);
+return this;
+  }
+
+  private void validateJsonFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.JSON_FILE_EXTENSION);
+for (CarbonFile dataFile : dataFiles) {
+  try {
+new JSONParser().parse(SDKUtil.buildJsonReader(dataFile, 
this.hadoopConf));
+  } catch (FileNotFoundException ex) {
+throw new FileNotFoundException("File " + dataFile + " not found to 
build carbon writer.");
+  } catch (ParseException ex) {
+throw new RuntimeException("File " + dataFile + " is not in json 
format.");
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading JSON files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withJsonInput();
+this.validateJsonFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts JSON file directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the json file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   * @throws IOException
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withJsonPath(filePath);
+return this;
+  }
+
+  private void validateFilePath(String filePath) {
+if (StringUtils.isEmpty(filePath)) {
+  throw new IllegalArgumentException("filePath can not be empty");
+}
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading Parquet files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withParquetPath(String filePath) throws 
IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.writerType = WRITER_TYPE.PARQUET;
+this.validateParquetFiles();
+return this;
+  }
+
+  private void setIsDirectory(String filePath) {
+if (this.hadoopConf == null) {
+  this.hadoopConf = new Configuration(FileFactory.getConfiguration());
+}
+CarbonFile carbonFile = FileFactory.getCarbonFile(filePath, hadoopConf);
+this.isDirectory = carbonFile.isDirectory();
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts parquet files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the parquet file exists.
+   * @param fileList list of files which has to be 

[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478343705



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -660,13 +1113,42 @@ public CarbonWriter build() throws IOException, 
InvalidLoadOptionException {
   // removed from the load. LoadWithoutConverter flag is going to point to 
the Loader Builder
   // which will skip Conversion Step.
   loadModel.setLoadWithoutConverterStep(true);
-  return new AvroCarbonWriter(loadModel, hadoopConf, this.avroSchema);
+  AvroCarbonWriter avroCarbonWriter = new AvroCarbonWriter(loadModel,
+  hadoopConf, this.avroSchema);
+  if (!StringUtils.isEmpty(filePath)) {

Review comment:
   This condition never seem to fail ?? Same for json and csv writer case 
below.
   We do not have similar check for parquet and orc types below





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478343705



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -660,13 +1113,42 @@ public CarbonWriter build() throws IOException, 
InvalidLoadOptionException {
   // removed from the load. LoadWithoutConverter flag is going to point to 
the Loader Builder
   // which will skip Conversion Step.
   loadModel.setLoadWithoutConverterStep(true);
-  return new AvroCarbonWriter(loadModel, hadoopConf, this.avroSchema);
+  AvroCarbonWriter avroCarbonWriter = new AvroCarbonWriter(loadModel,
+  hadoopConf, this.avroSchema);
+  if (!StringUtils.isEmpty(filePath)) {

Review comment:
   This condition never seem to fail ?? Same for json writer case below.
   We do not have similar check for parquet and orc types below





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478333555



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -660,13 +1113,42 @@ public CarbonWriter build() throws IOException, 
InvalidLoadOptionException {
   // removed from the load. LoadWithoutConverter flag is going to point to 
the Loader Builder
   // which will skip Conversion Step.
   loadModel.setLoadWithoutConverterStep(true);
-  return new AvroCarbonWriter(loadModel, hadoopConf, this.avroSchema);
+  AvroCarbonWriter avroCarbonWriter = new AvroCarbonWriter(loadModel,
+  hadoopConf, this.avroSchema);
+  if (!StringUtils.isEmpty(filePath)) {
+avroCarbonWriter.setDataFiles(this.dataFiles);
+  }
+  return avroCarbonWriter;
 } else if (this.writerType == WRITER_TYPE.JSON) {
   loadModel.setJsonFileLoad(true);
-  return new JsonCarbonWriter(loadModel, hadoopConf);
+  JsonCarbonWriter jsonCarbonWriter = new JsonCarbonWriter(loadModel, 
hadoopConf);
+  if (!StringUtils.isEmpty(filePath)) {
+jsonCarbonWriter.setDataFiles(this.dataFiles);
+  }
+  return jsonCarbonWriter;
+} else if (this.writerType == WRITER_TYPE.PARQUET) {
+  loadModel.setLoadWithoutConverterStep(true);
+  AvroCarbonWriter avroCarbonWriter = new AvroCarbonWriter(loadModel,
+  hadoopConf, this.avroSchema);
+  ParquetCarbonWriter parquetCarbonWriter = new
+  ParquetCarbonWriter(avroCarbonWriter, hadoopConf);

Review comment:
   Instead of creating instance of `AvroCarbonWriter` and passing it to 
`ParquetCarbonWriter`, create `AvroCarbonWriter` instance internally within the 
constructor of `ParquetCarbonWriter` Suggest same for `ORCCarbonWriter` below





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478323052



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -594,6 +607,446 @@ public CarbonWriterBuilder withJsonInput(Schema 
carbonSchema) {
 return this;
   }
 
+  private void validateCsvFiles() throws IOException {

Review comment:
   These validate methods(validate AvroFiles, ParquetFiles, CsvFiles, 
JsonFiles, OrcFiles) can be in the respective type of carbon writers. Because 
the way validate methods are implemented are specific to the respective format. 
Validate methods get readers/parse. Very similar code is in respective writers.
   
   Also can the validation method be called from `CarbonWriterBuilder.build()` 
method based respecitve `writerType` if `dataFiles` are not null ? I think, it 
can be abstract method in `CarbonWriter` to set input files to read, that is 
implemented by all the writers. Writers can validate and set them.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478323052



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -594,6 +607,446 @@ public CarbonWriterBuilder withJsonInput(Schema 
carbonSchema) {
 return this;
   }
 
+  private void validateCsvFiles() throws IOException {

Review comment:
   These validate methods(validate AvroFiles, ParquetFiles, CsvFiles, 
JsonFiles, OrcFiles) can be in the respective type of carbon writers. Because 
the way validate methods are implemented are specific to the respective format. 
Validate methods get readers/parse. Very similar code is in respective writers.
   
   Also can the validation method be called from CarbonWriterBuilder.build() 
method based respecitve writerType if dataFiles are not null ? I think, it can 
be abstract method in CarbonWriter to set input files to read, that is 
implemented by all the writers. Writers can validate and set them.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478286002



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -17,25 +17,19 @@
 
 package org.apache.carbondata.sdk.file;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.UUID;
+import java.util.*;

Review comment:
   wildcard import. same as above





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478286002



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -17,25 +17,19 @@
 
 package org.apache.carbondata.sdk.file;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.UUID;
+import java.util.*;

Review comment:
   same as above





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478284566



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java
##
@@ -72,6 +93,36 @@ public void write(Object object) throws IOException {
 }
   }
 
+  /**
+   * Load data of all or selected csv files at given location iteratively.
+   *
+   * @throws IOException
+   */
+  @Override
+  public void write() throws IOException {
+if (this.dataFiles == null || this.dataFiles.length == 0) {
+  throw new RuntimeException("'withCsvPath()' must be called to support 
load files");
+}
+this.csvParser = SDKUtil.buildCsvParser(this.configuration);
+Arrays.sort(this.dataFiles, Comparator.comparing(CarbonFile::getPath));
+for (CarbonFile dataFile : this.dataFiles) {
+  this.loadSingleFile(dataFile);
+}
+  }
+
+  private void loadSingleFile(CarbonFile file) throws IOException {
+this.csvParser.beginParsing(FileFactory.getDataInputStream(file.getPath(), 
-1, configuration));

Review comment:
   InputStream is not closed. same as above.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478195952



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
##
@@ -823,6 +829,31 @@ public void write(Object object) throws IOException {
 }
   }
 
+  /**
+   * Load data of all avro files at given location iteratively.
+   *
+   * @throws IOException
+   */
+  @Override
+  public void write() throws IOException {
+if (this.dataFiles == null || this.dataFiles.length == 0) {
+  throw new RuntimeException("'withAvroPath()' must be called to support 
loading avro files");
+}
+Arrays.sort(this.dataFiles, Comparator.comparing(CarbonFile::getPath));

Review comment:
   Is this sort required ? Same is applicable for other writers too.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478250989



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
##
@@ -823,6 +829,31 @@ public void write(Object object) throws IOException {
 }
   }
 
+  /**
+   * Load data of all avro files at given location iteratively.
+   *
+   * @throws IOException
+   */
+  @Override
+  public void write() throws IOException {
+if (this.dataFiles == null || this.dataFiles.length == 0) {
+  throw new RuntimeException("'withAvroPath()' must be called to support 
loading avro files");
+}
+Arrays.sort(this.dataFiles, Comparator.comparing(CarbonFile::getPath));
+for (CarbonFile dataFile : this.dataFiles) {
+  this.loadSingleFile(dataFile);
+}
+  }
+
+  private void loadSingleFile(CarbonFile file) throws IOException {
+DataFileStream avroReader = SDKUtil
+.buildAvroReader(file, this.configuration);

Review comment:
   `avroReader` stream is not closed neither in success case not in 
failure/exception cases. File InputStream used to create this DataFileStream in 
buildAvroReader is not closed. Check for all the stream reader/writer cases in 
this PR.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478227905



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/utils/SDKUtil.java
##
@@ -79,4 +98,75 @@ public static ArrayList listFiles(String sourceImageFolder,
 return (Object[]) input[i];
   }
 
+  public static List extractFilesFromFolder(String path,
+  String suf, Configuration hadoopConf) {
+List dataFiles = listFiles(path, suf, hadoopConf);
+List carbonFiles = new ArrayList<>();
+for (Object dataFile: dataFiles) {
+  carbonFiles.add(FileFactory.getCarbonFile(dataFile.toString(), 
hadoopConf));
+}
+if (CollectionUtils.isEmpty(dataFiles)) {
+  throw new RuntimeException("No file found at given location. Please 
provide" +
+  "the correct folder location.");
+}
+return carbonFiles;
+  }
+
+  public static DataFileStream buildAvroReader(CarbonFile 
carbonFile,
+   Configuration configuration) throws IOException {
+try {
+  GenericDatumReader genericDatumReader =
+  new GenericDatumReader<>();
+  DataFileStream avroReader =
+  new 
DataFileStream<>(FileFactory.getDataInputStream(carbonFile.getPath(),
+  -1, configuration), genericDatumReader);
+  return avroReader;
+} catch (FileNotFoundException ex) {
+  throw new FileNotFoundException("File " + carbonFile.getPath()
+  + " not found to build carbon writer.");
+} catch (IOException ex) {
+  if (ex.getMessage().contains("Not a data file")) {

Review comment:
   Why catch `IOException` and rethrow as `RuntimeException` here ? You 
converted checked exception to uncheked exception and consumed original 
exception completely. Better to preserve the orignal exception as there is no 
action after catching it except for the different exception message?





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478195952



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
##
@@ -823,6 +829,31 @@ public void write(Object object) throws IOException {
 }
   }
 
+  /**
+   * Load data of all avro files at given location iteratively.
+   *
+   * @throws IOException
+   */
+  @Override
+  public void write() throws IOException {
+if (this.dataFiles == null || this.dataFiles.length == 0) {
+  throw new RuntimeException("'withAvroPath()' must be called to support 
loading avro files");
+}
+Arrays.sort(this.dataFiles, Comparator.comparing(CarbonFile::getPath));

Review comment:
   Is this sort required ?





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r478194800



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
##
@@ -25,17 +25,12 @@
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.UUID;
+import java.util.*;

Review comment:
   Please do not use wildcard import.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-27 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r472407770



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -594,6 +607,446 @@ public CarbonWriterBuilder withJsonInput(Schema 
carbonSchema) {
 return this;
   }
 
+  private void validateCsvFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.CSV_FILE_EXTENSION);
+if (CollectionUtils.isEmpty(Arrays.asList(dataFiles))) {
+  throw new RuntimeException("CSV files can't be empty.");
+}
+for (CarbonFile dataFile : dataFiles) {
+  try {
+CsvParser csvParser = SDKUtil.buildCsvParser(this.hadoopConf);
+
csvParser.beginParsing(FileFactory.getDataInputStream(dataFile.getPath(),
+-1, this.hadoopConf));
+  } catch (IllegalArgumentException ex) {
+if (ex.getCause() instanceof FileNotFoundException) {
+  throw new FileNotFoundException("File " + dataFile +
+  " not found to build carbon writer.");
+}
+throw ex;
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading CSV files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withCsvInput();
+this.validateCsvFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts CSV files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the CSV file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withCsvPath(filePath);
+return this;
+  }
+
+  private void validateJsonFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.JSON_FILE_EXTENSION);
+for (CarbonFile dataFile : dataFiles) {
+  try {
+new JSONParser().parse(SDKUtil.buildJsonReader(dataFile, 
this.hadoopConf));
+  } catch (FileNotFoundException ex) {
+throw new FileNotFoundException("File " + dataFile + " not found to 
build carbon writer.");
+  } catch (ParseException ex) {
+throw new RuntimeException("File " + dataFile + " is not in json 
format.");
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading JSON files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withJsonInput();
+this.validateJsonFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts JSON file directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the json file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   * @throws IOException
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withJsonPath(filePath);
+return this;
+  }
+
+  private void validateFilePath(String filePath) {
+if (StringUtils.isEmpty(filePath)) {
+  throw new IllegalArgumentException("filePath can not be empty");
+}
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading Parquet files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withParquetPath(String filePath) throws 
IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.writerType = WRITER_TYPE.PARQUET;
+this.validateParquetFiles();
+return this;
+  }
+
+  private void setIsDirectory(String filePath) {
+if (this.hadoopConf == null) {
+  this.hadoopConf = new Configuration(FileFactory.getConfiguration());

Review comment:
   Had checked the base code. In the base code, we seem to directly assign 
the return value of FileFactory.getConfiguration() instead of new 
Configuration. Suggest to check and keep it consistent. Check for all the 
places in this PR.





This is an automated message from the Apache Git Service.
To respond 

[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-18 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r472408878



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/utils/SDKUtil.java
##
@@ -79,4 +98,75 @@ public static ArrayList listFiles(String sourceImageFolder,
 return (Object[]) input[i];
   }
 
+  public static List extractFilesFromFolder(String path,
+  String suf, Configuration hadoopConf) {
+List dataFiles = listFiles(path, suf, hadoopConf);
+List carbonFiles = new ArrayList<>();
+for (Object dataFile: dataFiles) {
+  carbonFiles.add(FileFactory.getCarbonFile(dataFile.toString(), 
hadoopConf));
+}
+if (CollectionUtils.isEmpty(dataFiles)) {
+  throw new RuntimeException("No file found at given location. Please 
provide" +
+  "the correct folder location.");
+}
+return carbonFiles;
+  }
+
+  public static DataFileStream buildAvroReader(CarbonFile 
carbonFile,
+   Configuration configuration) throws IOException {
+try {
+  GenericDatumReader genericDatumReader =
+  new GenericDatumReader<>();
+  DataFileStream avroReader =
+  new 
DataFileStream<>(FileFactory.getDataInputStream(carbonFile.getPath(),
+  -1, configuration), genericDatumReader);
+  return avroReader;
+} catch (FileNotFoundException ex) {
+  throw new FileNotFoundException("File " + carbonFile.getPath()
+  + " not found to build carbon writer.");
+} catch (IOException ex) {
+  if (ex.getMessage().contains("Not a data file")) {
+throw new RuntimeException("File " + carbonFile.getPath() + " is not 
in avro format.");
+  } else {
+throw ex;
+  }
+}
+  }
+
+  public static Reader buildOrcReader(String path, Configuration conf) throws 
IOException {
+try {
+  Reader orcReader = OrcFile.createReader(new Path(path),
+  OrcFile.readerOptions(conf));
+  return orcReader;
+} catch (FileFormatException ex) {
+  throw new RuntimeException("File " + path + " is not in ORC format");
+} catch (FileNotFoundException ex) {
+  throw new FileNotFoundException("File " + path + " not found to build 
carbon writer.");
+}
+  }
+
+  public static ParquetReader buildPqrquetReader(String path, 
Configuration conf)

Review comment:
   Please correct spelling mistake for parquet in method name.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-18 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r472407770



##
File path: 
sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
##
@@ -594,6 +607,446 @@ public CarbonWriterBuilder withJsonInput(Schema 
carbonSchema) {
 return this;
   }
 
+  private void validateCsvFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.CSV_FILE_EXTENSION);
+if (CollectionUtils.isEmpty(Arrays.asList(dataFiles))) {
+  throw new RuntimeException("CSV files can't be empty.");
+}
+for (CarbonFile dataFile : dataFiles) {
+  try {
+CsvParser csvParser = SDKUtil.buildCsvParser(this.hadoopConf);
+
csvParser.beginParsing(FileFactory.getDataInputStream(dataFile.getPath(),
+-1, this.hadoopConf));
+  } catch (IllegalArgumentException ex) {
+if (ex.getCause() instanceof FileNotFoundException) {
+  throw new FileNotFoundException("File " + dataFile +
+  " not found to build carbon writer.");
+}
+throw ex;
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading CSV files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withCsvInput();
+this.validateCsvFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts CSV files directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the CSV file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withCsvPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withCsvPath(filePath);
+return this;
+  }
+
+  private void validateJsonFiles() throws IOException {
+CarbonFile[] dataFiles = 
this.extractDataFiles(CarbonCommonConstants.JSON_FILE_EXTENSION);
+for (CarbonFile dataFile : dataFiles) {
+  try {
+new JSONParser().parse(SDKUtil.buildJsonReader(dataFile, 
this.hadoopConf));
+  } catch (FileNotFoundException ex) {
+throw new FileNotFoundException("File " + dataFile + " not found to 
build carbon writer.");
+  } catch (ParseException ex) {
+throw new RuntimeException("File " + dataFile + " is not in json 
format.");
+  }
+}
+this.dataFiles = dataFiles;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading JSON files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath) throws IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.withJsonInput();
+this.validateJsonFiles();
+return this;
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts JSON file directory and
+   * list of file which has to be loaded.
+   *
+   * @param filePath directory where the json file exists.
+   * @param fileList list of files which has to be loaded.
+   * @return CarbonWriterBuilder
+   * @throws IOException
+   */
+  public CarbonWriterBuilder withJsonPath(String filePath, List 
fileList)
+  throws IOException {
+this.fileList = fileList;
+this.withJsonPath(filePath);
+return this;
+  }
+
+  private void validateFilePath(String filePath) {
+if (StringUtils.isEmpty(filePath)) {
+  throw new IllegalArgumentException("filePath can not be empty");
+}
+  }
+
+  /**
+   * to build a {@link CarbonWriter}, which accepts loading Parquet files.
+   *
+   * @param filePath absolute path under which files should be loaded.
+   * @return CarbonWriterBuilder
+   */
+  public CarbonWriterBuilder withParquetPath(String filePath) throws 
IOException {
+this.validateFilePath(filePath);
+this.filePath = filePath;
+this.setIsDirectory(filePath);
+this.writerType = WRITER_TYPE.PARQUET;
+this.validateParquetFiles();
+return this;
+  }
+
+  private void setIsDirectory(String filePath) {
+if (this.hadoopConf == null) {
+  this.hadoopConf = new Configuration(FileFactory.getConfiguration());

Review comment:
   Had checked the base code. In the base code, we seem to directly assign 
the return value of FileFactory.getConfiguration() instead of new 
Configuration. Suggest to check and keep it consistent.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to 

[GitHub] [carbondata] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-18 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r472388572



##
File path: examples/spark/pom.xml
##
@@ -38,6 +38,12 @@
   org.apache.carbondata
   carbondata-spark_${spark.binary.version}
   ${project.version}
+  

Review comment:
   Was wondering why this exclusion in examples/spark/pom.xml & 
integration/spark/pom.xml . You don't seem to have any change in these 2 
modules. I think, you want to exclude elsewhere ?





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-18 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r472379038



##
File path: 
core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##
@@ -2456,4 +2471,24 @@ private CarbonCommonConstants() {
* property which defines the insert stage flow
*/
   public static final String IS_INSERT_STAGE = "is_insert_stage";
+
+  /**
+   * the level 1 complex delimiter default value
+   */
+  @CarbonProperty

Review comment:
   This looks to be just a value. Not the user configuration property. If 
so, @CarbonProperty is not required. please check and remove. check the same 
for below 2 more properties as well.





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] VenuReddy2103 commented on a change in pull request #3819: [CARBONDATA-3855]support carbon SDK to load data from different files

2020-08-18 Thread GitBox


VenuReddy2103 commented on a change in pull request #3819:
URL: https://github.com/apache/carbondata/pull/3819#discussion_r472379038



##
File path: 
core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##
@@ -2456,4 +2471,24 @@ private CarbonCommonConstants() {
* property which defines the insert stage flow
*/
   public static final String IS_INSERT_STAGE = "is_insert_stage";
+
+  /**
+   * the level 1 complex delimiter default value
+   */
+  @CarbonProperty

Review comment:
   This looks to be just a value. Not the user configuration property. If 
so, @CarbonProperty is not required. please check and remove





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