[GitHub] [carbondata] Karan-c980 commented on a change in pull request #3834: [CARBONDATA-3865] Implementation of delete/update feature in carbondata SDK.

2020-09-09 Thread GitBox


Karan-c980 commented on a change in pull request #3834:
URL: https://github.com/apache/carbondata/pull/3834#discussion_r485548941



##
File path: sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonIUD.java
##
@@ -0,0 +1,376 @@
+/*
+ * 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.FilenameFilter;
+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.Set;
+
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.Field;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.LiteralExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
+import org.apache.carbondata.core.scan.expression.logical.OrExpression;
+import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat;
+import org.apache.carbondata.hadoop.internal.ObjectArrayWritable;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.RecordWriter;
+
+public class CarbonIUD {
+
+  private final Map>> 
filterColumnToValueMappingForDelete;
+  private final Map>> 
filterColumnToValueMappingForUpdate;
+  private final Map> updateColumnToValueMapping;
+
+  private CarbonIUD() {
+filterColumnToValueMappingForDelete = new HashMap<>();
+filterColumnToValueMappingForUpdate = new HashMap<>();
+updateColumnToValueMapping = new HashMap<>();
+  }
+
+  /**
+   * @return CarbonIUD object
+   */
+  public static CarbonIUD getInstance() {

Review comment:
   Done

##
File path: sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonIUD.java
##
@@ -0,0 +1,376 @@
+/*
+ * 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.FilenameFilter;
+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.Set;
+
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.Field;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.LiteralExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
+import org.apache.carbondata.core.scan.expression.logical.OrExpression;
+import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat;
+import org.apache.carbondata.hadoop.internal.ObjectArrayWritable;
+
+import org.apache.hadoop.io.NullWritable;
+import 

[GitHub] [carbondata] Karan-c980 commented on a change in pull request #3834: [CARBONDATA-3865] Implementation of delete/update feature in carbondata SDK.

2020-09-09 Thread GitBox


Karan-c980 commented on a change in pull request #3834:
URL: https://github.com/apache/carbondata/pull/3834#discussion_r485548849



##
File path: sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonIUD.java
##
@@ -0,0 +1,376 @@
+/*
+ * 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.FilenameFilter;
+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.Set;
+
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.Field;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.LiteralExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
+import org.apache.carbondata.core.scan.expression.logical.OrExpression;
+import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat;
+import org.apache.carbondata.hadoop.internal.ObjectArrayWritable;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.RecordWriter;
+
+public class CarbonIUD {
+
+  private final Map>> 
filterColumnToValueMappingForDelete;
+  private final Map>> 
filterColumnToValueMappingForUpdate;
+  private final Map> updateColumnToValueMapping;
+
+  private CarbonIUD() {
+filterColumnToValueMappingForDelete = new HashMap<>();
+filterColumnToValueMappingForUpdate = new HashMap<>();
+updateColumnToValueMapping = new HashMap<>();
+  }
+
+  /**
+   * @return CarbonIUD object
+   */
+  public static CarbonIUD getInstance() {
+return new CarbonIUD();
+  }
+
+  /**
+   * @param path   is the table path on which delete is performed
+   * @param column is the columnName on which records have to be deleted
+   * @param value  of column on which the records have to be deleted
+   * @return CarbonIUD object
+   */
+  public CarbonIUD delete(String path, String column, String value) {
+prepareDelete(path, column, value, filterColumnToValueMappingForDelete);
+return this;
+  }
+
+  /**
+   * This method deletes the rows at given path by applying the 
filterExpression
+   *
+   * @param path is the table path on which delete is performed
+   * @param filterExpression is the expression to delete the records
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void delete(String path, Expression filterExpression)
+  throws IOException, InterruptedException {
+CarbonReader reader = CarbonReader.builder(path)
+.projection(new String[] { 
CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID })
+.filter(filterExpression).build();
+
+RecordWriter deleteDeltaWriter =
+CarbonTableOutputFormat.getDeleteDeltaRecordWriter(path);
+ObjectArrayWritable writable = new ObjectArrayWritable();
+while (reader.hasNext()) {
+  Object[] row = (Object[]) reader.readNextRow();
+  writable.set(row);
+  deleteDeltaWriter.write(NullWritable.get(), writable);
+}
+deleteDeltaWriter.close(null);
+reader.close();
+  }
+
+  /**
+   * Calling this method will start the execution of delete process
+   *
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void closeDelete() throws IOException, InterruptedException {
+for (Map.Entry>> path : 
this.filterColumnToValueMappingForDelete
+.entrySet()) {
+  deleteExecution(path.getKey());
+}
+  }
+
+  /**
+   * @param path  is the table path on which update is performed
+   * @param columnis the columnName on which records have to be updated
+   * @param value of column on which the records have to be updated
+   * @param updColumn is the name of updatedColumn
+   * @param updValue  is the value of 

[GitHub] [carbondata] Karan-c980 commented on a change in pull request #3834: [CARBONDATA-3865] Implementation of delete/update feature in carbondata SDK.

2020-09-08 Thread GitBox


Karan-c980 commented on a change in pull request #3834:
URL: https://github.com/apache/carbondata/pull/3834#discussion_r485093341



##
File path: README.md
##
@@ -100,3 +100,4 @@ To get involved in CarbonData:
 ## About
 Apache CarbonData is an open source project of The Apache Software Foundation 
(ASF).
 
+## PR

Review comment:
   Removed





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

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




[GitHub] [carbondata] Karan-c980 commented on a change in pull request #3834: [CARBONDATA-3865] Implementation of delete/update feature in carbondata SDK.

2020-09-07 Thread GitBox


Karan-c980 commented on a change in pull request #3834:
URL: https://github.com/apache/carbondata/pull/3834#discussion_r484539275



##
File path: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
##
@@ -260,7 +265,50 @@ public boolean accept(CarbonFile file) {
 getDataBlocksOfSegment(job, carbonTable, indexFilter, validSegments,
 new ArrayList(), new ArrayList());
 numBlocks = dataBlocksOfSegment.size();
-result.addAll(dataBlocksOfSegment);
-return result;
+List allDeleteDeltaFiles = 
getAllDeleteDeltaFiles(carbonTable.getTablePath());
+if(allDeleteDeltaFiles.size() > 0) {
+  for (CarbonInputSplit split : dataBlocksOfSegment) {
+split.setDeleteDeltaFiles(getDeleteDeltaFiles(split.getFilePath(), 
allDeleteDeltaFiles));
+  }
+}
+return new LinkedList<>(dataBlocksOfSegment);
+  }
+
+  private List getAllDeleteDeltaFiles(String path) {
+List deltaFiles = null;
+try (Stream walk = Files.walk(Paths.get(path))) {
+  deltaFiles = walk.map(x -> x.toString())
+  .filter(f -> f.endsWith(CarbonCommonConstants.DELETE_DELTA_FILE_EXT))
+  .collect(Collectors.toList());
+} catch (IOException e) {
+  e.printStackTrace();
+}
+return deltaFiles;
+  }
+
+  private String[] getDeleteDeltaFiles(String segmentPath, List 
allDeleteDeltaFiles) {
+ArrayList deleteDeltaFiles = new ArrayList<>();
+String[] pathElements = 
segmentPath.split(CarbonCommonConstants.FILE_SEPARATOR);
+String segmentFileName = pathElements[pathElements.length - 1];
+String ExpectedDeleteDeltaFileName = segmentFileName
+.substring(segmentFileName.indexOf(CarbonCommonConstants.HYPHEN) + 1,
+segmentFileName.indexOf(CarbonCommonConstants.UNDERSCORE));
+
+for (String deltaFile : allDeleteDeltaFiles) {
+  String[] deleteDeltapathElements =
+  deltaFile.split(CarbonCommonConstants.FILE_SEPARATOR);
+  String deleteDeltaFullFileName = 
deleteDeltapathElements[deleteDeltapathElements.length - 1];
+  String deleteDeltaFileName = deleteDeltaFullFileName
+  .substring(0, 
deleteDeltaFullFileName.indexOf(CarbonCommonConstants.UNDERSCORE));

Review comment:
   Handled

##
File path: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
##
@@ -559,4 +564,50 @@ public void close(TaskAttemptContext taskAttemptContext) 
throws InterruptedExcep
   super.close(taskAttemptContext);
 }
   }
+
+  public static RecordWriter 
getDeleteDeltaRecordWriter(String path) {
+return (new RecordWriter() {
+  private final ArrayList tupleId = new ArrayList<>();
+
+  @Override
+  public void write(NullWritable aVoid, ObjectArrayWritable objects) {
+this.tupleId.add((String) objects.get()[0]);
+  }
+
+  @Override
+  public void close(TaskAttemptContext taskAttemptContext) throws 
IOException {
+Map blockToDeleteDeltaBlockMapping = 
new HashMap<>();
+DeleteDeltaBlockDetails blockDetails;
+String blockName;
+for (String tuple : tupleId) {
+  blockName = CarbonUpdateUtil.getBlockName(
+  
(tuple.split(CarbonCommonConstants.FILE_SEPARATOR)[TupleIdEnum.BLOCK_ID

Review comment:
   Done

##
File path: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
##
@@ -559,4 +564,50 @@ public void close(TaskAttemptContext taskAttemptContext) 
throws InterruptedExcep
   super.close(taskAttemptContext);
 }
   }
+
+  public static RecordWriter 
getDeleteDeltaRecordWriter(String path) {
+return (new RecordWriter() {
+  private final ArrayList tupleId = new ArrayList<>();
+
+  @Override
+  public void write(NullWritable aVoid, ObjectArrayWritable objects) {
+this.tupleId.add((String) objects.get()[0]);
+  }
+
+  @Override
+  public void close(TaskAttemptContext taskAttemptContext) throws 
IOException {
+Map blockToDeleteDeltaBlockMapping = 
new HashMap<>();
+DeleteDeltaBlockDetails blockDetails;
+String blockName;
+for (String tuple : tupleId) {
+  blockName = CarbonUpdateUtil.getBlockName(
+  
(tuple.split(CarbonCommonConstants.FILE_SEPARATOR)[TupleIdEnum.BLOCK_ID
+  .getTupleIdIndex()]));
+
+  if (!blockToDeleteDeltaBlockMapping.containsKey(blockName)) {
+blockDetails = new DeleteDeltaBlockDetails(blockName);
+blockToDeleteDeltaBlockMapping.put(blockName, blockDetails);
+  }
+  blockDetails = blockToDeleteDeltaBlockMapping.get(blockName);
+  try {
+blockDetails.addBlocklet(
+CarbonUpdateUtil.getRequiredFieldFromTID(tuple, 
TupleIdEnum.BLOCKLET_ID),
+CarbonUpdateUtil.getRequiredFieldFromTID(tuple, 
TupleIdEnum.OFFSET), Integer
+.parseInt(
+

[GitHub] [carbondata] Karan-c980 commented on a change in pull request #3834: [CARBONDATA-3865] Implementation of delete/update feature in carbondata SDK.

2020-09-07 Thread GitBox


Karan-c980 commented on a change in pull request #3834:
URL: https://github.com/apache/carbondata/pull/3834#discussion_r484539395



##
File path: sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonIUD.java
##
@@ -0,0 +1,275 @@
+package org.apache.carbondata.sdk.file;
+
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.Field;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.LiteralExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
+import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat;
+import org.apache.carbondata.hadoop.internal.ObjectArrayWritable;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.jcraft.jsch.IO;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+public class CarbonIUD {
+
+  private final Map> filterColumnToValueMapping;
+  private final Map> updateColumnToValueMapping;
+
+  private CarbonIUD() {
+filterColumnToValueMapping = new HashMap<>();
+updateColumnToValueMapping = new HashMap<>();
+  }
+
+  /**
+   * @return CarbonIUD object
+   */
+  public static CarbonIUD getInstance() {
+return new CarbonIUD();
+  }
+
+  /**
+   * @param path   is the segment path on which delete is performed
+   * @param column is the columeName on which records have to be deleted
+   * @param value  of column on which the records have to be deleted
+   * @return CarbonIUD object
+   * 
+   * for eg: DELETE WHERE column = value
+   */
+  public CarbonIUD delete(String path, String column, String value) {
+prepareDelete(path, column, value);
+return this;
+  }
+
+  /**
+   * @param path is the segment path on which delete is performed
+   * @param filterExpression is the expression to delete the records
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void delete(String path, Expression filterExpression)
+  throws IOException, InterruptedException {
+CarbonReader reader = CarbonReader.builder(path, "_temp")
+.projection(new String[] { 
CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID })
+.filter(filterExpression).build();
+
+RecordWriter deleteDeltaWriter =
+CarbonTableOutputFormat.getDeleteDeltaRecordWriter(path);
+ObjectArrayWritable writable = new ObjectArrayWritable();
+while (reader.hasNext()) {
+  Object[] row = (Object[]) reader.readNextRow();
+  writable.set(row);
+  deleteDeltaWriter.write(NullWritable.get(), writable);
+}
+deleteDeltaWriter.close(null);
+reader.close();
+  }
+
+  /**
+   * Calling this method will start the execution of delete process
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void closeDelete() throws IOException, InterruptedException {
+for (Map.Entry> path : 
this.filterColumnToValueMapping.entrySet()) {
+  deleteExecution(path.getKey());
+}
+this.filterColumnToValueMapping.clear();
+  }
+
+  /**
+   * @param path  is the segment path on which update is performed
+   * @param columnis the columeName on which records have to be updated
+   * @param value of column on which the records have to be updated
+   * @param updColumn is the name of updatedColumn
+   * @param updValue  is the value of updatedColumn
+   * @return CarbonUID
+   * 
+   * for eg: UPDATE updColumn = updValue WHERE column = value
+   */
+  public CarbonIUD update(String path, String column, String value, String 
updColumn,
+  String updValue) {
+prepareUpdate(path, column, value, updColumn, updValue);
+return this;
+  }
+
+  /**
+   * @param pathis the segment path on which update is 
performed.
+   * @param filterExpressionis the expression object to update the 
records
+   * @param updatedColumnToValueMapping contains the mapping of updatedColumns 
to updatedValues
+   * @throws IOException
+   * @throws InterruptedException
+   * @throws InvalidLoadOptionException
+   */
+  public void update(String path, Expression filterExpression,
+  Map updatedColumnToValueMapping)
+  throws IOException, 

[GitHub] [carbondata] Karan-c980 commented on a change in pull request #3834: [CARBONDATA-3865] Implementation of delete/update feature in carbondata SDK.

2020-09-07 Thread GitBox


Karan-c980 commented on a change in pull request #3834:
URL: https://github.com/apache/carbondata/pull/3834#discussion_r484539366



##
File path: sdk/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonIUD.java
##
@@ -0,0 +1,275 @@
+package org.apache.carbondata.sdk.file;
+
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.Field;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.LiteralExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
+import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat;
+import org.apache.carbondata.hadoop.internal.ObjectArrayWritable;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.jcraft.jsch.IO;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+public class CarbonIUD {
+
+  private final Map> filterColumnToValueMapping;
+  private final Map> updateColumnToValueMapping;
+
+  private CarbonIUD() {
+filterColumnToValueMapping = new HashMap<>();
+updateColumnToValueMapping = new HashMap<>();
+  }
+
+  /**
+   * @return CarbonIUD object
+   */
+  public static CarbonIUD getInstance() {
+return new CarbonIUD();
+  }
+
+  /**
+   * @param path   is the segment path on which delete is performed
+   * @param column is the columeName on which records have to be deleted
+   * @param value  of column on which the records have to be deleted
+   * @return CarbonIUD object
+   * 
+   * for eg: DELETE WHERE column = value
+   */
+  public CarbonIUD delete(String path, String column, String value) {
+prepareDelete(path, column, value);
+return this;
+  }
+
+  /**
+   * @param path is the segment path on which delete is performed
+   * @param filterExpression is the expression to delete the records
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void delete(String path, Expression filterExpression)
+  throws IOException, InterruptedException {
+CarbonReader reader = CarbonReader.builder(path, "_temp")

Review comment:
   Removed table 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] Karan-c980 commented on a change in pull request #3834: [CARBONDATA-3865] Implementation of delete/update feature in carbondata SDK.

2020-09-07 Thread GitBox


Karan-c980 commented on a change in pull request #3834:
URL: https://github.com/apache/carbondata/pull/3834#discussion_r484538733



##
File path: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
##
@@ -260,7 +265,50 @@ public boolean accept(CarbonFile file) {
 getDataBlocksOfSegment(job, carbonTable, indexFilter, validSegments,
 new ArrayList(), new ArrayList());
 numBlocks = dataBlocksOfSegment.size();
-result.addAll(dataBlocksOfSegment);
-return result;
+List allDeleteDeltaFiles = 
getAllDeleteDeltaFiles(carbonTable.getTablePath());
+if(allDeleteDeltaFiles.size() > 0) {

Review comment:
   Done

##
File path: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
##
@@ -260,7 +265,50 @@ public boolean accept(CarbonFile file) {
 getDataBlocksOfSegment(job, carbonTable, indexFilter, validSegments,
 new ArrayList(), new ArrayList());
 numBlocks = dataBlocksOfSegment.size();
-result.addAll(dataBlocksOfSegment);
-return result;
+List allDeleteDeltaFiles = 
getAllDeleteDeltaFiles(carbonTable.getTablePath());
+if(allDeleteDeltaFiles.size() > 0) {
+  for (CarbonInputSplit split : dataBlocksOfSegment) {
+split.setDeleteDeltaFiles(getDeleteDeltaFiles(split.getFilePath(), 
allDeleteDeltaFiles));
+  }
+}
+return new LinkedList<>(dataBlocksOfSegment);
+  }
+
+  private List getAllDeleteDeltaFiles(String path) {
+List deltaFiles = null;
+try (Stream walk = Files.walk(Paths.get(path))) {
+  deltaFiles = walk.map(x -> x.toString())
+  .filter(f -> f.endsWith(CarbonCommonConstants.DELETE_DELTA_FILE_EXT))
+  .collect(Collectors.toList());
+} catch (IOException e) {
+  e.printStackTrace();

Review comment:
   Done

##
File path: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
##
@@ -260,7 +265,50 @@ public boolean accept(CarbonFile file) {
 getDataBlocksOfSegment(job, carbonTable, indexFilter, validSegments,
 new ArrayList(), new ArrayList());
 numBlocks = dataBlocksOfSegment.size();
-result.addAll(dataBlocksOfSegment);
-return result;
+List allDeleteDeltaFiles = 
getAllDeleteDeltaFiles(carbonTable.getTablePath());
+if(allDeleteDeltaFiles.size() > 0) {
+  for (CarbonInputSplit split : dataBlocksOfSegment) {
+split.setDeleteDeltaFiles(getDeleteDeltaFiles(split.getFilePath(), 
allDeleteDeltaFiles));
+  }
+}
+return new LinkedList<>(dataBlocksOfSegment);
+  }
+
+  private List getAllDeleteDeltaFiles(String path) {
+List deltaFiles = null;
+try (Stream walk = Files.walk(Paths.get(path))) {
+  deltaFiles = walk.map(x -> x.toString())
+  .filter(f -> f.endsWith(CarbonCommonConstants.DELETE_DELTA_FILE_EXT))
+  .collect(Collectors.toList());
+} catch (IOException e) {
+  e.printStackTrace();
+}
+return deltaFiles;
+  }
+
+  private String[] getDeleteDeltaFiles(String segmentPath, List 
allDeleteDeltaFiles) {
+ArrayList deleteDeltaFiles = new ArrayList<>();
+String[] pathElements = 
segmentPath.split(CarbonCommonConstants.FILE_SEPARATOR);

Review comment:
   Done

##
File path: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
##
@@ -260,7 +265,50 @@ public boolean accept(CarbonFile file) {
 getDataBlocksOfSegment(job, carbonTable, indexFilter, validSegments,
 new ArrayList(), new ArrayList());
 numBlocks = dataBlocksOfSegment.size();
-result.addAll(dataBlocksOfSegment);
-return result;
+List allDeleteDeltaFiles = 
getAllDeleteDeltaFiles(carbonTable.getTablePath());
+if(allDeleteDeltaFiles.size() > 0) {
+  for (CarbonInputSplit split : dataBlocksOfSegment) {
+split.setDeleteDeltaFiles(getDeleteDeltaFiles(split.getFilePath(), 
allDeleteDeltaFiles));
+  }
+}
+return new LinkedList<>(dataBlocksOfSegment);
+  }
+
+  private List getAllDeleteDeltaFiles(String path) {
+List deltaFiles = null;
+try (Stream walk = Files.walk(Paths.get(path))) {
+  deltaFiles = walk.map(x -> x.toString())
+  .filter(f -> f.endsWith(CarbonCommonConstants.DELETE_DELTA_FILE_EXT))
+  .collect(Collectors.toList());
+} catch (IOException e) {
+  e.printStackTrace();
+}
+return deltaFiles;
+  }
+
+  private String[] getDeleteDeltaFiles(String segmentPath, List 
allDeleteDeltaFiles) {
+ArrayList deleteDeltaFiles = new ArrayList<>();
+String[] pathElements = 
segmentPath.split(CarbonCommonConstants.FILE_SEPARATOR);
+String segmentFileName = pathElements[pathElements.length - 1];

Review comment:
   Done

##
File path: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
##
@@ -260,7 +265,50 @@ public boolean accept(CarbonFile file) {
 

[GitHub] [carbondata] Karan-c980 commented on a change in pull request #3834: [CARBONDATA-3865] Implementation of delete/update feature in carbondata SDK.

2020-09-07 Thread GitBox


Karan-c980 commented on a change in pull request #3834:
URL: https://github.com/apache/carbondata/pull/3834#discussion_r484537947



##
File path: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
##
@@ -189,16 +196,14 @@ public CarbonTable getOrCreateCarbonTable(Configuration 
configuration) throws IO
 info.setBlockSize(carbonFile.getLength());
 info.setVersionNumber(split.getVersion().number());
 info.setUseMinMaxForPruning(false);
+if(allDeleteDeltaFiles.size() != 0) {

Review comment:
   Done





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

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