[GitHub] carbondata pull request #1042: [CARBONDATA-1181] Show partitions

2017-06-24 Thread mayunSaicmotor
Github user mayunSaicmotor closed the pull request at:

https://github.com/apache/carbondata/pull/1042


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-24 Thread mayunSaicmotor
Github user mayunSaicmotor commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123875651
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/ShowPartitionInfoExample.scala
 ---
@@ -0,0 +1,110 @@
+/*
+ * 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.examples
+
+import java.io.File
+
+import org.apache.spark.sql.SparkSession
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+object ShowPartitionInfoExample {
--- End diff --

changed


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-23 Thread mayunSaicmotor
Github user mayunSaicmotor commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123868164
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
 ---
@@ -203,6 +205,7 @@ case class CarbonIUDAnalysisRule(sparkSession: 
SparkSession) extends Rule[Logica
 logicalplan transform {
   case UpdateTable(t, cols, sel, where) => processUpdateQuery(t, cols, 
sel, where)
   case DeleteRecords(statement, table) => 
processDeleteRecordsQuery(statement, table)
+  case ShowPartitionsCommand(t, cols) => ShowCarbonPartitionsCommand(t)
--- End diff --

do you mean move this to DDLStrategy?


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-23 Thread mayunSaicmotor
Github user mayunSaicmotor commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123867997
  
--- Diff: 
integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
 ---
@@ -170,6 +171,7 @@ object CarbonIUDAnalysisRule extends Rule[LogicalPlan] {
 logicalplan transform {
   case UpdateTable(t, cols, sel, where) => processUpdateQuery(t, cols, 
sel, where)
   case DeleteRecords(statement, table) => 
processDeleteRecordsQuery(statement, table)
+  case ShowPartitions(t) => ShowCarbonPartitionsCommand(t)
--- End diff --

this code is for spark 1.6 version, not spark2, moreover there is no 
DDLStrategy for spark1.6


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123464378
  
--- Diff: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/ShowCarbonPartitionsCommand.scala
 ---
@@ -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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Value", StringType, nullable = true,
+  new MetadataBuilder().putString("comment", "partition 
value").build())()
+  )
+  override def run(sqlContext: SQLContext): Seq[Row] = {
+val relation = CarbonEnv.get.carbonMetastore
+  .lookupRelation1(tableIdentifier)(sqlContext).
+  asInstanceOf[CarbonRelation]
+val carbonTable = relation.tableMeta.carbonTable
+var partitionInfo = carbonTable.getPartitionInfo(
--- End diff --

throw an exception if the table is not partitioned


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123463537
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/ShowPartitionInfoExample.scala
 ---
@@ -0,0 +1,110 @@
+/*
+ * 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.examples
+
+import java.io.File
+
+import org.apache.spark.sql.SparkSession
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+object ShowPartitionInfoExample {
--- End diff --

No need to add a new example. Write one example for "show partition" in 
CarbonPartitionExample


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123466148
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/ShowCarbonPartitionsCommand.scala
 ---
@@ -0,0 +1,98 @@
+/*
+ * 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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Name", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
name").build())(),
+AttributeReference("Value", StringType, nullable = true,
+  new MetadataBuilder().putString("comment", "partition 
value").build())()
+  )
+  override def run(sparkSession: SparkSession): Seq[Row] = {
--- End diff --

This code is almost same for 1.6 and 2.1. Can you move the common code to 
CommonUtil


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123465927
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/ShowCarbonPartitionsCommand.scala
 ---
@@ -0,0 +1,98 @@
+/*
+ * 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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Name", StringType, nullable = false,
--- End diff --

we don't need 3 columns.
Hive shows just one column and i think we should follow the same.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123466992
  
--- Diff: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/ShowCarbonPartitionsCommand.scala
 ---
@@ -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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Value", StringType, nullable = true,
+  new MetadataBuilder().putString("comment", "partition 
value").build())()
+  )
+  override def run(sqlContext: SQLContext): Seq[Row] = {
+val relation = CarbonEnv.get.carbonMetastore
+  .lookupRelation1(tableIdentifier)(sqlContext).
+  asInstanceOf[CarbonRelation]
+val carbonTable = relation.tableMeta.carbonTable
+var partitionInfo = carbonTable.getPartitionInfo(
+  
carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
+var partitionType = partitionInfo.getPartitionType
+var result = Seq.newBuilder[Row]
+columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
+LOGGER.info("partition column name:" + columnName)
+partitionType match {
+  case PartitionType.RANGE =>
+result.+=(RowFactory.create("0", "default"))
+var id = 1
+var rangeInfo = partitionInfo.getRangeInfo
+var size = rangeInfo.size() - 1
+for (index <- 0 to size) {
+  result.+=(RowFactory.create(id.toString(), "< " + 
rangeInfo.get(index)))
+  id += 1
+}
+  case PartitionType.RANGE_INTERVAL =>
+result.+=(RowFactory.create("", ""))
+  case PartitionType.LIST =>
+result.+=(RowFactory.create("0", "default"))
--- End diff --

for list:
column_name = 1,3,5,7,10


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123463416
  
--- Diff: 
examples/spark/src/main/scala/org/apache/carbondata/examples/ShowPartitionInfoExample.scala
 ---
@@ -0,0 +1,106 @@
+/*
+ * 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.examples
+
+import scala.collection.mutable.LinkedHashMap
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.util.ExampleUtils
+
+object ShowPartitionInfoExample {
--- End diff --

No need to add a new example. Write one example for "show partition" in 
CarbonPartitionExample


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123465686
  
--- Diff: 
integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
 ---
@@ -170,6 +171,7 @@ object CarbonIUDAnalysisRule extends Rule[LogicalPlan] {
 logicalplan transform {
   case UpdateTable(t, cols, sel, where) => processUpdateQuery(t, cols, 
sel, where)
   case DeleteRecords(statement, table) => 
processDeleteRecordsQuery(statement, table)
+  case ShowPartitions(t) => ShowCarbonPartitionsCommand(t)
--- End diff --

This should not be here, handle this in DDLStrategy. 
Spark2 already parses the "show partition" command for us so we can match 
the same in DDLStrategy.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123466861
  
--- Diff: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/ShowCarbonPartitionsCommand.scala
 ---
@@ -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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Value", StringType, nullable = true,
+  new MetadataBuilder().putString("comment", "partition 
value").build())()
+  )
+  override def run(sqlContext: SQLContext): Seq[Row] = {
+val relation = CarbonEnv.get.carbonMetastore
+  .lookupRelation1(tableIdentifier)(sqlContext).
+  asInstanceOf[CarbonRelation]
+val carbonTable = relation.tableMeta.carbonTable
+var partitionInfo = carbonTable.getPartitionInfo(
+  
carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
+var partitionType = partitionInfo.getPartitionType
+var result = Seq.newBuilder[Row]
+columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
+LOGGER.info("partition column name:" + columnName)
+partitionType match {
+  case PartitionType.RANGE =>
+result.+=(RowFactory.create("0", "default"))
+var id = 1
+var rangeInfo = partitionInfo.getRangeInfo
+var size = rangeInfo.size() - 1
+for (index <- 0 to size) {
+  result.+=(RowFactory.create(id.toString(), "< " + 
rangeInfo.get(index)))
--- End diff --

can we have the partitions be shown like this for range:
1<= column_name < 5
5<= column_name < 10



---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123467182
  
--- Diff: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/ShowCarbonPartitionsCommand.scala
 ---
@@ -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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Value", StringType, nullable = true,
+  new MetadataBuilder().putString("comment", "partition 
value").build())()
+  )
+  override def run(sqlContext: SQLContext): Seq[Row] = {
+val relation = CarbonEnv.get.carbonMetastore
+  .lookupRelation1(tableIdentifier)(sqlContext).
+  asInstanceOf[CarbonRelation]
+val carbonTable = relation.tableMeta.carbonTable
+var partitionInfo = carbonTable.getPartitionInfo(
+  
carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
+var partitionType = partitionInfo.getPartitionType
+var result = Seq.newBuilder[Row]
+columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
+LOGGER.info("partition column name:" + columnName)
+partitionType match {
+  case PartitionType.RANGE =>
+result.+=(RowFactory.create("0", "default"))
+var id = 1
+var rangeInfo = partitionInfo.getRangeInfo
+var size = rangeInfo.size() - 1
+for (index <- 0 to size) {
+  result.+=(RowFactory.create(id.toString(), "< " + 
rangeInfo.get(index)))
+  id += 1
+}
+  case PartitionType.RANGE_INTERVAL =>
+result.+=(RowFactory.create("", ""))
+  case PartitionType.LIST =>
+result.+=(RowFactory.create("0", "default"))
+var id = 1
+var listInfo = partitionInfo.getListInfo
+var size = listInfo.size() - 1
+for (index <- 0 to size) {
+  var listStr = ""
+  listInfo.get(index).toArray().foreach { x =>
+if (listStr.isEmpty()) {
+  listStr = x.toString()
+} else {
+  listStr += ", " + x.toString()
+}
+  }
+  result.+=(RowFactory.create(id.toString(), listStr))
+  id += 1
+}
+  case PartitionType.HASH =>
+var hashNumber = partitionInfo.getNumPartitions
+result.+=(RowFactory.create("HASH PARTITION", 
hashNumber.toString()))
--- End diff --

for hash:
column_name = HASH_NUMBER(num)


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123466013
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
 ---
@@ -203,6 +205,7 @@ case class CarbonIUDAnalysisRule(sparkSession: 
SparkSession) extends Rule[Logica
 logicalplan transform {
   case UpdateTable(t, cols, sel, where) => processUpdateQuery(t, cols, 
sel, where)
   case DeleteRecords(statement, table) => 
processDeleteRecordsQuery(statement, table)
+  case ShowPartitionsCommand(t, cols) => ShowCarbonPartitionsCommand(t)
--- End diff --

Not to be handled here


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-22 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r123463948
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
 ---
@@ -0,0 +1,157 @@
+/*
+ * 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.testsuite.partition
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+class TestShowPartition  extends QueryTest with BeforeAndAfterAll {
+  override def beforeAll = {
+
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, 
"dd-MM-")
+
+  }
+
+  test("show partition table: hash table") {
+sql(
+  """
+| CREATE TABLE hashTable (empname String, designation String, doj 
Timestamp,
--- End diff --

1. Write the create statement in beforeAll()
2. 1 or 2 columns are enough
3. no need to load data


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-20 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122897963
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/ShowCarbonPartitionsCommand.scala
 ---
@@ -0,0 +1,98 @@
+/*
+ * 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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Name", StringType, nullable = false,
--- End diff --

We can reach an agreement for keeping Name or not.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-20 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122897255
  
--- Diff: 
examples/spark/src/main/scala/org/apache/carbondata/examples/ShowPartitionInfoExample.scala
 ---
@@ -0,0 +1,108 @@
+/*
+ * 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.examples
+
+import scala.collection.mutable.LinkedHashMap
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.util.ExampleUtils
+
+object ShowPartitionInfoExample {
+  def main(args: Array[String]) {
+ShowPartitionInfoExample.extracted("t3", args)
+  }
+  def extracted(tableName: String, args: Array[String]): Unit = {
+val cc = ExampleUtils.createCarbonContext("CarbonShowPartitionInfo")
+val testData = ExampleUtils.currentPath + 
"/src/main/resources/data.csv"
+
+// range partition
+cc.sql("DROP TABLE IF EXISTS t1")
+
+cc.sql("""
+| CREATE TABLE IF NOT EXISTS t1
+| (
--- End diff --

If you can put "(" after t1, I think it will be better.
Still, you can keep your code 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 #1042: [CARBONDATA-1181] Show partitions

2017-06-20 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122896730
  
--- Diff: 
examples/spark/src/main/scala/org/apache/carbondata/examples/ShowPartitionInfoExample.scala
 ---
@@ -0,0 +1,108 @@
+/*
+ * 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.examples
+
+import scala.collection.mutable.LinkedHashMap
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.util.ExampleUtils
+
+object ShowPartitionInfoExample {
+  def main(args: Array[String]) {
+ShowPartitionInfoExample.extracted("t3", args)
+  }
+  def extracted(tableName: String, args: Array[String]): Unit = {
+val cc = ExampleUtils.createCarbonContext("CarbonShowPartitionInfo")
+val testData = ExampleUtils.currentPath + 
"/src/main/resources/data.csv"
+
+// range partition
+cc.sql("DROP TABLE IF EXISTS t1")
+
+cc.sql("""
+| CREATE TABLE IF NOT EXISTS t1
+| (
+| vin STRING,
+| phonenumber INT,
+| country STRING,
+| area STRING
+| )
+| PARTITIONED BY (logdate TIMESTAMP)
+| STORED BY 'carbondata'
+| TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+| 'RANGE_INFO'='2014/01/01,2015/01/01,2016/01/01')
--- End diff --

Better set timestamp format for this. like 
CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,"/MM/dd")


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122864627
  
--- Diff: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/ShowPartitionsCommand.scala
 ---
@@ -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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
--- End diff --

Why  this class name is different from class name ?
Just my doubt.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122864102
  
--- Diff: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/ShowPartitionsCommand.scala
 ---
@@ -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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Name", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
name").build())(),
+AttributeReference("Value(" + columnName + "=)", StringType, nullable 
= true,
+  new MetadataBuilder().putString("comment", "partition 
value").build())()
+  )
+  override def run(sqlContext: SQLContext): Seq[Row] = {
+val relation = CarbonEnv.get.carbonMetastore
+  .lookupRelation1(tableIdentifier)(sqlContext).
+  asInstanceOf[CarbonRelation]
+val carbonTable = relation.tableMeta.carbonTable
+var partitionInfo = carbonTable.getPartitionInfo(
+  
carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
+var partitionType = partitionInfo.getPartitionType
+var result = Seq.newBuilder[Row]
+columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
+if (PartitionType.RANGE.equals(partitionType)) {
+  result.+=(RowFactory.create("0", "", "default"))
+  var id = 1
+  // var name = "partition_"
--- End diff --

delete useless 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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122864076
  
--- Diff: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/ShowPartitionsCommand.scala
 ---
@@ -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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Name", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
name").build())(),
+AttributeReference("Value(" + columnName + "=)", StringType, nullable 
= true,
+  new MetadataBuilder().putString("comment", "partition 
value").build())()
+  )
+  override def run(sqlContext: SQLContext): Seq[Row] = {
--- End diff --

Suggest use case match for different partition type


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122863911
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/ShowPartitionsCommand.scala
 ---
@@ -0,0 +1,96 @@
+/*
+ * 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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Name", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
name").build())(),
+AttributeReference("Value(" + columnName + "=)", StringType, nullable 
= true,
+  new MetadataBuilder().putString("comment", "partition 
value").build())()
+  )
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+  .lookupRelation(tableIdentifier)(sparkSession).
+  asInstanceOf[CarbonRelation]
+val carbonTable = relation.tableMeta.carbonTable
+var partitionInfo = carbonTable.getPartitionInfo(
+  
carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
+var partitionType = partitionInfo.getPartitionType
+var result = Seq.newBuilder[Row]
+columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
+if (PartitionType.RANGE.equals(partitionType)) {
+  result.+=(RowFactory.create("0", "", "default"))
+  var id = 1
+  // var name = "partition_"
--- End diff --

delete useless 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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122863841
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/ShowPartitionsCommand.scala
 ---
@@ -0,0 +1,96 @@
+/*
+ * 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.spark.sql.execution.command
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+
+
+
+private[sql] case class ShowCarbonPartitionsCommand(
+tableIdentifier: TableIdentifier) extends RunnableCommand {
+  val LOGGER = 
LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+  var columnName = ""
+  override val output: Seq[Attribute] = Seq(
+// Column names are based on Hive.
+AttributeReference("ID", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
id").build())(),
+AttributeReference("Name", StringType, nullable = false,
+  new MetadataBuilder().putString("comment", "partition 
name").build())(),
+AttributeReference("Value(" + columnName + "=)", StringType, nullable 
= true,
+  new MetadataBuilder().putString("comment", "partition 
value").build())()
+  )
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
--- End diff --

Suggest use case match for different partition type 


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122717829
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * 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.testsuite.partition
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+
+class TestShowPartition  extends QueryTest with BeforeAndAfterAll {
+  override def beforeAll = {
+dropTable
+
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, 
"dd-MM-")
+
+  }
+
+  test("show partition table: hash table") {
+sql(
+  """
+| CREATE TABLE hashTable (empname String, designation String, doj 
Timestamp,
+|  workgroupcategory int, workgroupcategoryname String, deptno 
int, deptname String,
+|  projectcode int, projectjoindate Timestamp, projectenddate 
Timestamp,attendance int,
+|  utilization int,salary int)
+| PARTITIONED BY (empno int)
+| STORED BY 'org.apache.carbondata.format'
+| TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
+  """.stripMargin)
+sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE 
hashTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+
+// EqualTo
+checkAnswer(sql("show partitions hashTable"), Seq(Row("HASH 
PARTITION", "", "3")))
+
+sql("drop table hashTable")
+  }
+
+  test("show partition table: range partition") {
+sql(
+  """
+| CREATE TABLE rangeTable (empno int, empname String, designation 
String,
+|  workgroupcategory int, workgroupcategoryname String, deptno 
int, deptname String,
+|  projectcode int, projectjoindate Timestamp, projectenddate 
Timestamp,attendance int,
+|  utilization int,salary int)
+| PARTITIONED BY (doj Timestamp)
+| STORED BY 'org.apache.carbondata.format'
+| TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+|  'RANGE_INFO'='01-01-2010, 01-01-2015')
+  """.stripMargin)
+sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE 
rangeTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+
+// EqualTo
+checkAnswer(sql("show partitions rangeTable"), Seq(Row("0", "", 
"default"), Row("1", "", "< 01-01-2010"), Row("2", "", "< 01-01-2015")))
+sql("drop table rangeTable")
+  }
+
+  test("show partition table: list partition") {
+sql(
+  """
+| CREATE TABLE listTable (empno int, empname String, designation 
String, doj Timestamp,
+|  workgroupcategoryname String, deptno int, deptname String,
+|  projectcode int, projectjoindate Timestamp, projectenddate 
Timestamp,attendance int,
+|  utilization int,salary int)
+| PARTITIONED BY (workgroupcategory int)
+| STORED BY 'org.apache.carbondata.format'
+| TBLPROPERTIES('PARTITION_TYPE'='LIST',
+|  'LIST_INFO'='0, 1, (2, 3)')
+  """.stripMargin)
+sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE 
listTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+
+// EqualTo
+checkAnswer(sql("show partitions listTable"), Seq(Row("0", "", "0"), 
Row("1", "", "1"), Row("2", "", "2, 3")))
+
+  sql("drop table listTable")
+  }
+  test("show partition table: not default db") {
+sql(s"CREATE DATABASE if not exists partitionDB")
+
+sql(
+  """
+| CREATE TABLE partitionDB.listTable (empno int, empname St

[GitHub] carbondata pull request #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122717016
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * 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.testsuite.partition
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+
+class TestShowPartition  extends QueryTest with BeforeAndAfterAll {
+  override def beforeAll = {
+dropTable
+
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, 
"dd-MM-")
+
+  }
+
+  test("show partition table: hash table") {
+sql(
+  """
+| CREATE TABLE hashTable (empname String, designation String, doj 
Timestamp,
+|  workgroupcategory int, workgroupcategoryname String, deptno 
int, deptname String,
+|  projectcode int, projectjoindate Timestamp, projectenddate 
Timestamp,attendance int,
+|  utilization int,salary int)
+| PARTITIONED BY (empno int)
+| STORED BY 'org.apache.carbondata.format'
+| TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
+  """.stripMargin)
+sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE 
hashTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+
+// EqualTo
+checkAnswer(sql("show partitions hashTable"), Seq(Row("HASH 
PARTITION", "", "3")))
+
+sql("drop table hashTable")
+  }
+
+  test("show partition table: range partition") {
+sql(
+  """
+| CREATE TABLE rangeTable (empno int, empname String, designation 
String,
+|  workgroupcategory int, workgroupcategoryname String, deptno 
int, deptname String,
+|  projectcode int, projectjoindate Timestamp, projectenddate 
Timestamp,attendance int,
+|  utilization int,salary int)
+| PARTITIONED BY (doj Timestamp)
+| STORED BY 'org.apache.carbondata.format'
+| TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+|  'RANGE_INFO'='01-01-2010, 01-01-2015')
+  """.stripMargin)
+sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE 
rangeTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+
+// EqualTo
+checkAnswer(sql("show partitions rangeTable"), Seq(Row("0", "", 
"default"), Row("1", "", "< 01-01-2010"), Row("2", "", "< 01-01-2015")))
--- End diff --

over 100 chars ?


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122715807
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * 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.testsuite.partition
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+
+class TestShowPartition  extends QueryTest with BeforeAndAfterAll {
--- End diff --

extra space between “TestShowPartition” and “extend”


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122676528
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * 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.testsuite.partition
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+
+class TestShowPartition  extends QueryTest with BeforeAndAfterAll {
+  override def beforeAll = {
+dropTable
+
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, 
"dd-MM-")
+
+  }
+
+  test("show partition table: hash table") {
+sql(
+  """
+| CREATE TABLE hashTable (empname String, designation String, doj 
Timestamp,
+|  workgroupcategory int, workgroupcategoryname String, deptno 
int, deptname String,
+|  projectcode int, projectjoindate Timestamp, projectenddate 
Timestamp,attendance int,
+|  utilization int,salary int)
+| PARTITIONED BY (empno int)
+| STORED BY 'org.apache.carbondata.format'
+| TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
+  """.stripMargin)
+sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE 
hashTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
--- End diff --

This line maybe over 100 chars.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122658883
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * 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.testsuite.partition
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+
+class TestShowPartition  extends QueryTest with BeforeAndAfterAll {
+  override def beforeAll = {
+dropTable
+
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, 
"dd-MM-")
+
+  }
+
+  test("show partition table: hash table") {
+sql(
+  """
+| CREATE TABLE hashTable (empname String, designation String, doj 
Timestamp,
+|  workgroupcategory int, workgroupcategoryname String, deptno 
int, deptname String,
+|  projectcode int, projectjoindate Timestamp, projectenddate 
Timestamp,attendance int,
+|  utilization int,salary int)
+| PARTITIONED BY (empno int)
+| STORED BY 'org.apache.carbondata.format'
+| TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
+  """.stripMargin)
+sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE 
hashTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+
+// EqualTo
+checkAnswer(sql("show partitions hashTable"), Seq(Row("HASH 
PARTITION", "", "3")))
+
+sql("drop table hashTable")
+  }
+
+  test("show partition table: range partition") {
+sql(
+  """
+| CREATE TABLE rangeTable (empno int, empname String, designation 
String,
+|  workgroupcategory int, workgroupcategoryname String, deptno 
int, deptname String,
+|  projectcode int, projectjoindate Timestamp, projectenddate 
Timestamp,attendance int,
+|  utilization int,salary int)
+| PARTITIONED BY (doj Timestamp)
+| STORED BY 'org.apache.carbondata.format'
+| TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+|  'RANGE_INFO'='01-01-2010, 01-01-2015')
+  """.stripMargin)
+sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE 
rangeTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+
+// EqualTo
+checkAnswer(sql("show partitions rangeTable"), Seq(Row("0", "", 
"default"), Row("1", "", "< 01-01-2010"), Row("2", "", "< 01-01-2015")))
+sql("drop table rangeTable")
+  }
+
+  test("show partition table: list partition") {
+sql(
+  """
+| CREATE TABLE listTable (empno int, empname String, designation 
String, doj Timestamp,
+|  workgroupcategoryname String, deptno int, deptname String,
+|  projectcode int, projectjoindate Timestamp, projectenddate 
Timestamp,attendance int,
+|  utilization int,salary int)
+| PARTITIONED BY (workgroupcategory int)
+| STORED BY 'org.apache.carbondata.format'
+| TBLPROPERTIES('PARTITION_TYPE'='LIST',
+|  'LIST_INFO'='0, 1, (2, 3)')
+  """.stripMargin)
+sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE 
listTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+
+// EqualTo
+checkAnswer(sql("show partitions listTable"), Seq(Row("0", "", "0"), 
Row("1", "", "1"), Row("2", "", "2, 3")))
+
+  sql("drop table listTable")
+  }
+  test("show partition table: not default db") {
+sql(s"CREATE DATABASE if not exists partitionDB")
+
+sql(
+  """
+| CREATE TABLE partitionDB.listTable (empno int, empname St

[GitHub] carbondata pull request #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122658340
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonShowPartitionInfo.scala
 ---
@@ -0,0 +1,111 @@
+/*
+ * 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.examples
+
+import java.io.File
+
+import org.apache.spark.sql.SparkSession
+
+object CarbonShowPartitionInfo {
+  def main(args: Array[String]) {
+
+CarbonShowPartitionInfo.extracted("t3", args)
+  }
+  def extracted(tableName: String, args: Array[String]): Unit = {
+val rootPath = new File(this.getClass.getResource("/").getPath
+  + "../../../..").getCanonicalPath
+val storeLocation = s"$rootPath/examples/spark2/target/store"
+val warehouse = s"$rootPath/examples/spark2/target/warehouse"
+val metastoredb = s"$rootPath/examples/spark2/target"
+val testData = 
s"$rootPath/examples/spark2/src/main/resources/bitmaptest2.csv"
+import org.apache.spark.sql.CarbonSession._
+val spark = SparkSession
+  .builder()
+  .master("local")
+  .appName("CarbonDataLoad")
+  .config("spark.sql.warehouse.dir", warehouse)
+  .getOrCreateCarbonSession(storeLocation, metastoredb)
+
+// range partition
+spark.sql("DROP TABLE IF EXISTS t1")
+// hash partition
+spark.sql("DROP TABLE IF EXISTS t3")
+// list partition
+spark.sql("DROP TABLE IF EXISTS t5")
+
+spark.sql("""
+| CREATE TABLE IF NOT EXISTS t1
+| (
+| vin String,
+| phonenumber Long,
+| country String,
+| area String
+| )
+| PARTITIONED BY (logdate Timestamp)
+| STORED BY 'carbondata'
+| TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+| 'RANGE_INFO'='20140101, 2015/01/01 ,2016-01-01')
+  """.stripMargin)
+
+spark.sql("""
+| CREATE TABLE IF NOT EXISTS t3
+| (
+| logdate Timestamp,
+| phonenumber Long,
+| country String,
+| area String
+| )
+| PARTITIONED BY (vin String)
+| STORED BY 'carbondata'
+| 
TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='5')
+""".stripMargin)
+
+spark.sql("""
+   | CREATE TABLE IF NOT EXISTS t5
+   | (
+   | vin String,
+   | logdate Timestamp,
+   | phonenumber Long,
+   | area String
+   |)
+   | PARTITIONED BY (country string)
+   | STORED BY 'carbondata'
+   | TBLPROPERTIES('PARTITION_TYPE'='LIST',
+   | 'LIST_INFO'='(China,United States),UK ,japan,(Canada,Russia), 
South Korea ')
+   """.stripMargin)
+
+spark.sparkContext.setLogLevel("WARN")
+spark.sql(s"""
+  SHOW PARTITIONS t1
+ """).show()
+spark.sql(s"""
+  SHOW PARTITIONS t3
+ """).show()
+spark.sql(s"""
+  SHOW PARTITIONS t5
+ """).show()
+
+// range partition
+spark.sql("DROP TABLE IF EXISTS t1")
+// hash partition
+spark.sql("DROP TABLE IF EXISTS t3")
+// list partition
+spark.sql("DROP TABLE IF EXISTS t5")
+
--- End diff --

Suggest close spark session as last.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122658297
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonShowPartitionInfo.scala
 ---
@@ -0,0 +1,111 @@
+/*
+ * 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.examples
+
+import java.io.File
+
+import org.apache.spark.sql.SparkSession
+
+object CarbonShowPartitionInfo {
+  def main(args: Array[String]) {
+
+CarbonShowPartitionInfo.extracted("t3", args)
+  }
+  def extracted(tableName: String, args: Array[String]): Unit = {
+val rootPath = new File(this.getClass.getResource("/").getPath
+  + "../../../..").getCanonicalPath
+val storeLocation = s"$rootPath/examples/spark2/target/store"
+val warehouse = s"$rootPath/examples/spark2/target/warehouse"
+val metastoredb = s"$rootPath/examples/spark2/target"
+val testData = 
s"$rootPath/examples/spark2/src/main/resources/bitmaptest2.csv"
+import org.apache.spark.sql.CarbonSession._
+val spark = SparkSession
+  .builder()
+  .master("local")
+  .appName("CarbonDataLoad")
+  .config("spark.sql.warehouse.dir", warehouse)
+  .getOrCreateCarbonSession(storeLocation, metastoredb)
+
+// range partition
+spark.sql("DROP TABLE IF EXISTS t1")
+// hash partition
+spark.sql("DROP TABLE IF EXISTS t3")
+// list partition
+spark.sql("DROP TABLE IF EXISTS t5")
+
+spark.sql("""
+| CREATE TABLE IF NOT EXISTS t1
+| (
+| vin String,
+| phonenumber Long,
+| country String,
+| area String
+| )
+| PARTITIONED BY (logdate Timestamp)
+| STORED BY 'carbondata'
+| TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+| 'RANGE_INFO'='20140101, 2015/01/01 ,2016-01-01')
+  """.stripMargin)
+
+spark.sql("""
+| CREATE TABLE IF NOT EXISTS t3
+| (
+| logdate Timestamp,
+| phonenumber Long,
+| country String,
+| area String
+| )
+| PARTITIONED BY (vin String)
+| STORED BY 'carbondata'
+| 
TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='5')
+""".stripMargin)
+
+spark.sql("""
+   | CREATE TABLE IF NOT EXISTS t5
+   | (
+   | vin String,
+   | logdate Timestamp,
+   | phonenumber Long,
+   | area String
+   |)
+   | PARTITIONED BY (country string)
+   | STORED BY 'carbondata'
+   | TBLPROPERTIES('PARTITION_TYPE'='LIST',
+   | 'LIST_INFO'='(China,United States),UK ,japan,(Canada,Russia), 
South Korea ')
+   """.stripMargin)
+
+spark.sparkContext.setLogLevel("WARN")
+spark.sql(s"""
+  SHOW PARTITIONS t1
+ """).show()
+spark.sql(s"""
--- End diff --

One line, same problem.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122658089
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonShowPartitionInfo.scala
 ---
@@ -0,0 +1,111 @@
+/*
+ * 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.examples
+
+import java.io.File
+
+import org.apache.spark.sql.SparkSession
+
+object CarbonShowPartitionInfo {
+  def main(args: Array[String]) {
+
+CarbonShowPartitionInfo.extracted("t3", args)
+  }
+  def extracted(tableName: String, args: Array[String]): Unit = {
+val rootPath = new File(this.getClass.getResource("/").getPath
+  + "../../../..").getCanonicalPath
+val storeLocation = s"$rootPath/examples/spark2/target/store"
+val warehouse = s"$rootPath/examples/spark2/target/warehouse"
+val metastoredb = s"$rootPath/examples/spark2/target"
+val testData = 
s"$rootPath/examples/spark2/src/main/resources/bitmaptest2.csv"
+import org.apache.spark.sql.CarbonSession._
+val spark = SparkSession
+  .builder()
+  .master("local")
+  .appName("CarbonDataLoad")
+  .config("spark.sql.warehouse.dir", warehouse)
+  .getOrCreateCarbonSession(storeLocation, metastoredb)
+
+// range partition
+spark.sql("DROP TABLE IF EXISTS t1")
+// hash partition
+spark.sql("DROP TABLE IF EXISTS t3")
+// list partition
+spark.sql("DROP TABLE IF EXISTS t5")
+
+spark.sql("""
+| CREATE TABLE IF NOT EXISTS t1
+| (
+| vin String,
+| phonenumber Long,
+| country String,
+| area String
+| )
+| PARTITIONED BY (logdate Timestamp)
+| STORED BY 'carbondata'
+| TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+| 'RANGE_INFO'='20140101, 2015/01/01 ,2016-01-01')
--- End diff --

Same as mentioned 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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122657979
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonShowPartitionInfo.scala
 ---
@@ -0,0 +1,111 @@
+/*
+ * 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.examples
+
+import java.io.File
+
+import org.apache.spark.sql.SparkSession
+
+object CarbonShowPartitionInfo {
--- End diff --

name definition, same problem as spark1.6.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122657713
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
 ---
@@ -107,7 +105,6 @@ object CarbonPartitionExample {
| vin String,
| logdate Timestamp,
| phonenumber Long,
-   | country String,
--- End diff --

already have PR-1049 to modify this class, you can help to review.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122656172
  
--- Diff: 
examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonShowPartitionInfo.scala
 ---
@@ -0,0 +1,114 @@
+/*
+ * 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.examples
+
+import scala.collection.mutable.LinkedHashMap
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.util.ExampleUtils
+
+object CarbonShowPartitionInfo {
+  def main(args: Array[String]) {
+
+CarbonShowPartitionInfo.extracted("t3", args)
+  }
+  def extracted(tableName: String, args: Array[String]): Unit = {
+val cc = ExampleUtils.createCarbonContext("CarbonShowPartitionInfo")
+val testData = ExampleUtils.currentPath + 
"/src/main/resources/data.csv"
+
+// range partition
+cc.sql("DROP TABLE IF EXISTS t1")
+
+cc.sql("""
+| CREATE TABLE IF NOT EXISTS t1
+| (
+| vin String,
+| phonenumber Int,
+| country String,
+| area String
+| )
+| PARTITIONED BY (logdate Timestamp)
+| STORED BY 'carbondata'
+| TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+| 'RANGE_INFO'='20140101, 2015/01/01 ,2016-01-01')
+  """.stripMargin)
+cc.sql(s"""
+  SHOW PARTITIONS t1
+ """).show()
+
+cc.sql("""
+| CREATE TABLE IF NOT EXISTS t3
+| (
+| logdate Timestamp,
+| phonenumber Int,
+| country String,
+| area String
+| )
+| PARTITIONED BY (vin String)
+| STORED BY 'carbondata'
+| 
TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='5')
+""".stripMargin)
+cc.sql(s"""
+  SHOW PARTITIONS t3
+ """).show()
+// list partition
+cc.sql("DROP TABLE IF EXISTS t5")
+
+cc.sql("""
+   | CREATE TABLE IF NOT EXISTS t5
+   | (
+   | vin String,
+   | logdate Timestamp,
+   | phonenumber Int,
+   | area String
+   |)
+   | PARTITIONED BY (country string)
+   | STORED BY 'carbondata'
+   | TBLPROPERTIES('PARTITION_TYPE'='LIST',
+   | 'LIST_INFO'='(China,United States),UK ,japan,(Canada,Russia), 
South Korea ')
+   """.stripMargin)
+cc.sql(s"""
+  SHOW PARTITIONS t5
+ """).show()
+
+cc.sql(s"DROP TABLE IF EXISTS partitionDB.$tableName")
+cc.sql(s"DROP DATABASE IF EXISTS partitionDB")
+cc.sql(s"CREATE DATABASE partitionDB")
+cc.sql(s"""
+| CREATE TABLE IF NOT EXISTS partitionDB.$tableName
+| (
+| logdate Timestamp,
+| phonenumber Int,
+| country String,
+| area String
+| )
+| PARTITIONED BY (vin String)
+| STORED BY 'carbondata'
+| 
TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='5')
+""".stripMargin)
+cc.sql(s"""
+  SHOW PARTITIONS partitionDB.$tableName
+ """).show()
+
+cc.sql(s"""
+  SHOW PARTITIONS $tableName
+ """).show()
+
--- End diff --

Suggest do some clear work, such as drop created tables and database.


---
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

[GitHub] carbondata pull request #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122652358
  
--- Diff: 
examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonShowPartitionInfo.scala
 ---
@@ -0,0 +1,114 @@
+/*
+ * 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.examples
+
+import scala.collection.mutable.LinkedHashMap
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.util.ExampleUtils
+
+object CarbonShowPartitionInfo {
+  def main(args: Array[String]) {
+
+CarbonShowPartitionInfo.extracted("t3", args)
+  }
+  def extracted(tableName: String, args: Array[String]): Unit = {
+val cc = ExampleUtils.createCarbonContext("CarbonShowPartitionInfo")
+val testData = ExampleUtils.currentPath + 
"/src/main/resources/data.csv"
+
+// range partition
+cc.sql("DROP TABLE IF EXISTS t1")
+
+cc.sql("""
+| CREATE TABLE IF NOT EXISTS t1
+| (
+| vin String,
+| phonenumber Int,
+| country String,
+| area String
+| )
+| PARTITIONED BY (logdate Timestamp)
+| STORED BY 'carbondata'
+| TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+| 'RANGE_INFO'='20140101, 2015/01/01 ,2016-01-01')
+  """.stripMargin)
+cc.sql(s"""
+  SHOW PARTITIONS t1
+ """).show()
--- End diff --

Suggest command “SHOW PARTITIONS t1” can be on one 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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122650894
  
--- Diff: 
examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonShowPartitionInfo.scala
 ---
@@ -0,0 +1,114 @@
+/*
+ * 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.examples
+
+import scala.collection.mutable.LinkedHashMap
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.util.ExampleUtils
+
+object CarbonShowPartitionInfo {
+  def main(args: Array[String]) {
+
--- End diff --

delete extra 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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122650800
  
--- Diff: 
examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonShowPartitionInfo.scala
 ---
@@ -0,0 +1,114 @@
+/*
+ * 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.examples
+
+import scala.collection.mutable.LinkedHashMap
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.util.ExampleUtils
+
+object CarbonShowPartitionInfo {
+  def main(args: Array[String]) {
+
+CarbonShowPartitionInfo.extracted("t3", args)
+  }
+  def extracted(tableName: String, args: Array[String]): Unit = {
+val cc = ExampleUtils.createCarbonContext("CarbonShowPartitionInfo")
+val testData = ExampleUtils.currentPath + 
"/src/main/resources/data.csv"
+
+// range partition
+cc.sql("DROP TABLE IF EXISTS t1")
+
+cc.sql("""
+| CREATE TABLE IF NOT EXISTS t1
+| (
+| vin String,
+| phonenumber Int,
+| country String,
+| area String
+| )
--- End diff --

I think key words in sql command and data type should be upper case. 
for example, "String" -> "STRING"
anyway, it's not a big problem.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122650261
  
--- Diff: 
examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonShowPartitionInfo.scala
 ---
@@ -0,0 +1,114 @@
+/*
+ * 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.examples
+
+import scala.collection.mutable.LinkedHashMap
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.util.ExampleUtils
+
+object CarbonShowPartitionInfo {
+  def main(args: Array[String]) {
+
+CarbonShowPartitionInfo.extracted("t3", args)
+  }
+  def extracted(tableName: String, args: Array[String]): Unit = {
+val cc = ExampleUtils.createCarbonContext("CarbonShowPartitionInfo")
+val testData = ExampleUtils.currentPath + 
"/src/main/resources/data.csv"
+
+// range partition
+cc.sql("DROP TABLE IF EXISTS t1")
+
+cc.sql("""
+| CREATE TABLE IF NOT EXISTS t1
+| (
+| vin String,
+| phonenumber Int,
+| country String,
+| area String
+| )
+| PARTITIONED BY (logdate Timestamp)
+| STORED BY 'carbondata'
+| TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+| 'RANGE_INFO'='20140101, 2015/01/01 ,2016-01-01')
--- End diff --

We will add partition check mechanism later, So change 'RANGE_INFO'  = 
'2014/01/01,2015/01/01,2016/01/01', otherwise it will fail after check 
mechanism merged.
Besides, it will be better for specify the timestamp format suitable for 
'2014/0101' before running the sql command.


---
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 #1042: [CARBONDATA-1181] Show partitions

2017-06-19 Thread chenerlu
Github user chenerlu commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1042#discussion_r122648924
  
--- Diff: 
examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonShowPartitionInfo.scala
 ---
@@ -0,0 +1,114 @@
+/*
+ * 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.examples
+
+import scala.collection.mutable.LinkedHashMap
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.util.ExampleUtils
+
+object CarbonShowPartitionInfo {
--- End diff --

Suggest the name to ShowPartitionInfoExample, because we may unify the 
management of Examples.
This is just my idea, we can easily know this is a point to point test case 
if its name end with Example.


---
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.
---