You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by ravipesala <gi...@git.apache.org> on 2017/11/12 17:36:47 UTC

[GitHub] carbondata pull request #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support Da...

GitHub user ravipesala opened a pull request:

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

    [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap show

    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
      Added support for show datamap on table.
      ```
       SHOW DATAMAP ON TABLE test
      ```
      The above command shows all datamaps on the table name test.
     Output of it shows the following attributes
      ```
      +-----------+---------+----------------+
    |DataMapName|ClassName|Associated Table|
    +-----------+---------+----------------+
      ```
    
    This PR is dependent on the PRs https://github.com/apache/carbondata/pull/1481  and https://github.com/apache/carbondata/pull/1489 . First those PR need to be merged
     - [X] Any interfaces changed?
       NO
     - [X] Any backward compatibility impacted?
       NO
     - [X] Document update required?
       Yes, sql command needs to be added to document
     - [X] Testing done
           Tests are added        
     - [X] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    


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

    $ git pull https://github.com/ravipesala/incubator-carbondata datamap-show

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

    https://github.com/apache/carbondata/pull/1490.patch

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

    This closes #1490
    
----
commit cac54d9a1578796cf1d90abacab069a0413a49a0
Author: ravipesala <ra...@gmail.com>
Date:   2017-11-12T11:38:09Z

    Added create datamap parser and saved to schema file

commit aebcdc01ce66ed79bac17db61fd62a325622e9c4
Author: ravipesala <ra...@gmail.com>
Date:   2017-11-12T06:57:47Z

    Added parser for drop datamap and handled events

commit dd1ee26742f8c8f8731260ac623173c5309cb153
Author: ravipesala <ra...@gmail.com>
Date:   2017-11-12T17:26:17Z

    Added parser for Show Datamap command

----


---

[GitHub] carbondata pull request #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support Da...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1490#discussion_r150679772
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala ---
    @@ -81,8 +82,25 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
         assert(dataMapSchemaList.size() == 3)
       }
     
    +  test("test show datamap without preaggregate") {
    +    sql("drop table if exists datamapshowtest")
    +    sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
    +    sql("create datamap datamap1 on table datamapshowtest using 'new.class' dmproperties('key'='value')")
    +    sql("create datamap datamap2 on table datamapshowtest using 'new.class' dmproperties('key'='value')")
    +    checkExistence(sql("show datamap on table datamapshowtest"), true, "datamap1", "datamap2", "-NA-", "new.class")
    --- End diff --
    
    No, it is NA only. The associated table available only for preaggregate datamaps


---

[GitHub] carbondata pull request #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support Da...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1490#discussion_r150808687
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDataMapShowCommand.scala ---
    @@ -0,0 +1,67 @@
    +/*
    + * 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.datamap
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
    +import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, RunnableCommand}
    +import org.apache.spark.sql.hive.CarbonRelation
    +import org.apache.spark.sql.types.StringType
    +
    +/**
    + * Show the datamaps on the table
    + * @param databaseNameOp
    + * @param tableName
    + */
    +case class CarbonDataMapShowCommand(
    +    databaseNameOp: Option[String],
    +    tableName: String)
    +  extends RunnableCommand with DataProcessCommand {
    +
    +  override def output: Seq[Attribute] = {
    +    Seq(AttributeReference("DataMapName", StringType, nullable = false)(),
    +      AttributeReference("ClassName", StringType, nullable = false)(),
    +      AttributeReference("Associated Table", StringType, nullable = false)())
    +  }
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    processData(sparkSession)
    +  }
    +
    +  override def processData(sparkSession: SparkSession): Seq[Row] = {
    +    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
    +    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
    +      lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
    +      tableMeta.carbonTable
    +    val schemaList = carbonTable.getTableInfo.getDataMapSchemaList
    +    if (schemaList != null && schemaList.size() > 0) {
    +      schemaList.asScala.map { s =>
    +        var table = "-NA-"
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1709/



---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1651/



---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1653/



---

[GitHub] carbondata pull request #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support Da...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1490#discussion_r150812888
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala ---
    @@ -81,8 +82,25 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
         assert(dataMapSchemaList.size() == 3)
       }
     
    +  test("test show datamap without preaggregate") {
    +    sql("drop table if exists datamapshowtest")
    +    sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
    +    sql("create datamap datamap1 on table datamapshowtest using 'new.class' dmproperties('key'='value')")
    +    sql("create datamap datamap2 on table datamapshowtest using 'new.class' dmproperties('key'='value')")
    +    checkExistence(sql("show datamap on table datamapshowtest"), true, "datamap1", "datamap2", "-NA-", "new.class")
    +  }
    +
    +  test("test show datamap with preaggregate") {
    +    sql("drop table if exists datamapshowtest")
    +    sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
    +    sql("create datamap datamap1 on table datamapshowtest using 'preaggregate' as select count(a) from datamapshowtest")
    +    sql("create datamap datamap2 on table datamapshowtest using 'new.class' dmproperties('key'='value')")
    +    checkExistence(sql("show datamap on table datamapshowtest"), true, "datamap1", "datamap2", "-NA-", "new.class", "default.datamap1")
    +  }
    +
    --- End diff --
    
    added


---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    retest this please


---

[GitHub] carbondata pull request #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support Da...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

[GitHub] carbondata pull request #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support Da...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1490#discussion_r150506530
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDataMapShowCommand.scala ---
    @@ -0,0 +1,67 @@
    +/*
    + * 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.datamap
    +
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
    +import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, RunnableCommand}
    +import org.apache.spark.sql.hive.CarbonRelation
    +import org.apache.spark.sql.types.StringType
    +
    +/**
    + * Show the datamaps on the table
    + * @param databaseNameOp
    + * @param tableName
    + */
    +case class CarbonDataMapShowCommand(
    +    databaseNameOp: Option[String],
    +    tableName: String)
    +  extends RunnableCommand with DataProcessCommand {
    +
    +  override def output: Seq[Attribute] = {
    +    Seq(AttributeReference("DataMapName", StringType, nullable = false)(),
    +      AttributeReference("ClassName", StringType, nullable = false)(),
    +      AttributeReference("Associated Table", StringType, nullable = false)())
    +  }
    +
    +  override def run(sparkSession: SparkSession): Seq[Row] = {
    +    processData(sparkSession)
    +  }
    +
    +  override def processData(sparkSession: SparkSession): Seq[Row] = {
    +    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
    +    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
    +      lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
    +      tableMeta.carbonTable
    +    val schemaList = carbonTable.getTableInfo.getDataMapSchemaList
    +    if (schemaList != null && schemaList.size() > 0) {
    +      schemaList.asScala.map { s =>
    +        var table = "-NA-"
    --- End diff --
    
    I think `(NA)` is better


---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/1712/



---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1033/



---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    LGTM


---

[GitHub] carbondata pull request #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support Da...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1490#discussion_r150507602
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala ---
    @@ -81,8 +82,25 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
         assert(dataMapSchemaList.size() == 3)
       }
     
    +  test("test show datamap without preaggregate") {
    +    sql("drop table if exists datamapshowtest")
    +    sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
    +    sql("create datamap datamap1 on table datamapshowtest using 'new.class' dmproperties('key'='value')")
    +    sql("create datamap datamap2 on table datamapshowtest using 'new.class' dmproperties('key'='value')")
    +    checkExistence(sql("show datamap on table datamapshowtest"), true, "datamap1", "datamap2", "-NA-", "new.class")
    +  }
    +
    +  test("test show datamap with preaggregate") {
    +    sql("drop table if exists datamapshowtest")
    +    sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
    +    sql("create datamap datamap1 on table datamapshowtest using 'preaggregate' as select count(a) from datamapshowtest")
    +    sql("create datamap datamap2 on table datamapshowtest using 'new.class' dmproperties('key'='value')")
    +    checkExistence(sql("show datamap on table datamapshowtest"), true, "datamap1", "datamap2", "-NA-", "new.class", "default.datamap1")
    +  }
    +
    --- End diff --
    
    please add a testcase to show datamap after dropping the datamap


---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1120/



---

[GitHub] carbondata pull request #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support Da...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1490#discussion_r150809947
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala ---
    @@ -142,6 +142,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
             CarbonDropDataMapCommand(dmname, ifexists.isDefined, dbName, tableName)
         }
     
    +  protected lazy val showDataMap: Parser[LogicalPlan] =
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    retest this please


---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1035/



---

[GitHub] carbondata pull request #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support Da...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1490#discussion_r150507280
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala ---
    @@ -81,8 +82,25 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
         assert(dataMapSchemaList.size() == 3)
       }
     
    +  test("test show datamap without preaggregate") {
    +    sql("drop table if exists datamapshowtest")
    +    sql("create table datamapshowtest (a string, b string, c string) stored by 'carbondata'")
    +    sql("create datamap datamap1 on table datamapshowtest using 'new.class' dmproperties('key'='value')")
    +    sql("create datamap datamap2 on table datamapshowtest using 'new.class' dmproperties('key'='value')")
    +    checkExistence(sql("show datamap on table datamapshowtest"), true, "datamap1", "datamap2", "-NA-", "new.class")
    --- End diff --
    
    In output string, it should show `datamapshowtest` instead of `-NA-` for the table name, right?


---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1110/



---

[GitHub] carbondata issue #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support DataMap s...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/1490
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/1113/



---

[GitHub] carbondata pull request #1490: [CARBONDATA-1579][PREAGG][DATAMAP] Support Da...

Posted by jackylk <gi...@git.apache.org>.
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1490#discussion_r150506799
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala ---
    @@ -142,6 +142,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
             CarbonDropDataMapCommand(dmname, ifexists.isDefined, dbName, tableName)
         }
     
    +  protected lazy val showDataMap: Parser[LogicalPlan] =
    --- End diff --
    
    please add syntax description


---