[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

classic Classic list List threaded Threaded
21 messages Options
12
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
GitHub user sounakr opened a pull request:

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

    [CARBONDATA-691] After Compaction records count are mismatched.

    **Problem** : After Compaction record count mismatches with actual count.
   
    **Analysis** :The Partitioning method of compaction was wrong. In getPartition method of CarbonScanRDD.scala supposed to make a list all the blocks of all the segments that needs to be merged and then make the partition based on taskNo.  Then each partitioned list is given to each executor. But currently after partitioning the complete list of blocks are being send to each executor for merging. As each executors merging all the blocks of all the segments, multiple executors doubles the data.
   
    **Fix** : Fix the getPartition method logic to process proper list of blocks to executors.
                 Fix Horizontal Partitioning which merged with IUD.  

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

    $ git pull https://github.com/sounakr/incubator-carbondata master

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

    https://github.com/apache/incubator-carbondata/pull/604.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #604
   
----
commit 4e5ea804a5ab36d79efdb4df425e729245e990ee
Author: sounakr <[hidden email]>
Date:   2017-02-17T14:42:39Z

    Compaction Partitioning changes

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #604: [CARBONDATA-691] After Compaction records c...

qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r102114630
 
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala ---
    @@ -255,77 +256,75 @@ class CarbonMergerRDD[K, V](
     
         // mapping of the node and block list.
         var nodeBlockMapping: java.util.Map[String, java.util.List[Distributable]] = new
    -            java.util.HashMap[String, java.util.List[Distributable]]
    +        java.util.HashMap[String, java.util.List[Distributable]]
     
         var noOfBlocks = 0
         val taskInfoList = new java.util.ArrayList[Distributable]
         var carbonInputSplits = mutable.Seq[CarbonInputSplit]()
     
    -    var blocksOfLastSegment: List[TableBlockInfo] = null
    +    var splitsOfLastSegment: List[CarbonInputSplit] = null
    +    // map for keeping the relation of a task and its blocks.
    +    val taskIdMapping: java.util.Map[String, java.util.List[CarbonInputSplit]] = new
    +        java.util.HashMap[String, java.util.List[CarbonInputSplit]]
    +
     
         // for each valid segment.
         for (eachSeg <- carbonMergerMapping.validSegments) {
    -      // map for keeping the relation of a task and its blocks.
    -      val taskIdMapping: java.util.Map[String, java.util.List[TableBlockInfo]] = new
    -            java.util.HashMap[String, java.util.List[TableBlockInfo]]
     
           // map for keeping the relation of a task and its blocks.
           job.getConfiguration.set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS, eachSeg)
     
    +      val updateDetails: UpdateVO = updateStatusManager.getInvalidTimestampRange(eachSeg)
    +      // val updateStatusManager = carbonLoadModel.getSegmentUpdateStatusManager
    --- End diff --
   
    remove commented code


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #604: [CARBONDATA-691] After Compaction records c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user jackylk commented on the issue:

    https://github.com/apache/incubator-carbondata/pull/604
 
    Is there a testcase to reproduce this bug and verify this PR?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r102224420
 
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java ---
    @@ -0,0 +1,129 @@
    +/*
    --- End diff --
   
    It seems some problem with commit, this class is shown as new class


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r102224519
 
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java ---
    @@ -0,0 +1,129 @@
    +/*
    + * 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.hadoop.util;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +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 org.apache.carbondata.core.datastore.block.Distributable;
    +import org.apache.carbondata.hadoop.CarbonInputSplit;
    +
    +public class CarbonInputSplitTaskInfo implements Distributable {
    +
    +  private final List<CarbonInputSplit> carbonBlockInfoList;
    +
    +  private final String taskId;
    +
    +  public String getTaskId() {
    +    return taskId;
    +  }
    +
    +  public List<CarbonInputSplit> getCarbonInputSplitList() {
    +    return carbonBlockInfoList;
    +  }
    +
    +  public CarbonInputSplitTaskInfo(String taskId, List<CarbonInputSplit> carbonSplitListInfo) {
    +    this.taskId = taskId;
    +    this.carbonBlockInfoList = carbonSplitListInfo;
    +  }
    +
    +  @Override public String[] getLocations() {
    +    Set<String> locations = new HashSet<String>();
    +    for (CarbonInputSplit splitInfo : carbonBlockInfoList) {
    +      try {
    +        locations.addAll(Arrays.asList(splitInfo.getLocations()));
    +      } catch (IOException e) {
    +        e.printStackTrace();
    --- End diff --
   
    Add logger or throw the exception


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r102224557
 
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java ---
    @@ -0,0 +1,129 @@
    +/*
    + * 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.hadoop.util;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +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 org.apache.carbondata.core.datastore.block.Distributable;
    +import org.apache.carbondata.hadoop.CarbonInputSplit;
    +
    +public class CarbonInputSplitTaskInfo implements Distributable {
    +
    +  private final List<CarbonInputSplit> carbonBlockInfoList;
    +
    +  private final String taskId;
    +
    +  public String getTaskId() {
    +    return taskId;
    +  }
    +
    +  public List<CarbonInputSplit> getCarbonInputSplitList() {
    +    return carbonBlockInfoList;
    +  }
    +
    +  public CarbonInputSplitTaskInfo(String taskId, List<CarbonInputSplit> carbonSplitListInfo) {
    +    this.taskId = taskId;
    +    this.carbonBlockInfoList = carbonSplitListInfo;
    +  }
    +
    +  @Override public String[] getLocations() {
    +    Set<String> locations = new HashSet<String>();
    +    for (CarbonInputSplit splitInfo : carbonBlockInfoList) {
    +      try {
    +        locations.addAll(Arrays.asList(splitInfo.getLocations()));
    +      } catch (IOException e) {
    +        e.printStackTrace();
    +      }
    +    }
    +    locations.toArray(new String[locations.size()]);
    +    List<String> nodes = CarbonInputSplitTaskInfo.maxNoNodes(carbonBlockInfoList);
    +    return nodes.toArray(new String[nodes.size()]);
    +  }
    +
    +  @Override public int compareTo(Distributable o) {
    +    return taskId.compareTo(((CarbonInputSplitTaskInfo) o).getTaskId());
    +  }
    +
    +  /**
    +   * Finding which node has the maximum number of blocks for it.
    +   *
    +   * @param blockList
    +   * @return
    +   */
    +  public static List<String> maxNoNodes(List<CarbonInputSplit> splitList) {
    +    boolean useIndex = true;
    +    Integer maxOccurence = 0;
    +    String maxNode = null;
    +    Map<String, Integer> nodeAndOccurenceMapping = new TreeMap<>();
    +
    +    // populate the map of node and number of occurences of that node.
    +    for (CarbonInputSplit split : splitList) {
    +      try {
    +        for (String node : split.getLocations()) {
    +          Integer nodeOccurence = nodeAndOccurenceMapping.get(node);
    +          if (null == nodeOccurence) {
    +            nodeAndOccurenceMapping.put(node, 1);
    +          } else {
    +            nodeOccurence++;
    +          }
    +        }
    +      } catch (IOException e) {
    +        e.printStackTrace();
    --- End diff --
   
    Add logger or throw the exception


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user sounakr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r103379976
 
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java ---
    @@ -0,0 +1,129 @@
    +/*
    --- End diff --
   
    This is a new class. Could you please tell me what is the problem with commit?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #604: [CARBONDATA-691] After Compaction records c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user sounakr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r104141178
 
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala ---
    @@ -255,77 +256,75 @@ class CarbonMergerRDD[K, V](
     
         // mapping of the node and block list.
         var nodeBlockMapping: java.util.Map[String, java.util.List[Distributable]] = new
    -            java.util.HashMap[String, java.util.List[Distributable]]
    +        java.util.HashMap[String, java.util.List[Distributable]]
     
         var noOfBlocks = 0
         val taskInfoList = new java.util.ArrayList[Distributable]
         var carbonInputSplits = mutable.Seq[CarbonInputSplit]()
     
    -    var blocksOfLastSegment: List[TableBlockInfo] = null
    +    var splitsOfLastSegment: List[CarbonInputSplit] = null
    +    // map for keeping the relation of a task and its blocks.
    +    val taskIdMapping: java.util.Map[String, java.util.List[CarbonInputSplit]] = new
    +        java.util.HashMap[String, java.util.List[CarbonInputSplit]]
    +
     
         // for each valid segment.
         for (eachSeg <- carbonMergerMapping.validSegments) {
    -      // map for keeping the relation of a task and its blocks.
    -      val taskIdMapping: java.util.Map[String, java.util.List[TableBlockInfo]] = new
    -            java.util.HashMap[String, java.util.List[TableBlockInfo]]
     
           // map for keeping the relation of a task and its blocks.
           job.getConfiguration.set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS, eachSeg)
     
    +      val updateDetails: UpdateVO = updateStatusManager.getInvalidTimestampRange(eachSeg)
    +      // val updateStatusManager = carbonLoadModel.getSegmentUpdateStatusManager
    --- End diff --
   
    Done. Removed the commented code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user ravikiran23 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r104142134
 
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala ---
    @@ -235,121 +238,141 @@ class CarbonMergerRDD[K, V](
         iter
       }
     
    -  override def getPreferredLocations(split: Partition): Seq[String] = {
    -    val theSplit = split.asInstanceOf[CarbonSparkPartition]
    -    theSplit.split.value.getLocations.filter(_ != "localhost")
    -  }
     
       override def getPartitions: Array[Partition] = {
         val startTime = System.currentTimeMillis()
         val absoluteTableIdentifier: AbsoluteTableIdentifier = new AbsoluteTableIdentifier(
           hdfsStoreLocation, new CarbonTableIdentifier(databaseName, factTableName, tableId)
         )
    -    val updateStatusManger: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
    +    val updateStatusManager: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
           absoluteTableIdentifier)
         val jobConf: JobConf = new JobConf(new Configuration)
         val job: Job = new Job(jobConf)
         val format = CarbonInputFormatUtil.createCarbonInputFormat(absoluteTableIdentifier, job)
         var defaultParallelism = sparkContext.defaultParallelism
         val result = new java.util.ArrayList[Partition](defaultParallelism)
    +    var partitionNo = 0
    +    var columnSize = 0
    +    var noOfBlocks = 0
     
         // mapping of the node and block list.
         var nodeBlockMapping: java.util.Map[String, java.util.List[Distributable]] = new
    -            java.util.HashMap[String, java.util.List[Distributable]]
    +        java.util.HashMap[String, java.util.List[Distributable]]
     
    -    var noOfBlocks = 0
         val taskInfoList = new java.util.ArrayList[Distributable]
         var carbonInputSplits = mutable.Seq[CarbonInputSplit]()
     
    -    var blocksOfLastSegment: List[TableBlockInfo] = null
    +    var splitsOfLastSegment: List[CarbonInputSplit] = null
    +    // map for keeping the relation of a task and its blocks.
    +    val taskIdMapping: java.util.Map[String, java.util.List[CarbonInputSplit]] = new
    +        java.util.HashMap[String, java.util.List[CarbonInputSplit]]
     
         // for each valid segment.
         for (eachSeg <- carbonMergerMapping.validSegments) {
    -      // map for keeping the relation of a task and its blocks.
    -      val taskIdMapping: java.util.Map[String, java.util.List[TableBlockInfo]] = new
    -            java.util.HashMap[String, java.util.List[TableBlockInfo]]
     
           // map for keeping the relation of a task and its blocks.
           job.getConfiguration.set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS, eachSeg)
     
    +      val updateDetails: UpdateVO = updateStatusManager.getInvalidTimestampRange(eachSeg)
    +
           // get splits
           val splits = format.getSplits(job)
    -      carbonInputSplits ++:= splits.asScala.map(_.asInstanceOf[CarbonInputSplit])
    -
    -      val updateDetails: UpdateVO = updateStatusManger.getInvalidTimestampRange(eachSeg)
    -
    -      // take the blocks of one segment.
    -      val blocksOfOneSegment = carbonInputSplits.map(inputSplit =>
    -        new TableBlockInfo(inputSplit.getPath.toString,
    -          inputSplit.getStart, inputSplit.getSegmentId,
    -          inputSplit.getLocations, inputSplit.getLength, inputSplit.getVersion
    -        )
    -      )
    -        .filter(blockInfo => !CarbonUtil
    -          .isInvalidTableBlock(blockInfo, updateDetails, updateStatusManger))
     
           // keep on assigning till last one is reached.
    -      if (null != blocksOfOneSegment && blocksOfOneSegment.size > 0) {
    -        blocksOfLastSegment = blocksOfOneSegment.asJava
    +      if (null != splits && splits.size > 0) {
    +        splitsOfLastSegment = splits.asScala.map(_.asInstanceOf[CarbonInputSplit]).toList.asJava
           }
     
    -      // populate the task and its block mapping.
    -      blocksOfOneSegment.foreach(f = tableBlockInfo => {
    -        val taskNo = CarbonTablePath.DataFileUtil.getTaskNo(tableBlockInfo.getFilePath)
    -        val blockList = taskIdMapping.get(taskNo)
    -        if (null == blockList) {
    -          val blockListTemp = new java.util.ArrayList[TableBlockInfo]()
    -          blockListTemp.add(tableBlockInfo)
    -          taskIdMapping.put(taskNo, blockListTemp)
    -        }
    -        else {
    -          blockList.add(tableBlockInfo)
    -        }
    +      carbonInputSplits ++:= splits.asScala.map(_.asInstanceOf[CarbonInputSplit]).filter(entry => {
    +        val blockInfo = new TableBlockInfo(entry.getPath.toString,
    +          entry.getStart, entry.getSegmentId,
    +          entry.getLocations, entry.getLength, entry.getVersion
    +        )
    +        !CarbonUtil
    +          .isInvalidTableBlock(blockInfo, updateDetails, updateStatusManager)
           })
    -
    -      noOfBlocks += blocksOfOneSegment.size
    -      taskIdMapping.asScala.foreach(
    -        entry =>
    -          taskInfoList.add(new TableTaskInfo(entry._1, entry._2).asInstanceOf[Distributable])
    -      )
    -
         }
     
         // prepare the details required to extract the segment properties using last segment.
    -    if (null != carbonInputSplits && carbonInputSplits.nonEmpty) {
    -      // taking head as scala sequence is use and while adding it will add at first
    -      // so as we need to update the update the key of older segments with latest keygenerator
    -      // we need to take the top of the split
    -      val carbonInputSplit = carbonInputSplits.head
    +    if (null != splitsOfLastSegment && splitsOfLastSegment.size() > 0) {
    +      val carbonInputSplit = splitsOfLastSegment.get(0)
           var dataFileFooter: DataFileFooter = null
     
           try {
             dataFileFooter = CarbonUtil.readMetadatFile(
    -            CarbonInputSplit.getTableBlockInfo(carbonInputSplit))
    +          CarbonInputSplit.getTableBlockInfo(carbonInputSplit))
           } catch {
             case e: IOException =>
               logError("Exception in preparing the data file footer for compaction " + e.getMessage)
               throw e
           }
     
    -      carbonMergerMapping.maxSegmentColCardinality = dataFileFooter.getSegmentInfo
    -        .getColumnCardinality
    +      columnSize = dataFileFooter.getSegmentInfo.getColumnCardinality.size
           carbonMergerMapping.maxSegmentColumnSchemaList = dataFileFooter.getColumnInTable.asScala
             .toList
         }
     
    -    // val blocks = carbonInputSplits.map(_.asInstanceOf[Distributable]).asJava
    -    // send complete list of blocks to the mapping util.
    -    nodeBlockMapping = CarbonLoaderUtil.nodeBlockMapping(taskInfoList, -1)
    +    var cardinality = new Array[Int](columnSize)
    +
    +    carbonInputSplits.foreach(splits => {
    +      val taskNo = splits.taskId
    +      var dataFileFooter: DataFileFooter = null
    +
    +      val splitList = taskIdMapping.get(taskNo)
    +      noOfBlocks += 1
    +      if (null == splitList) {
    +        val splitTempList = new util.ArrayList[CarbonInputSplit]()
    +        splitTempList.add(splits)
    +        taskIdMapping.put(taskNo, splitTempList)
    +      } else {
    +        splitList.add(splits)
    +      }
    +
    +      // Check the cardinality of each columns and set the highest.
    +      try {
    +        dataFileFooter = CarbonUtil.readMetadatFile(
    +          CarbonInputSplit.getTableBlockInfo(splits))
    +      } catch {
    +        case e: IOException =>
    +          logError("Exception in preparing the data file footer for compaction " + e.getMessage)
    +          throw e
    +      }
    +
    +      // Calculate the Cardinality of the new segment
    +      var targetCardinality = dataFileFooter.getSegmentInfo.getColumnCardinality
    --- End diff --
   
    move the target cardinality calculation to a new method


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #604: [CARBONDATA-691] After Compaction records c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user sounakr commented on the issue:

    https://github.com/apache/incubator-carbondata/pull/604
 
    @jackylk : This problem will get reproduce in a multi node system where a single Load will create multiple carbon data file is by multiple tasks. I have tested the code changes in a multi node cluster with large data and output ic coming correctly.
    Another problem observed while testing is choosing the correct cardinality of the new segment formed due to compaction. Previously we used to copy the last segment cardinality to the new compacted segment. But with IUD features updates can run on any segments and in case the updates happens on any intermediate segments then there is high probability that cardinality of that segment will be more than the last segment. So now instead of copying the cardinality from the last segment we calculate the highest cardinility values from all the segments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user sounakr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r104144870
 
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java ---
    @@ -0,0 +1,129 @@
    +/*
    + * 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.hadoop.util;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +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 org.apache.carbondata.core.datastore.block.Distributable;
    +import org.apache.carbondata.hadoop.CarbonInputSplit;
    +
    +public class CarbonInputSplitTaskInfo implements Distributable {
    +
    +  private final List<CarbonInputSplit> carbonBlockInfoList;
    +
    +  private final String taskId;
    +
    +  public String getTaskId() {
    +    return taskId;
    +  }
    +
    +  public List<CarbonInputSplit> getCarbonInputSplitList() {
    +    return carbonBlockInfoList;
    +  }
    +
    +  public CarbonInputSplitTaskInfo(String taskId, List<CarbonInputSplit> carbonSplitListInfo) {
    +    this.taskId = taskId;
    +    this.carbonBlockInfoList = carbonSplitListInfo;
    +  }
    +
    +  @Override public String[] getLocations() {
    +    Set<String> locations = new HashSet<String>();
    +    for (CarbonInputSplit splitInfo : carbonBlockInfoList) {
    +      try {
    +        locations.addAll(Arrays.asList(splitInfo.getLocations()));
    +      } catch (IOException e) {
    +        e.printStackTrace();
    --- End diff --
   
    Done. Throwing exception now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user sounakr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r104145103
 
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java ---
    @@ -0,0 +1,129 @@
    +/*
    + * 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.hadoop.util;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +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 org.apache.carbondata.core.datastore.block.Distributable;
    +import org.apache.carbondata.hadoop.CarbonInputSplit;
    +
    +public class CarbonInputSplitTaskInfo implements Distributable {
    +
    +  private final List<CarbonInputSplit> carbonBlockInfoList;
    +
    +  private final String taskId;
    +
    +  public String getTaskId() {
    +    return taskId;
    +  }
    +
    +  public List<CarbonInputSplit> getCarbonInputSplitList() {
    +    return carbonBlockInfoList;
    +  }
    +
    +  public CarbonInputSplitTaskInfo(String taskId, List<CarbonInputSplit> carbonSplitListInfo) {
    +    this.taskId = taskId;
    +    this.carbonBlockInfoList = carbonSplitListInfo;
    +  }
    +
    +  @Override public String[] getLocations() {
    +    Set<String> locations = new HashSet<String>();
    +    for (CarbonInputSplit splitInfo : carbonBlockInfoList) {
    +      try {
    +        locations.addAll(Arrays.asList(splitInfo.getLocations()));
    +      } catch (IOException e) {
    +        e.printStackTrace();
    --- End diff --
   
    Done. Throwing Exception.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user sounakr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r104145140
 
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java ---
    @@ -0,0 +1,129 @@
    +/*
    + * 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.hadoop.util;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +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 org.apache.carbondata.core.datastore.block.Distributable;
    +import org.apache.carbondata.hadoop.CarbonInputSplit;
    +
    +public class CarbonInputSplitTaskInfo implements Distributable {
    +
    +  private final List<CarbonInputSplit> carbonBlockInfoList;
    +
    +  private final String taskId;
    +
    +  public String getTaskId() {
    +    return taskId;
    +  }
    +
    +  public List<CarbonInputSplit> getCarbonInputSplitList() {
    +    return carbonBlockInfoList;
    +  }
    +
    +  public CarbonInputSplitTaskInfo(String taskId, List<CarbonInputSplit> carbonSplitListInfo) {
    +    this.taskId = taskId;
    +    this.carbonBlockInfoList = carbonSplitListInfo;
    +  }
    +
    +  @Override public String[] getLocations() {
    +    Set<String> locations = new HashSet<String>();
    +    for (CarbonInputSplit splitInfo : carbonBlockInfoList) {
    +      try {
    +        locations.addAll(Arrays.asList(splitInfo.getLocations()));
    +      } catch (IOException e) {
    +        e.printStackTrace();
    +      }
    +    }
    +    locations.toArray(new String[locations.size()]);
    +    List<String> nodes = CarbonInputSplitTaskInfo.maxNoNodes(carbonBlockInfoList);
    +    return nodes.toArray(new String[nodes.size()]);
    +  }
    +
    +  @Override public int compareTo(Distributable o) {
    +    return taskId.compareTo(((CarbonInputSplitTaskInfo) o).getTaskId());
    +  }
    +
    +  /**
    +   * Finding which node has the maximum number of blocks for it.
    +   *
    +   * @param blockList
    +   * @return
    +   */
    +  public static List<String> maxNoNodes(List<CarbonInputSplit> splitList) {
    +    boolean useIndex = true;
    +    Integer maxOccurence = 0;
    +    String maxNode = null;
    +    Map<String, Integer> nodeAndOccurenceMapping = new TreeMap<>();
    +
    +    // populate the map of node and number of occurences of that node.
    +    for (CarbonInputSplit split : splitList) {
    +      try {
    +        for (String node : split.getLocations()) {
    +          Integer nodeOccurence = nodeAndOccurenceMapping.get(node);
    +          if (null == nodeOccurence) {
    +            nodeAndOccurenceMapping.put(node, 1);
    +          } else {
    +            nodeOccurence++;
    +          }
    +        }
    +      } catch (IOException e) {
    +        e.printStackTrace();
    --- End diff --
   
    Done. Throwing Exception.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #604: [CARBONDATA-691] After Compaction records c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #604: [CARBONDATA-691] After Compaction records c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata pull request #604: [CARBONDATA-691] After Compaction re...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user sounakr commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/604#discussion_r104346516
 
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala ---
    @@ -235,121 +238,141 @@ class CarbonMergerRDD[K, V](
         iter
       }
     
    -  override def getPreferredLocations(split: Partition): Seq[String] = {
    -    val theSplit = split.asInstanceOf[CarbonSparkPartition]
    -    theSplit.split.value.getLocations.filter(_ != "localhost")
    -  }
     
       override def getPartitions: Array[Partition] = {
         val startTime = System.currentTimeMillis()
         val absoluteTableIdentifier: AbsoluteTableIdentifier = new AbsoluteTableIdentifier(
           hdfsStoreLocation, new CarbonTableIdentifier(databaseName, factTableName, tableId)
         )
    -    val updateStatusManger: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
    +    val updateStatusManager: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
           absoluteTableIdentifier)
         val jobConf: JobConf = new JobConf(new Configuration)
         val job: Job = new Job(jobConf)
         val format = CarbonInputFormatUtil.createCarbonInputFormat(absoluteTableIdentifier, job)
         var defaultParallelism = sparkContext.defaultParallelism
         val result = new java.util.ArrayList[Partition](defaultParallelism)
    +    var partitionNo = 0
    +    var columnSize = 0
    +    var noOfBlocks = 0
     
         // mapping of the node and block list.
         var nodeBlockMapping: java.util.Map[String, java.util.List[Distributable]] = new
    -            java.util.HashMap[String, java.util.List[Distributable]]
    +        java.util.HashMap[String, java.util.List[Distributable]]
     
    -    var noOfBlocks = 0
         val taskInfoList = new java.util.ArrayList[Distributable]
         var carbonInputSplits = mutable.Seq[CarbonInputSplit]()
     
    -    var blocksOfLastSegment: List[TableBlockInfo] = null
    +    var splitsOfLastSegment: List[CarbonInputSplit] = null
    +    // map for keeping the relation of a task and its blocks.
    +    val taskIdMapping: java.util.Map[String, java.util.List[CarbonInputSplit]] = new
    +        java.util.HashMap[String, java.util.List[CarbonInputSplit]]
     
         // for each valid segment.
         for (eachSeg <- carbonMergerMapping.validSegments) {
    -      // map for keeping the relation of a task and its blocks.
    -      val taskIdMapping: java.util.Map[String, java.util.List[TableBlockInfo]] = new
    -            java.util.HashMap[String, java.util.List[TableBlockInfo]]
     
           // map for keeping the relation of a task and its blocks.
           job.getConfiguration.set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS, eachSeg)
     
    +      val updateDetails: UpdateVO = updateStatusManager.getInvalidTimestampRange(eachSeg)
    +
           // get splits
           val splits = format.getSplits(job)
    -      carbonInputSplits ++:= splits.asScala.map(_.asInstanceOf[CarbonInputSplit])
    -
    -      val updateDetails: UpdateVO = updateStatusManger.getInvalidTimestampRange(eachSeg)
    -
    -      // take the blocks of one segment.
    -      val blocksOfOneSegment = carbonInputSplits.map(inputSplit =>
    -        new TableBlockInfo(inputSplit.getPath.toString,
    -          inputSplit.getStart, inputSplit.getSegmentId,
    -          inputSplit.getLocations, inputSplit.getLength, inputSplit.getVersion
    -        )
    -      )
    -        .filter(blockInfo => !CarbonUtil
    -          .isInvalidTableBlock(blockInfo, updateDetails, updateStatusManger))
     
           // keep on assigning till last one is reached.
    -      if (null != blocksOfOneSegment && blocksOfOneSegment.size > 0) {
    -        blocksOfLastSegment = blocksOfOneSegment.asJava
    +      if (null != splits && splits.size > 0) {
    +        splitsOfLastSegment = splits.asScala.map(_.asInstanceOf[CarbonInputSplit]).toList.asJava
           }
     
    -      // populate the task and its block mapping.
    -      blocksOfOneSegment.foreach(f = tableBlockInfo => {
    -        val taskNo = CarbonTablePath.DataFileUtil.getTaskNo(tableBlockInfo.getFilePath)
    -        val blockList = taskIdMapping.get(taskNo)
    -        if (null == blockList) {
    -          val blockListTemp = new java.util.ArrayList[TableBlockInfo]()
    -          blockListTemp.add(tableBlockInfo)
    -          taskIdMapping.put(taskNo, blockListTemp)
    -        }
    -        else {
    -          blockList.add(tableBlockInfo)
    -        }
    +      carbonInputSplits ++:= splits.asScala.map(_.asInstanceOf[CarbonInputSplit]).filter(entry => {
    +        val blockInfo = new TableBlockInfo(entry.getPath.toString,
    +          entry.getStart, entry.getSegmentId,
    +          entry.getLocations, entry.getLength, entry.getVersion
    +        )
    +        !CarbonUtil
    +          .isInvalidTableBlock(blockInfo, updateDetails, updateStatusManager)
           })
    -
    -      noOfBlocks += blocksOfOneSegment.size
    -      taskIdMapping.asScala.foreach(
    -        entry =>
    -          taskInfoList.add(new TableTaskInfo(entry._1, entry._2).asInstanceOf[Distributable])
    -      )
    -
         }
     
         // prepare the details required to extract the segment properties using last segment.
    -    if (null != carbonInputSplits && carbonInputSplits.nonEmpty) {
    -      // taking head as scala sequence is use and while adding it will add at first
    -      // so as we need to update the update the key of older segments with latest keygenerator
    -      // we need to take the top of the split
    -      val carbonInputSplit = carbonInputSplits.head
    +    if (null != splitsOfLastSegment && splitsOfLastSegment.size() > 0) {
    +      val carbonInputSplit = splitsOfLastSegment.get(0)
           var dataFileFooter: DataFileFooter = null
     
           try {
             dataFileFooter = CarbonUtil.readMetadatFile(
    -            CarbonInputSplit.getTableBlockInfo(carbonInputSplit))
    +          CarbonInputSplit.getTableBlockInfo(carbonInputSplit))
           } catch {
             case e: IOException =>
               logError("Exception in preparing the data file footer for compaction " + e.getMessage)
               throw e
           }
     
    -      carbonMergerMapping.maxSegmentColCardinality = dataFileFooter.getSegmentInfo
    -        .getColumnCardinality
    +      columnSize = dataFileFooter.getSegmentInfo.getColumnCardinality.size
           carbonMergerMapping.maxSegmentColumnSchemaList = dataFileFooter.getColumnInTable.asScala
             .toList
         }
     
    -    // val blocks = carbonInputSplits.map(_.asInstanceOf[Distributable]).asJava
    -    // send complete list of blocks to the mapping util.
    -    nodeBlockMapping = CarbonLoaderUtil.nodeBlockMapping(taskInfoList, -1)
    +    var cardinality = new Array[Int](columnSize)
    +
    +    carbonInputSplits.foreach(splits => {
    +      val taskNo = splits.taskId
    +      var dataFileFooter: DataFileFooter = null
    +
    +      val splitList = taskIdMapping.get(taskNo)
    +      noOfBlocks += 1
    +      if (null == splitList) {
    +        val splitTempList = new util.ArrayList[CarbonInputSplit]()
    +        splitTempList.add(splits)
    +        taskIdMapping.put(taskNo, splitTempList)
    +      } else {
    +        splitList.add(splits)
    +      }
    +
    +      // Check the cardinality of each columns and set the highest.
    +      try {
    +        dataFileFooter = CarbonUtil.readMetadatFile(
    +          CarbonInputSplit.getTableBlockInfo(splits))
    +      } catch {
    +        case e: IOException =>
    +          logError("Exception in preparing the data file footer for compaction " + e.getMessage)
    +          throw e
    +      }
    +
    +      // Calculate the Cardinality of the new segment
    +      var targetCardinality = dataFileFooter.getSegmentInfo.getColumnCardinality
    --- End diff --
   
    Done. Moved the logic of cardinality calculation to a function.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #604: [CARBONDATA-691] After Compaction records c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

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



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-carbondata issue #604: [CARBONDATA-691] After Compaction records c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user ravipesala commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [hidden email] or file a JIRA ticket
with INFRA.
---
12