[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-07 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131635037
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
 ---
@@ -181,9 +181,11 @@ public static void createLocations(String[] locations) 
{
* @return
*/
   public static String[] getLocalDataFolderLocation(String databaseName, 
String tableName,
--- End diff --

refactor it, make it two functions, should not add more boolean to it.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-07 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131634496
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
 ---
@@ -183,6 +190,138 @@ case class AlterTableCompaction(alterTableModel: 
AlterTableModel) extends Runnab
   }
 }
 
+/**
+ * Command for Alter Table Add & Split partition
+ * Add is a special case of Splitting the default partition (part0)
+ * @param alterTableSplitPartitionModel
+ */
+case class AlterTableSplitPartition(alterTableSplitPartitionModel: 
AlterTableSplitPartitionModel)
+  extends RunnableCommand with DataProcessCommand with 
SchemaProcessCommand {
+  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+  val tableName = alterTableSplitPartitionModel.tableName
+  val splitInfo = alterTableSplitPartitionModel.splitInfo
+  val partitionId = alterTableSplitPartitionModel.partitionId.toInt
+  var partitionInfo: PartitionInfo = null
+  var carbonMetaStore: CarbonMetaStore = null
+  var relation: CarbonRelation = null
+  var dbName: String = null
+  var storePath: String = null
+  var carbonTableIdentifier: CarbonTableIdentifier = null
+  val timestampFormatter = new 
SimpleDateFormat(CarbonProperties.getInstance
+.getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+  val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
+.getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+  CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+  val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+LockUsage.COMPACTION_LOCK,
+LockUsage.DELETE_SEGMENT_LOCK,
+LockUsage.DROP_TABLE_LOCK,
+LockUsage.CLEAN_FILES_LOCK,
+LockUsage.ALTER_PARTITION_LOCK)
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+var success = false
+try {
+  processData(sparkSession)
+  success = true
+} catch {
+  case e: Exception => sys.error(s"Error in alter table split 
partition : ${ e.getMessage }")
+  success = false
+}
+if (success) {
+  processSchema(sparkSession)
--- End diff --

processSchema first then processData


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-07 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131634414
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
 ---
@@ -183,6 +190,138 @@ case class AlterTableCompaction(alterTableModel: 
AlterTableModel) extends Runnab
   }
 }
 
+/**
+ * Command for Alter Table Add & Split partition
+ * Add is a special case of Splitting the default partition (part0)
+ * @param alterTableSplitPartitionModel
+ */
+case class AlterTableSplitPartition(alterTableSplitPartitionModel: 
AlterTableSplitPartitionModel)
--- End diff --

rename to `AlterTableSplitPartitionCommand`


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-07 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131630948
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
 ---
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.rdd
+
+import java.util.ArrayList
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.mapreduce.Job
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.hive.DistributionUtil
+import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.util.PartitionUtils
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.datastore.block.{Distributable, 
SegmentProperties, TaskBlockInfo}
+import 
org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, 
CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+import 
org.apache.carbondata.core.metadata.schema.table.column.{CarbonDimension, 
CarbonMeasure}
+import 
org.apache.carbondata.core.scan.result.iterator.PartitionSpliterRawResultIterator
+import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper
+import org.apache.carbondata.core.util.{ByteUtil, DataTypeUtil}
+import org.apache.carbondata.hadoop.{CarbonInputSplit, 
CarbonMultiBlockSplit}
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
+import org.apache.carbondata.processing.merger.CarbonCompactionUtil
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.spliter.CarbonSplitExecutor
+import org.apache.carbondata.spark.load.CarbonLoaderUtil
+
+
+/**
+ *
--- End diff --

Add comment for this RDD


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-03 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131067233
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/spliter/CarbonDataSpliterUtil.java
 ---
@@ -0,0 +1,40 @@
+/*
+ * 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.processing.spliter;
+
+import java.util.List;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.scan.result.BatchResult;
+
+public final class CarbonDataSpliterUtil {
--- End diff --

Oops, it should be removed.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-03 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131067082
  
--- Diff: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
 ---
@@ -321,6 +321,84 @@ private AbsoluteTableIdentifier 
getAbsoluteTableIdentifier(Configuration configu
   }
 
   /**
+   * Read data in one segment. For alter table partition statement
+   * @param job
+   * @param targetSegment
+   * @param oldPartitionIdList  get old partitionId before partitionInfo 
was changed
+   * @return
+   * @throws IOException
+   */
+  public List getSplitsOfOneSegment(JobContext job, String 
targetSegment,
+  List oldPartitionIdList, PartitionInfo partitionInfo)
+  throws IOException {
+AbsoluteTableIdentifier identifier = 
getAbsoluteTableIdentifier(job.getConfiguration());
+List invalidSegments = new ArrayList<>();
+List invalidTimestampsList = new ArrayList<>();
+
+List segmentList = new ArrayList<>();
+segmentList.add(targetSegment);
+setSegmentsToAccess(job.getConfiguration(), segmentList);
+try {
+
+  // process and resolve the expression
+  Expression filter = getFilterPredicates(job.getConfiguration());
+  CarbonTable carbonTable = 
getOrCreateCarbonTable(job.getConfiguration());
+  // this will be null in case of corrupt schema file.
+  if (null == carbonTable) {
+throw new IOException("Missing/Corrupt schema file for table.");
+  }
+
+  CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
+
+  // prune partitions for filter query on partition table
+  String partitionIds = job.getConfiguration().get(ALTER_PARTITION_ID);
+  BitSet matchedPartitions = null;
+  if (partitionInfo != null) {
+matchedPartitions = setMatchedPartitions(partitionIds, filter, 
partitionInfo);
+if (matchedPartitions != null) {
+  if (matchedPartitions.cardinality() == 0) {
+return new ArrayList();
+  } else if (matchedPartitions.cardinality() == 
partitionInfo.getNumPartitions()) {
+matchedPartitions = null;
+  }
+}
+  }
+
+  FilterResolverIntf filterInterface = 
CarbonInputFormatUtil.resolveFilter(filter, identifier);
+  // do block filtering and get split
+  List splits = getSplits(job, filterInterface, 
segmentList, matchedPartitions,
+  partitionInfo, oldPartitionIdList);
+  // pass the invalid segment to task side in order to remove index 
entry in task side
+  if (invalidSegments.size() > 0) {
+for (InputSplit split : splits) {
+  ((CarbonInputSplit) split).setInvalidSegments(invalidSegments);
+  ((CarbonInputSplit) 
split).setInvalidTimestampRange(invalidTimestampsList);
+}
+  }
+  return splits;
+} catch (IOException e) {
+  throw new RuntimeException("Can't get splits of the target segment 
", e);
+}
+  }
+
+  private BitSet setMatchedPartitions(String partitionIds, Expression 
filter,
+  PartitionInfo partitionInfo) {
+BitSet matchedPartitions = null;
+if (null != partitionIds) {
+  String[] partList = partitionIds.replace("[", "").replace("]", 
"").split(",");
+  matchedPartitions = new BitSet(Integer.parseInt(partList[0]));
--- End diff --

Sure, I can add some simple comments in the 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-03 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131067032
  
--- Diff: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
 ---
@@ -321,6 +321,84 @@ private AbsoluteTableIdentifier 
getAbsoluteTableIdentifier(Configuration configu
   }
 
   /**
+   * Read data in one segment. For alter table partition statement
+   * @param job
+   * @param targetSegment
+   * @param oldPartitionIdList  get old partitionId before partitionInfo 
was changed
+   * @return
+   * @throws IOException
+   */
+  public List getSplitsOfOneSegment(JobContext job, String 
targetSegment,
+  List oldPartitionIdList, PartitionInfo partitionInfo)
+  throws IOException {
+AbsoluteTableIdentifier identifier = 
getAbsoluteTableIdentifier(job.getConfiguration());
+List invalidSegments = new ArrayList<>();
+List invalidTimestampsList = new ArrayList<>();
+
+List segmentList = new ArrayList<>();
+segmentList.add(targetSegment);
+setSegmentsToAccess(job.getConfiguration(), segmentList);
+try {
+
+  // process and resolve the expression
+  Expression filter = getFilterPredicates(job.getConfiguration());
+  CarbonTable carbonTable = 
getOrCreateCarbonTable(job.getConfiguration());
+  // this will be null in case of corrupt schema file.
+  if (null == carbonTable) {
+throw new IOException("Missing/Corrupt schema file for table.");
+  }
+
+  CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
+
+  // prune partitions for filter query on partition table
+  String partitionIds = job.getConfiguration().get(ALTER_PARTITION_ID);
+  BitSet matchedPartitions = null;
+  if (partitionInfo != null) {
+matchedPartitions = setMatchedPartitions(partitionIds, filter, 
partitionInfo);
+if (matchedPartitions != null) {
+  if (matchedPartitions.cardinality() == 0) {
+return new ArrayList();
+  } else if (matchedPartitions.cardinality() == 
partitionInfo.getNumPartitions()) {
+matchedPartitions = null;
+  }
+}
+  }
+
+  FilterResolverIntf filterInterface = 
CarbonInputFormatUtil.resolveFilter(filter, identifier);
+  // do block filtering and get split
+  List splits = getSplits(job, filterInterface, 
segmentList, matchedPartitions,
+  partitionInfo, oldPartitionIdList);
+  // pass the invalid segment to task side in order to remove index 
entry in task side
+  if (invalidSegments.size() > 0) {
+for (InputSplit split : splits) {
+  ((CarbonInputSplit) split).setInvalidSegments(invalidSegments);
+  ((CarbonInputSplit) 
split).setInvalidTimestampRange(invalidTimestampsList);
+}
+  }
+  return splits;
+} catch (IOException e) {
+  throw new RuntimeException("Can't get splits of the target segment 
", e);
+}
+  }
+
+  private BitSet setMatchedPartitions(String partitionIds, Expression 
filter,
+  PartitionInfo partitionInfo) {
+BitSet matchedPartitions = null;
+if (null != partitionIds) {
+  String[] partList = partitionIds.replace("[", "").replace("]", 
"").split(",");
+  matchedPartitions = new BitSet(Integer.parseInt(partList[0]));
--- End diff --

Currently the partitionIds from alter table statement could only be one 
element.
For example 
'alter table t0 split(4) into XXX'
'alter table t1 drop partition(3)' (will submit in another PR ) 
We suppose drop partition is a dangerous operation and allow to drop only 
one partition in a time.
Maybe in the future we can discuss to extend drop multiple partitions in 
one statement.
And also we can extend MERGE PARTITION action in later versions(no plan in 
version1.2)


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-03 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131063903
  
--- Diff: 
hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
@@ -107,6 +107,7 @@
   // comma separated list of input files
   public static final String INPUT_FILES =
   "mapreduce.input.carboninputformat.files";
+  public static final String ALTER_PARTITION_ID = 
"mapreduce.input.carboninputformat.partitionid";
--- End diff --

I have migrate all the changes to CarbonTableInputFormat. Just keep the 
modification in CarbonInputFormat. It's already not used and you can remove it 
safely.
But sure, I can recover this file if it's necessary.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-03 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131063552
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
 ---
@@ -101,17 +126,40 @@ object CarbonPartitionExample {
 spark.sql("""
| CREATE TABLE IF NOT EXISTS t5
| (
+   | id Int,
| vin String,
| logdate Timestamp,
| phonenumber Long,
-   | area String
+   | area String,
+   | salary Int
|)
| PARTITIONED BY (country String)
| STORED BY 'carbondata'
| TBLPROPERTIES('PARTITION_TYPE'='LIST',
-   | 'LIST_INFO'='(China,United States),UK ,japan,(Canada,Russia), 
South Korea ')
+   | 'LIST_INFO'='(China, US),UK ,Japan,(Canada,Russia, Good, 
NotGood), Korea ')
""".stripMargin)
 
+// load data into partition table
+spark.sql(s"""
+   LOAD DATA LOCAL INPATH '$testData' into table t0 
options('BAD_RECORDS_ACTION'='FORCE')
+   """)
+spark.sql(s"""
+   LOAD DATA LOCAL INPATH '$testData' into table t5 
options('BAD_RECORDS_ACTION'='FORCE')
+   """)
+
+// alter list partition table t5 to add a partition
+spark.sql(s"""Alter table t5 add partition ('OutSpace')""".stripMargin)
+// alter list partition table t5 to split partition 4 into 3 
independent partition
+spark.sql(
+  s"""
+ Alter table t5 split partition(4) into ('Canada', 'Russia', 
'(Good, NotGood)')
+   """.stripMargin)
--- End diff --

yes, I have written the test case to verify it. Please refer to 
TestAlterPartitionTable.scala


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-03 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131063435
  
--- Diff: conf/carbon.properties.template ---
@@ -42,6 +42,9 @@ carbon.enableXXHash=true
 #carbon.max.level.cache.size=-1
 #enable prefetch of data during merge sort while reading data from sort 
temp files in data loading
 #carbon.merge.sort.prefetch=true
+ Alter Partition Configuration 
+#Number of cores to be used while alter partition
--- End diff --

it will be used when take action of multiple segments in parallel. this 
configuration will allow user to set the threads according to their hardware.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-03 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131059761
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
 ---
@@ -184,6 +189,161 @@ case class AlterTableCompaction(alterTableModel: 
AlterTableModel) extends Runnab
   }
 }
 
+/**
+ * Command for Alter Table Add & Split partition
+ * Add is a special case of Splitting the default partition (part0)
+ * @param alterTableSplitPartitionModel
+ */
+case class AlterTableSplitPartition(alterTableSplitPartitionModel: 
AlterTableSplitPartitionModel)
+  extends RunnableCommand {
+  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+
+val tableName = alterTableSplitPartitionModel.tableName
+val dbName = alterTableSplitPartitionModel.databaseName
+  .getOrElse(sparkSession.catalog.currentDatabase)
+val splitInfo = alterTableSplitPartitionModel.splitInfo
+val partitionId = 
Integer.parseInt(alterTableSplitPartitionModel.partitionId)
+val timestampFormatter = new 
SimpleDateFormat(CarbonProperties.getInstance
+  .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
+  .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+
+val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+  LockUsage.COMPACTION_LOCK,
+  LockUsage.DELETE_SEGMENT_LOCK,
+  LockUsage.DROP_TABLE_LOCK,
+  LockUsage.CLEAN_FILES_LOCK,
+  LockUsage.ALTER_PARTITION_LOCK)
+var locks = List.empty[ICarbonLock]
+try {
+  locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
+locksToBeAcquired)(sparkSession)
+  val carbonMetastore = 
CarbonEnv.getInstance(sparkSession).carbonMetastore
+  val relation = carbonMetastore.lookupRelation(Option(dbName), 
tableName)(sparkSession)
+.asInstanceOf[CarbonRelation]
+  val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+  val storePath = relation.tableMeta.storePath
+  if (relation == null) {
+sys.error(s"Table $dbName.$tableName does not exist")
+  }
+  carbonMetastore.checkSchemasModifiedTimeAndReloadTables(storePath)
+  if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + 
tableName)) {
+LOGGER.error(s"Alter table failed. table not found: 
$dbName.$tableName")
+sys.error(s"Alter table failed. table not found: 
$dbName.$tableName")
+  }
+  val carbonLoadModel = new CarbonLoadModel()
+
+  val table = relation.tableMeta.carbonTable
+  val partitionInfo = table.getPartitionInfo(tableName)
+  val partitionIdList = partitionInfo.getPartitionIds.asScala
+  // keep a copy of partitionIdList before update partitionInfo.
+  // will be used in partition data scan
+  val oldPartitionIdList: ArrayBuffer[Int] = new ArrayBuffer[Int]()
+  for (i: Integer <- partitionIdList) {
+oldPartitionIdList.append(i)
+  }
+
+  if (partitionInfo == null) {
+sys.error(s"Table $tableName is not a partition table.")
+  }
+  if (partitionInfo.getPartitionType == PartitionType.HASH) {
+sys.error(s"Hash partition table cannot be added or split!")
+  }
+  /**
+   * verify the add/split information and update the partitionInfo:
+   *  1. update rangeInfo/listInfo
+   *  2. update partitionIds
+   */
+  val columnDataType = 
partitionInfo.getColumnSchemaList.get(0).getDataType
+  val index = partitionIdList.indexOf(partitionId)
+  if (partitionInfo.getPartitionType == PartitionType.RANGE) {
--- End diff --

try to create more private functions and use it in `run`, thus `run` is 
shorter, better for readability


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-03 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131059448
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
 ---
@@ -184,6 +189,161 @@ case class AlterTableCompaction(alterTableModel: 
AlterTableModel) extends Runnab
   }
 }
 
+/**
+ * Command for Alter Table Add & Split partition
+ * Add is a special case of Splitting the default partition (part0)
+ * @param alterTableSplitPartitionModel
+ */
+case class AlterTableSplitPartition(alterTableSplitPartitionModel: 
AlterTableSplitPartitionModel)
+  extends RunnableCommand {
--- End diff --

with `SchemaProcessCommand` and `DataProcessCommand`
split `run` method into `processSchema` and `processData`


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-03 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131059447
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
 ---
@@ -345,6 +365,65 @@ object CarbonDataRDDFactory {
 compactionThread.run()
   }
 
+  case class SplitThread(sqlContext: SQLContext,
+  carbonLoadModel: CarbonLoadModel,
+  executor: ExecutorService,
+  storePath: String,
+  segmentId: String,
+  partitionId: String,
+  oldPartitionIdList: List[Int]) extends Thread {
+  override def run(): Unit = {
+try {
+  DataManagementFunc.executePartitionSplit(sqlContext,
+carbonLoadModel, executor, storePath, segmentId, partitionId,
+oldPartitionIdList)
+} catch {
+  case e: Exception =>
+LOGGER.error(s"Exception in partition split thread: ${ 
e.getMessage } }")
+}
+  }
+  }
+
+  def startSplitThreads(sqlContext: SQLContext,
+  carbonLoadModel: CarbonLoadModel,
+  storePath: String,
+  partitionId: String,
+  oldPartitionIdList: List[Int]): Unit = {
+val numberOfCores = CarbonProperties.getInstance()
+  .getProperty(CarbonCommonConstants.NUM_CORES_ALT_PARTITION,
+CarbonCommonConstants.DEFAULT_NUMBER_CORES)
+val executor : ExecutorService = 
Executors.newFixedThreadPool(numberOfCores.toInt)
+try {
+  val carbonTable = 
carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+  val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+  val segmentStatusManager = new 
SegmentStatusManager(absoluteTableIdentifier)
+  val validSegments = 
segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala
+  val threadArray: Array[SplitThread] = new 
Array[SplitThread](validSegments.size)
+  var i = 0
+  for (segmentId: String <- validSegments) {
--- End diff --

use `map` or `forEach`


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-03 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131059444
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
 ---
@@ -345,6 +365,65 @@ object CarbonDataRDDFactory {
 compactionThread.run()
   }
 
+  case class SplitThread(sqlContext: SQLContext,
+  carbonLoadModel: CarbonLoadModel,
+  executor: ExecutorService,
+  storePath: String,
+  segmentId: String,
+  partitionId: String,
+  oldPartitionIdList: List[Int]) extends Thread {
+  override def run(): Unit = {
+try {
+  DataManagementFunc.executePartitionSplit(sqlContext,
+carbonLoadModel, executor, storePath, segmentId, partitionId,
+oldPartitionIdList)
+} catch {
+  case e: Exception =>
+LOGGER.error(s"Exception in partition split thread: ${ 
e.getMessage } }")
+}
+  }
+  }
+
+  def startSplitThreads(sqlContext: SQLContext,
+  carbonLoadModel: CarbonLoadModel,
+  storePath: String,
+  partitionId: String,
+  oldPartitionIdList: List[Int]): Unit = {
+val numberOfCores = CarbonProperties.getInstance()
+  .getProperty(CarbonCommonConstants.NUM_CORES_ALT_PARTITION,
+CarbonCommonConstants.DEFAULT_NUMBER_CORES)
+val executor : ExecutorService = 
Executors.newFixedThreadPool(numberOfCores.toInt)
+try {
+  val carbonTable = 
carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+  val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+  val segmentStatusManager = new 
SegmentStatusManager(absoluteTableIdentifier)
+  val validSegments = 
segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala
+  val threadArray: Array[SplitThread] = new 
Array[SplitThread](validSegments.size)
+  var i = 0
+  for (segmentId: String <- validSegments) {
--- End diff --

use `map` or `forEach`


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-02 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131058343
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
 ---
@@ -184,6 +189,161 @@ case class AlterTableCompaction(alterTableModel: 
AlterTableModel) extends Runnab
   }
 }
 
+/**
+ * Command for Alter Table Add & Split partition
+ * Add is a special case of Splitting the default partition (part0)
+ * @param alterTableSplitPartitionModel
+ */
+case class AlterTableSplitPartition(alterTableSplitPartitionModel: 
AlterTableSplitPartitionModel)
+  extends RunnableCommand {
+  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+
+val tableName = alterTableSplitPartitionModel.tableName
+val dbName = alterTableSplitPartitionModel.databaseName
+  .getOrElse(sparkSession.catalog.currentDatabase)
+val splitInfo = alterTableSplitPartitionModel.splitInfo
+val partitionId = 
Integer.parseInt(alterTableSplitPartitionModel.partitionId)
+val timestampFormatter = new 
SimpleDateFormat(CarbonProperties.getInstance
+  .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
+  .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+
+val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+  LockUsage.COMPACTION_LOCK,
+  LockUsage.DELETE_SEGMENT_LOCK,
+  LockUsage.DROP_TABLE_LOCK,
+  LockUsage.CLEAN_FILES_LOCK,
+  LockUsage.ALTER_PARTITION_LOCK)
+var locks = List.empty[ICarbonLock]
+try {
+  locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
+locksToBeAcquired)(sparkSession)
+  val carbonMetastore = 
CarbonEnv.getInstance(sparkSession).carbonMetastore
+  val relation = carbonMetastore.lookupRelation(Option(dbName), 
tableName)(sparkSession)
+.asInstanceOf[CarbonRelation]
+  val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+  val storePath = relation.tableMeta.storePath
+  if (relation == null) {
+sys.error(s"Table $dbName.$tableName does not exist")
+  }
+  carbonMetastore.checkSchemasModifiedTimeAndReloadTables(storePath)
+  if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + 
tableName)) {
+LOGGER.error(s"Alter table failed. table not found: 
$dbName.$tableName")
+sys.error(s"Alter table failed. table not found: 
$dbName.$tableName")
+  }
+  val carbonLoadModel = new CarbonLoadModel()
+
+  val table = relation.tableMeta.carbonTable
+  val partitionInfo = table.getPartitionInfo(tableName)
+  val partitionIdList = partitionInfo.getPartitionIds.asScala
+  // keep a copy of partitionIdList before update partitionInfo.
+  // will be used in partition data scan
+  val oldPartitionIdList: ArrayBuffer[Int] = new ArrayBuffer[Int]()
+  for (i: Integer <- partitionIdList) {
+oldPartitionIdList.append(i)
+  }
+
+  if (partitionInfo == null) {
+sys.error(s"Table $tableName is not a partition table.")
+  }
+  if (partitionInfo.getPartitionType == PartitionType.HASH) {
+sys.error(s"Hash partition table cannot be added or split!")
+  }
+  /**
+   * verify the add/split information and update the partitionInfo:
+   *  1. update rangeInfo/listInfo
+   *  2. update partitionIds
+   */
+  val columnDataType = 
partitionInfo.getColumnSchemaList.get(0).getDataType
+  val index = partitionIdList.indexOf(partitionId)
+  if (partitionInfo.getPartitionType == PartitionType.RANGE) {
+val rangeInfo = partitionInfo.getRangeInfo.asScala.toList
+val newRangeInfo = partitionId match {
+  case 0 => rangeInfo ++ splitInfo
+  case _ => rangeInfo.take(index - 1) ++ splitInfo ++
+rangeInfo.takeRight(rangeInfo.size - index)
+}
+CommonUtil.validateRangeInfo(newRangeInfo, columnDataType,
+  timestampFormatter, dateFormatter)
+partitionInfo.setRangeInfo(newRangeInfo.asJava)
+  } else if (partitionInfo.getPartitionType == PartitionType.LIST) {
+val originList = 
partitionInfo.getListInfo.asScala.map(_.asScala.toList).toList
+if (partitionId != 0) {
+  val targetListInfo = partitionInfo.getListInfo.get(index - 1)
+  

[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-02 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131058295
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
 ---
@@ -184,6 +189,161 @@ case class AlterTableCompaction(alterTableModel: 
AlterTableModel) extends Runnab
   }
 }
 
+/**
+ * Command for Alter Table Add & Split partition
+ * Add is a special case of Splitting the default partition (part0)
+ * @param alterTableSplitPartitionModel
+ */
+case class AlterTableSplitPartition(alterTableSplitPartitionModel: 
AlterTableSplitPartitionModel)
+  extends RunnableCommand {
+  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+
+val tableName = alterTableSplitPartitionModel.tableName
+val dbName = alterTableSplitPartitionModel.databaseName
+  .getOrElse(sparkSession.catalog.currentDatabase)
+val splitInfo = alterTableSplitPartitionModel.splitInfo
+val partitionId = 
Integer.parseInt(alterTableSplitPartitionModel.partitionId)
+val timestampFormatter = new 
SimpleDateFormat(CarbonProperties.getInstance
+  .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
+  .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+
+val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+  LockUsage.COMPACTION_LOCK,
+  LockUsage.DELETE_SEGMENT_LOCK,
+  LockUsage.DROP_TABLE_LOCK,
+  LockUsage.CLEAN_FILES_LOCK,
+  LockUsage.ALTER_PARTITION_LOCK)
+var locks = List.empty[ICarbonLock]
+try {
+  locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
+locksToBeAcquired)(sparkSession)
+  val carbonMetastore = 
CarbonEnv.getInstance(sparkSession).carbonMetastore
+  val relation = carbonMetastore.lookupRelation(Option(dbName), 
tableName)(sparkSession)
+.asInstanceOf[CarbonRelation]
+  val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+  val storePath = relation.tableMeta.storePath
+  if (relation == null) {
+sys.error(s"Table $dbName.$tableName does not exist")
+  }
+  carbonMetastore.checkSchemasModifiedTimeAndReloadTables(storePath)
+  if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + 
tableName)) {
+LOGGER.error(s"Alter table failed. table not found: 
$dbName.$tableName")
+sys.error(s"Alter table failed. table not found: 
$dbName.$tableName")
+  }
+  val carbonLoadModel = new CarbonLoadModel()
+
+  val table = relation.tableMeta.carbonTable
+  val partitionInfo = table.getPartitionInfo(tableName)
+  val partitionIdList = partitionInfo.getPartitionIds.asScala
+  // keep a copy of partitionIdList before update partitionInfo.
+  // will be used in partition data scan
+  val oldPartitionIdList: ArrayBuffer[Int] = new ArrayBuffer[Int]()
+  for (i: Integer <- partitionIdList) {
--- End diff --

do not use for loop, use `map` or `addAll`


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-02 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131058061
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/spliter/CarbonDataSpliterUtil.java
 ---
@@ -0,0 +1,40 @@
+/*
+ * 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.processing.spliter;
+
+import java.util.List;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.scan.result.BatchResult;
+
+public final class CarbonDataSpliterUtil {
--- End diff --

Is this needed?


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-02 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131057576
  
--- Diff: 
hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
 ---
@@ -321,6 +321,84 @@ private AbsoluteTableIdentifier 
getAbsoluteTableIdentifier(Configuration configu
   }
 
   /**
+   * Read data in one segment. For alter table partition statement
+   * @param job
+   * @param targetSegment
+   * @param oldPartitionIdList  get old partitionId before partitionInfo 
was changed
+   * @return
+   * @throws IOException
+   */
+  public List getSplitsOfOneSegment(JobContext job, String 
targetSegment,
+  List oldPartitionIdList, PartitionInfo partitionInfo)
+  throws IOException {
+AbsoluteTableIdentifier identifier = 
getAbsoluteTableIdentifier(job.getConfiguration());
+List invalidSegments = new ArrayList<>();
+List invalidTimestampsList = new ArrayList<>();
+
+List segmentList = new ArrayList<>();
+segmentList.add(targetSegment);
+setSegmentsToAccess(job.getConfiguration(), segmentList);
+try {
+
+  // process and resolve the expression
+  Expression filter = getFilterPredicates(job.getConfiguration());
+  CarbonTable carbonTable = 
getOrCreateCarbonTable(job.getConfiguration());
+  // this will be null in case of corrupt schema file.
+  if (null == carbonTable) {
+throw new IOException("Missing/Corrupt schema file for table.");
+  }
+
+  CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
+
+  // prune partitions for filter query on partition table
+  String partitionIds = job.getConfiguration().get(ALTER_PARTITION_ID);
+  BitSet matchedPartitions = null;
+  if (partitionInfo != null) {
+matchedPartitions = setMatchedPartitions(partitionIds, filter, 
partitionInfo);
+if (matchedPartitions != null) {
+  if (matchedPartitions.cardinality() == 0) {
+return new ArrayList();
+  } else if (matchedPartitions.cardinality() == 
partitionInfo.getNumPartitions()) {
+matchedPartitions = null;
+  }
+}
+  }
+
+  FilterResolverIntf filterInterface = 
CarbonInputFormatUtil.resolveFilter(filter, identifier);
+  // do block filtering and get split
+  List splits = getSplits(job, filterInterface, 
segmentList, matchedPartitions,
+  partitionInfo, oldPartitionIdList);
+  // pass the invalid segment to task side in order to remove index 
entry in task side
+  if (invalidSegments.size() > 0) {
+for (InputSplit split : splits) {
+  ((CarbonInputSplit) split).setInvalidSegments(invalidSegments);
+  ((CarbonInputSplit) 
split).setInvalidTimestampRange(invalidTimestampsList);
+}
+  }
+  return splits;
+} catch (IOException e) {
+  throw new RuntimeException("Can't get splits of the target segment 
", e);
+}
+  }
+
+  private BitSet setMatchedPartitions(String partitionIds, Expression 
filter,
+  PartitionInfo partitionInfo) {
+BitSet matchedPartitions = null;
+if (null != partitionIds) {
+  String[] partList = partitionIds.replace("[", "").replace("]", 
"").split(",");
+  matchedPartitions = new BitSet(Integer.parseInt(partList[0]));
--- End diff --

Can you add some comment here, why it is using `partList[0]`?


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-02 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131057207
  
--- Diff: 
hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
@@ -107,6 +107,7 @@
   // comma separated list of input files
   public static final String INPUT_FILES =
   "mapreduce.input.carboninputformat.files";
+  public static final String ALTER_PARTITION_ID = 
"mapreduce.input.carboninputformat.partitionid";
--- End diff --

modify in `CarbonTableInputFormat`, we will use it in the future


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-02 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131057020
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
 ---
@@ -101,17 +126,40 @@ object CarbonPartitionExample {
 spark.sql("""
| CREATE TABLE IF NOT EXISTS t5
| (
+   | id Int,
| vin String,
| logdate Timestamp,
| phonenumber Long,
-   | area String
+   | area String,
+   | salary Int
|)
| PARTITIONED BY (country String)
| STORED BY 'carbondata'
| TBLPROPERTIES('PARTITION_TYPE'='LIST',
-   | 'LIST_INFO'='(China,United States),UK ,japan,(Canada,Russia), 
South Korea ')
+   | 'LIST_INFO'='(China, US),UK ,Japan,(Canada,Russia, Good, 
NotGood), Korea ')
""".stripMargin)
 
+// load data into partition table
+spark.sql(s"""
+   LOAD DATA LOCAL INPATH '$testData' into table t0 
options('BAD_RECORDS_ACTION'='FORCE')
+   """)
+spark.sql(s"""
+   LOAD DATA LOCAL INPATH '$testData' into table t5 
options('BAD_RECORDS_ACTION'='FORCE')
+   """)
+
+// alter list partition table t5 to add a partition
+spark.sql(s"""Alter table t5 add partition ('OutSpace')""".stripMargin)
+// alter list partition table t5 to split partition 4 into 3 
independent partition
+spark.sql(
+  s"""
+ Alter table t5 split partition(4) into ('Canada', 'Russia', 
'(Good, NotGood)')
+   """.stripMargin)
--- End diff --

after doing this, can you verify the partition is correct by using desc 
formatted?


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-08-02 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r131056387
  
--- Diff: conf/carbon.properties.template ---
@@ -42,6 +42,9 @@ carbon.enableXXHash=true
 #carbon.max.level.cache.size=-1
 #enable prefetch of data during merge sort while reading data from sort 
temp files in data loading
 #carbon.merge.sort.prefetch=true
+ Alter Partition Configuration 
+#Number of cores to be used while alter partition
--- End diff --

Can you explain in document for what purpose these cores are required?


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-31 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r130506636
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
 ---
@@ -65,6 +65,31 @@ public PartitionInfo(List 
columnSchemaList, PartitionType partitio
 this.partitionIds = new ArrayList<>();
   }
 
+  /**
+   * add partition means split default partition, add in last directly
+   */
+  public void  addPartition(int addPartitionCount) {
+for (int i = 0; i < addPartitionCount; i++) {
+  partitionIds.add(++MAX_PARTITION);
+  numPartitions++;
+}
+  }
+
+  /**
+   * e.g. original partition[0,1,2,3,4,5]
+   * split partition 2 to partition 6,7,8 (will not reuse 2)
+   * then sourcePartitionId is 2, newPartitionNumbers is 3
+   * @param sourcePartitionIndex
+   * @param newPartitionNumbers
+   */
+  public void splitPartition(int sourcePartitionIndex, int 
newPartitionNumbers) {
--- End diff --

"When will partition-split action be triggered?"
Please refer to my description in 'Conversation' Tab, user can execute 
alter table add or split statement.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-31 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r130506012
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java ---
@@ -303,6 +303,20 @@ public String getCarbonIndexFilePath(String taskId, 
String partitionId, String s
 }
   }
 
+  public String getCarbonIndexFilePath(String taskId, String partitionId, 
String segmentId,
+  int batchNo, String bucketNumber, String timeStamp,
+  ColumnarFormatVersion columnarFormatVersion) {
+switch (columnarFormatVersion) {
+  case V1:
+  case V2:
+return getCarbonIndexFilePath(taskId, partitionId, segmentId, 
bucketNumber);
+  default:
--- End diff --

V3 will go as default, this function is not created by myself, just added 
batchNo based on original one to get correct index path.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-31 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r130505076
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
 ---
@@ -65,6 +65,31 @@ public PartitionInfo(List 
columnSchemaList, PartitionType partitio
 this.partitionIds = new ArrayList<>();
   }
 
+  /**
+   * add partition means split default partition, add in last directly
+   */
+  public void  addPartition(int addPartitionCount) {
+for (int i = 0; i < addPartitionCount; i++) {
+  partitionIds.add(++MAX_PARTITION);
+  numPartitions++;
+}
+  }
+
+  /**
+   * e.g. original partition[0,1,2,3,4,5]
+   * split partition 2 to partition 6,7,8 (will not reuse 2)
+   * then sourcePartitionId is 2, newPartitionNumbers is 3
+   * @param sourcePartitionIndex
+   * @param newPartitionNumbers
+   */
+  public void splitPartition(int sourcePartitionIndex, int 
newPartitionNumbers) {
--- End diff --

Carbon partition design is different from Hive partition, it's not based on 
path but blocks. Split means add new partition between existed partitions for 
range partition table. For list partition table, split allow user to reduce the 
partition data volume.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-31 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r130504413
  
--- Diff: 
hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
@@ -440,9 +510,17 @@ protected Expression getFilterPredicates(Configuration 
configuration) {
 for (Map.Entry entry :
 segmentIndexMap.entrySet()) {
   SegmentTaskIndexStore.TaskBucketHolder taskHolder = 
entry.getKey();
-  int taskId = 
CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskHolder.taskNo);
+  int partitionId = 
CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskHolder.taskNo);
--- End diff --

Yes, for partition table, we use partitionId as taskId


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-31 Thread xuchuanyin
Github user xuchuanyin commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r130501996
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java ---
@@ -303,6 +303,20 @@ public String getCarbonIndexFilePath(String taskId, 
String partitionId, String s
 }
   }
 
+  public String getCarbonIndexFilePath(String taskId, String partitionId, 
String segmentId,
+  int batchNo, String bucketNumber, String timeStamp,
+  ColumnarFormatVersion columnarFormatVersion) {
+switch (columnarFormatVersion) {
+  case V1:
+  case V2:
+return getCarbonIndexFilePath(taskId, partitionId, segmentId, 
bucketNumber);
+  default:
--- End diff --

Whats the different between V1/V2 and default? 

Does V3 acts the same as default?


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-31 Thread xuchuanyin
Github user xuchuanyin commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r130501212
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
 ---
@@ -65,6 +65,31 @@ public PartitionInfo(List 
columnSchemaList, PartitionType partitio
 this.partitionIds = new ArrayList<>();
   }
 
+  /**
+   * add partition means split default partition, add in last directly
+   */
+  public void  addPartition(int addPartitionCount) {
+for (int i = 0; i < addPartitionCount; i++) {
+  partitionIds.add(++MAX_PARTITION);
+  numPartitions++;
+}
+  }
+
+  /**
+   * e.g. original partition[0,1,2,3,4,5]
+   * split partition 2 to partition 6,7,8 (will not reuse 2)
+   * then sourcePartitionId is 2, newPartitionNumbers is 3
+   * @param sourcePartitionIndex
+   * @param newPartitionNumbers
+   */
+  public void splitPartition(int sourcePartitionIndex, int 
newPartitionNumbers) {
--- End diff --

When will partition-split action be triggered?

In my memory, there is no “partition-split” action in Hive, etc. 


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-31 Thread xuchuanyin
Github user xuchuanyin commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r130503124
  
--- Diff: 
hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
@@ -440,9 +510,17 @@ protected Expression getFilterPredicates(Configuration 
configuration) {
 for (Map.Entry entry :
 segmentIndexMap.entrySet()) {
   SegmentTaskIndexStore.TaskBucketHolder taskHolder = 
entry.getKey();
-  int taskId = 
CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskHolder.taskNo);
+  int partitionId = 
CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskHolder.taskNo);
--- End diff --

**partitionId**...get**TaskId**FromTaskNo?

Is this OK?


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129744678
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ---
@@ -308,6 +308,10 @@
   @CarbonProperty
   public static final String NUM_CORES_COMPACTING = 
"carbon.number.of.cores.while.compacting";
   /**
+   * Number of cores to be used while alter partition
+   */
+  public static final String NUM_CORES_ALT_PARTITION = 
"carbon.number.of.cores.while.altPartition";
+  /**
--- End diff --

No space line in other variables here, so keep one style.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129740890
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
 ---
@@ -101,17 +126,40 @@ object CarbonPartitionExample {
 spark.sql("""
| CREATE TABLE IF NOT EXISTS t5
| (
+   | id Int,
| vin String,
| logdate Timestamp,
| phonenumber Long,
-   | area String
+   | area String,
+   | salary Int
|)
| PARTITIONED BY (country String)
| STORED BY 'carbondata'
| TBLPROPERTIES('PARTITION_TYPE'='LIST',
-   | 'LIST_INFO'='(China,United States),UK ,japan,(Canada,Russia), 
South Korea ')
+   | 'LIST_INFO'='(China, US),UK ,Japan,(Canada,Russia, Good, 
NotGood), Korea ')
--- End diff --

Hi @chenerlu , here in DDL statement, it's designed to leave no space to 
mock real situation which could happen in customer writing.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129598686
  
--- Diff: 
hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
@@ -440,9 +510,16 @@ protected Expression getFilterPredicates(Configuration 
configuration) {
 for (Map.Entry entry :
 segmentIndexMap.entrySet()) {
   SegmentTaskIndexStore.TaskBucketHolder taskHolder = 
entry.getKey();
-  int taskId = 
CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskHolder.taskNo);
+  int partitionId = 
CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskHolder.taskNo);
+  //oldPartitionIdList is only used in alter table partition 
command because it change
+  //partition info first and then read data.
+  //for other normal query should use newest partitionIdList
--- End diff --

use /** */ instead if multi line notes.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129598531
  
--- Diff: 
hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
@@ -440,9 +510,16 @@ protected Expression getFilterPredicates(Configuration 
configuration) {
 for (Map.Entry entry :
 segmentIndexMap.entrySet()) {
   SegmentTaskIndexStore.TaskBucketHolder taskHolder = 
entry.getKey();
-  int taskId = 
CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskHolder.taskNo);
+  int partitionId = 
CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskHolder.taskNo);
+  //oldPartitionIdList is only used in alter table partition 
command because it change
+  //partition info first and then read data.
+  //for other normal query should use newest partitionIdList
--- End diff --

use /** */  instead


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129584623
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
 ---
@@ -101,17 +126,40 @@ object CarbonPartitionExample {
 spark.sql("""
| CREATE TABLE IF NOT EXISTS t5
| (
+   | id Int,
| vin String,
| logdate Timestamp,
| phonenumber Long,
-   | area String
+   | area String,
+   | salary Int
|)
| PARTITIONED BY (country String)
| STORED BY 'carbondata'
| TBLPROPERTIES('PARTITION_TYPE'='LIST',
-   | 'LIST_INFO'='(China,United States),UK ,japan,(Canada,Russia), 
South Korea ')
+   | 'LIST_INFO'='(China, US),UK ,Japan,(Canada,Russia, Good, 
NotGood), Korea ')
--- End diff --

add space before ,


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129583765
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
 ---
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.scan.result.iterator;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.scan.result.BatchResult;
+
+
+public class PartitionSpliterRawResultIterator extends 
CarbonIterator {
+
+  private CarbonIterator iterator;
+  private BatchResult batch;
+  private int counter;
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(PartitionSpliterRawResultIterator.class.getName());
+
+  public PartitionSpliterRawResultIterator(CarbonIterator 
iterator) {
+this.iterator = iterator;
+  }
+
+
+  @Override public boolean hasNext() {
+if (null == batch || checkBatchEnd(batch)) {
+  if (iterator.hasNext()) {
+batch = iterator.next();
+counter = 0;
+  } else {
+return false;
+  }
+}
+
+if (!checkBatchEnd(batch)) {
+  return true;
+} else {
+  return false;
+}
+  }
+
+  @Override public Object[] next() {
+if (batch == null) {
+  batch = iterator.next();
+}
+if (!checkBatchEnd(batch)) {
+  try {
+return batch.getRawRow(counter++);
+  } catch (Exception e) {
+LOGGER.error(e.getMessage());
+return null;
+  }
+} else {
+  batch = iterator.next();
+  counter = 0;
+}
+try {
+  return batch.getRawRow(counter++);
+} catch (Exception e) {
+  LOGGER.error(e.getMessage());
+  return null;
--- End diff --

This logical can be optimized.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129583246
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
 ---
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.scan.result.iterator;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.scan.result.BatchResult;
+
+
+public class PartitionSpliterRawResultIterator extends 
CarbonIterator {
+
+  private CarbonIterator iterator;
+  private BatchResult batch;
+  private int counter;
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(PartitionSpliterRawResultIterator.class.getName());
+
+  public PartitionSpliterRawResultIterator(CarbonIterator 
iterator) {
+this.iterator = iterator;
+  }
+
+
+  @Override public boolean hasNext() {
+if (null == batch || checkBatchEnd(batch)) {
+  if (iterator.hasNext()) {
+batch = iterator.next();
+counter = 0;
+  } else {
+return false;
+  }
+}
+
+if (!checkBatchEnd(batch)) {
+  return true;
+} else {
+  return false;
+}
--- End diff --

use return !checkBatchEnd(batch) instead.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129583064
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
 ---
@@ -65,6 +65,31 @@ public PartitionInfo(List 
columnSchemaList, PartitionType partitio
 this.partitionIds = new ArrayList<>();
   }
 
+  /**
+   * add partition means split default partition, add in last directly
--- End diff --

because maybe there is data existed in default partition need to be filled 
in new partition


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129582365
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
 ---
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.scan.result.iterator;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.scan.result.BatchResult;
+
+
--- End diff --

delete space line.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129582258
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
 ---
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.scan.result.iterator;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.scan.result.BatchResult;
+
+
+public class PartitionSpliterRawResultIterator extends 
CarbonIterator {
+
+  private CarbonIterator iterator;
+  private BatchResult batch;
+  private int counter;
+
+  /**
+   * LOGGER
+   */
--- End diff --

I think this is not necessary.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129582111
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
 ---
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.scan.result.iterator;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.scan.result.BatchResult;
+
+
+public class PartitionSpliterRawResultIterator extends 
CarbonIterator {
+
+  private CarbonIterator iterator;
+  private BatchResult batch;
+  private int counter;
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(PartitionSpliterRawResultIterator.class.getName());
+
+  public PartitionSpliterRawResultIterator(CarbonIterator 
iterator) {
+this.iterator = iterator;
+  }
+
+
--- End diff --

delete useless space line


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129534273
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
 ---
@@ -65,6 +65,31 @@ public PartitionInfo(List 
columnSchemaList, PartitionType partitio
 this.partitionIds = new ArrayList<>();
   }
 
+  /**
+   * add partition means split default partition, add in last directly
--- End diff --

default partition is 0, so why split partition ?


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129533186
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ---
@@ -308,6 +308,10 @@
   @CarbonProperty
   public static final String NUM_CORES_COMPACTING = 
"carbon.number.of.cores.while.compacting";
   /**
+   * Number of cores to be used while alter partition
+   */
+  public static final String NUM_CORES_ALT_PARTITION = 
"carbon.number.of.cores.while.altPartition";
+  /**
--- End diff --

Add spaceline


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129516143
  
--- Diff: 
integration/spark-common-test/src/test/resources/partition_data.csv ---
@@ -0,0 +1,27 @@
+id,vin,logdate,phonenumber,country,area,salary
--- End diff --

Oh, this file is copied from example package. Maybe I can reduce them and 
keep only one.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129515426
  
--- Diff: conf/carbon.properties.template ---
@@ -42,6 +42,9 @@ carbon.enableXXHash=true
 #carbon.max.level.cache.size=-1
 #enable prefetch of data during merge sort while reading data from sort 
temp files in data loading
 #carbon.merge.sort.prefetch=true
+ Alter Partition Configuration 
+#Number of cores to be used while alter partition
+carbon.number.of.cores.while.altPartition=2
--- End diff --

Yes, it will be used when take action of multiple segments in parallel. 
this configuration will allow user to set the threads according to their 
hardware.
Sure, I will make the change.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread lionelcao
Github user lionelcao commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129514953
  
--- Diff: 
integration/spark-common-test/src/test/resources/partition_data.csv ---
@@ -0,0 +1,27 @@
+id,vin,logdate,phonenumber,country,area,salary
--- End diff --

Hi @chenliang613 this csv data is already existed for partition example and 
test case. It's simple and clear to understand the partition concept. this PR 
just added two columns.


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenliang613
Github user chenliang613 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129514137
  
--- Diff: conf/carbon.properties.template ---
@@ -42,6 +42,9 @@ carbon.enableXXHash=true
 #carbon.max.level.cache.size=-1
 #enable prefetch of data during merge sort while reading data from sort 
temp files in data loading
 #carbon.merge.sort.prefetch=true
+ Alter Partition Configuration 
+#Number of cores to be used while alter partition
+carbon.number.of.cores.while.altPartition=2
--- End diff --

1. Please check whether the parameter  
"carbon.number.of.cores.while.altPartition=2" is necessary , or not ?
2. If yes, suggest directly using : 
carbon.number.of.cores.while.alterPartition


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


[GitHub] carbondata pull request #1192: [CARBONDATA-940] alter table add/split partit...

2017-07-26 Thread chenliang613
Github user chenliang613 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1192#discussion_r129513477
  
--- Diff: 
integration/spark-common-test/src/test/resources/partition_data.csv ---
@@ -0,0 +1,27 @@
+id,vin,logdate,phonenumber,country,area,salary
--- End diff --

can you try to reuse the current csv files or generate data. 
Don't suggest adding so many csv file to repo.


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