You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by ndwangsen <gi...@git.apache.org> on 2018/08/10 01:42:29 UTC

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

GitHub user ndwangsen opened a pull request:

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

    [CARBONDATA-2835] [MVDataMap] Block MV datamap on streaming table

    This PR  block creating MV datamap on streaming table and also block setting streaming property for table which has MV datamap.
    
    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
     NA
     - [ ] Any backward compatibility impacted?
     NA
     - [ ] Document update required?
    NA
     - [ ] Testing done
            Please provide details on 
            - Whether new unit test cases have been added or why no new tests are required?
            - How it is tested? Please attach test report.
            - Is it a performance related change? Please attach the performance test report.
            - Any additional information to help reviewers in testing this change.
          Add test case and test pass  
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    NA
    


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

    $ git pull https://github.com/ndwangsen/incubator-carbondata block_stream_mv

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

    https://github.com/apache/carbondata/pull/2627.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 #2627
    
----
commit e7494f6390226e475a0ab9d6d894eafe2c45bed9
Author: ndwangsen <lu...@...>
Date:   2018-08-10T01:32:59Z

    [CARBONDATA-2835] Block MV datamap on streaming table

----


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

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


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

    https://github.com/apache/carbondata/pull/2627
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6597/



---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210202221
  
    --- Diff: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala ---
    @@ -80,6 +81,16 @@ object MVHelper {
         dmProperties.foreach(t => tableProperties.put(t._1, t._2))
     
         val selectTables = getTables(logicalPlan)
    +    selectTables.map { selectTable =>
    +      val mainCarbonTable = CarbonEnv.getCarbonTableOption(selectTable.identifier.database,
    +        selectTable.identifier.table)(sparkSession)
    +
    +      if (!mainCarbonTable.isEmpty && mainCarbonTable.get.isStreamingSink ) {
    +        throw new MalformedCarbonCommandException(s"Streaming table does not support creating " +
    --- End diff --
    
    weird indentation here.
    You can start the statement in a new line to make the message in a complete line for better reading.


---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210239381
  
    --- Diff: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala ---
    @@ -80,6 +81,16 @@ object MVHelper {
         dmProperties.foreach(t => tableProperties.put(t._1, t._2))
     
         val selectTables = getTables(logicalPlan)
    +    selectTables.map { selectTable =>
    +      val mainCarbonTable = CarbonEnv.getCarbonTableOption(selectTable.identifier.database,
    +        selectTable.identifier.table)(sparkSession)
    +
    +      if (!mainCarbonTable.isEmpty && mainCarbonTable.get.isStreamingSink ) {
    +        throw new MalformedCarbonCommandException(s"Streaming table does not support creating " +
    --- End diff --
    
    ok,i modify it


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210202460
  
    --- Diff: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala ---
    @@ -80,6 +81,16 @@ object MVHelper {
         dmProperties.foreach(t => tableProperties.put(t._1, t._2))
     
         val selectTables = getTables(logicalPlan)
    +    selectTables.map { selectTable =>
    --- End diff --
    
    since it will take any effect, you can use `foreach` instead of `map`


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

    https://github.com/apache/carbondata/pull/2627
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6598/



---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

    https://github.com/apache/carbondata/pull/2627
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6642/



---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

    https://github.com/apache/carbondata/pull/2627
  
    retest sdv please


---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210782975
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala ---
    @@ -237,6 +237,21 @@ object CarbonEnv {
         getCarbonTable(tableIdentifier.database, tableIdentifier.table)(sparkSession)
       }
     
    +  /**
    +   * This method returns corresponding CarbonTable, it will return None if it's not a CarbonTable
    +   */
    +  def getCarbonTableOption(
    --- End diff --
    
    There is already a `getCarbonTable` func defined in `CarbonEnv`, is this needed??


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210241324
  
    --- Diff: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala ---
    @@ -80,6 +81,16 @@ object MVHelper {
         dmProperties.foreach(t => tableProperties.put(t._1, t._2))
     
         val selectTables = getTables(logicalPlan)
    +    selectTables.map { selectTable =>
    +      val mainCarbonTable = CarbonEnv.getCarbonTableOption(selectTable.identifier.database,
    +        selectTable.identifier.table)(sparkSession)
    +
    +      if (!mainCarbonTable.isEmpty && mainCarbonTable.get.isStreamingSink ) {
    +        throw new MalformedCarbonCommandException(s"Streaming table does not support creating " +
    +                                                    s"MV datamap")
    +      }
    +      selectTable
    --- End diff --
    
    ok,I remove it


---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210204028
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala ---
    @@ -237,6 +237,21 @@ object CarbonEnv {
         getCarbonTable(tableIdentifier.database, tableIdentifier.table)(sparkSession)
       }
     
    +  /**
    +   * whether the classification identifier is a carbon table
    --- End diff --
    
    The current expression is improper, it seems that this method returns a boolean which actually is not.
    So it can be optimized to
    ```
    This method returns corresponding CarbonTable, it will return None if it's not a CarbonTable
    ```


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210241115
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala ---
    @@ -73,13 +73,8 @@ case class CarbonCreateDataMapCommand(
           }
         }
     
    -    if (mainTable != null &&
    -        mainTable.isStreamingSink &&
    -        !(dmProviderName.equalsIgnoreCase(DataMapClassProvider.PREAGGREGATE.toString)
    -          || dmProviderName.equalsIgnoreCase(DataMapClassProvider.TIMESERIES.toString))) {
    -      throw new MalformedCarbonCommandException(s"Streaming table does not support creating " +
    -                                                s"$dmProviderName datamap")
    -    }
    +    // delete this code because streaming table only does not support creating MV datamap,
    --- End diff --
    
    Delete the comments here


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210202351
  
    --- Diff: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala ---
    @@ -80,6 +81,16 @@ object MVHelper {
         dmProperties.foreach(t => tableProperties.put(t._1, t._2))
     
         val selectTables = getTables(logicalPlan)
    +    selectTables.map { selectTable =>
    +      val mainCarbonTable = CarbonEnv.getCarbonTableOption(selectTable.identifier.database,
    +        selectTable.identifier.table)(sparkSession)
    +
    +      if (!mainCarbonTable.isEmpty && mainCarbonTable.get.isStreamingSink ) {
    +        throw new MalformedCarbonCommandException(s"Streaming table does not support creating " +
    +                                                    s"MV datamap")
    +      }
    +      selectTable
    --- End diff --
    
    This is useless and can be removed


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

    https://github.com/apache/carbondata/pull/2627
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6631/



---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210787370
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala ---
    @@ -237,6 +237,21 @@ object CarbonEnv {
         getCarbonTable(tableIdentifier.database, tableIdentifier.table)(sparkSession)
       }
     
    +  /**
    +   * This method returns corresponding CarbonTable, it will return None if it's not a CarbonTable
    +   */
    +  def getCarbonTableOption(
    --- End diff --
    
    the getCarbonTable will throw a exception when get a non-carbon table


---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210203693
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala ---
    @@ -277,6 +278,10 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
                   throw new MalformedCarbonCommandException(
                     "Streaming property value is incorrect")
                 }
    +            if (CarbonTable.hasMVDataMap(carbonTable)) {
    +              throw new MalformedCarbonCommandException(
    +                "The table which has MV datamap, does not support set streaming property")
    --- End diff --
    
    Please optimize the error message to
    ```
    Cannot set streaming property on table which has MV datamap
    ```
    OR
    just remove the comma in your original error message.


---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210203145
  
    --- Diff: datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/MVCreateTestCase.scala ---
    @@ -917,6 +917,104 @@ class MVCreateTestCase extends QueryTest with BeforeAndAfterAll {
         sql("drop table if exists mvtable2")
       }
     
    +  test("test create datamap with streaming table")  {
    +    sql("drop datamap if exists dm_stream_test1")
    +    sql("drop datamap if exists dm_stream_bloom")
    +    sql("drop datamap if exists dm_stream_PreAggMax")
    +    sql("drop table if exists fact_streaming_table1")
    +    sql(
    +      """
    +        | CREATE TABLE fact_streaming_table1 (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)
    +        | STORED BY 'org.apache.carbondata.format'
    +        | tblproperties('streaming'='true')
    +      """.stripMargin)
    +    sql(
    +      s"""
    +         | CREATE DATAMAP dm_stream_bloom ON TABLE fact_streaming_table1
    +         | USING 'bloomfilter'
    +         | DMProperties('INDEX_COLUMNS'='empname,deptname', 'BLOOM_SIZE'='640000')
    +      """.stripMargin)
    +
    +    sql("create datamap dm_stream_PreAggMax on table fact_streaming_table1 using 'preaggregate' " +
    +        "as select empname,max(salary) as max from fact_streaming_table1 group by empname")
    +    
    +    val exception_tb_mv: Exception = intercept[Exception] {
    +      sql("create datamap dm_stream_test1 using 'mv' as select empname, sum(utilization) from " +
    +          "fact_streaming_table1 group by empname")
    +    }
    +    assert(exception_tb_mv.getMessage
    +      .contains("Streaming table does not support creating MV datamap"))
    +  }
    +
    +  test("test create datamap with streaming table join carbon table ")  {
    +    sql("drop datamap if exists dm_stream_test2")
    +    sql("drop table if exists fact_streaming_table2")
    +    sql(
    +      """
    +        | CREATE TABLE fact_streaming_table2 (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)
    +        | STORED BY 'org.apache.carbondata.format'
    +        | tblproperties('streaming'='true')
    +      """.stripMargin)
    +    
    +    val exception_tb_mv2: Exception = intercept[Exception] {
    +      sql("create datamap dm_stream_test2 using 'mv' as select t1.empname as c1, t2.designation, " +
    +          "t2.empname as c2 from fact_table1 t1 inner join fact_streaming_table2 t2  " +
    +          "on (t1.empname = t2.empname)")
    +    }
    +    assert(exception_tb_mv2.getMessage
    +      .contains("Streaming table does not support creating MV datamap"))
    +  }
    +
    +  test("test create datamap with streaming table join not carbon table ")  {
    --- End diff --
    
    I  think you can meld this test case to the last one.
    For example you can create a MV whose query statement contains CarbonTable JOIN Non-CarbonTable and StreamingTable


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210239499
  
    --- Diff: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVHelper.scala ---
    @@ -80,6 +81,16 @@ object MVHelper {
         dmProperties.foreach(t => tableProperties.put(t._1, t._2))
     
         val selectTables = getTables(logicalPlan)
    +    selectTables.map { selectTable =>
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

    https://github.com/apache/carbondata/pull/2627
  
    retest sdv please


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

    https://github.com/apache/carbondata/pull/2627
  
    retest sdv please


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

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



---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

    https://github.com/apache/carbondata/pull/2627
  
    Build Success with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/70/



---

[GitHub] carbondata pull request #2627: [CARBONDATA-2835] [MVDataMap] Block MV datama...

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

    https://github.com/apache/carbondata/pull/2627#discussion_r210206238
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala ---
    @@ -73,13 +73,8 @@ case class CarbonCreateDataMapCommand(
           }
         }
     
    -    if (mainTable != null &&
    -        mainTable.isStreamingSink &&
    -        !(dmProviderName.equalsIgnoreCase(DataMapClassProvider.PREAGGREGATE.toString)
    -          || dmProviderName.equalsIgnoreCase(DataMapClassProvider.TIMESERIES.toString))) {
    -      throw new MalformedCarbonCommandException(s"Streaming table does not support creating " +
    -                                                s"$dmProviderName datamap")
    -    }
    +    // delete this code because streaming table only does not support creating MV datamap,
    --- End diff --
    
    These two statement can be optimized to
    ```
    // Carbondata support index/preagg datamap on streaming table and does not support MV on streaming table.
    // It will be blocked when we parse the original table of the query statement of MV, refer to CARBONDATA-2835
    ```


---

[GitHub] carbondata issue #2627: [CARBONDATA-2835] [MVDataMap] Block MV datamap on st...

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

    https://github.com/apache/carbondata/pull/2627
  
    retest sdv please


---