You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by akashrn5 <gi...@git.apache.org> on 2018/07/05 06:00:34 UTC

[GitHub] carbondata pull request #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local d...

GitHub user akashrn5 opened a pull request:

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

    [CARBONDATA-2585][CARBONDATA-2586]Fix local dictionary support for preagg and set localdict info in column schema

    **Changes**
    This PR fixes local dictionary support for preaggregate and set the column dict info of each column in column schema read and write for backward compatibility.
    
    Existing test cases of local dictionary will take care of testing this

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

    $ git pull https://github.com/akashrn5/incubator-carbondata local_preag_query

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

    https://github.com/apache/carbondata/pull/2451.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 #2451
    
----
commit 78fc7e1acec23e4fd1c9f6cb5ad57e4ea87d3df5
Author: akashrn5 <ak...@...>
Date:   2018-07-05T05:53:40Z

    fix local dictionary for preagg and set localdict info in column schema

----


---

[GitHub] carbondata pull request #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local d...

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

    https://github.com/apache/carbondata/pull/2451#discussion_r200277263
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala ---
    @@ -116,14 +116,33 @@ case class PreAggregateTableHelper(
             parentTable.getTableInfo.getFactTable.getTableProperties.asScala
               .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD,
                 CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD_DEFAULT))
    -    tableProperties
    -      .put(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE,
    -        parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    -          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE, ""))
    -    tableProperties
    -      .put(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE,
    -        parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    -          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE, ""))
    +    val parentdict_include = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    --- End diff --
    
    done


---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata pull request #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local d...

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

    https://github.com/apache/carbondata/pull/2451#discussion_r200277149
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java ---
    @@ -521,6 +521,7 @@ public void write(DataOutput out) throws IOException {
           }
         }
         out.writeBoolean(isDimensionColumn);
    +    out.writeBoolean(isLocalDictColumn);
    --- End diff --
    
    i have added the field at last while writing, so it should not create any problem


---

[GitHub] carbondata pull request #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local d...

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

    https://github.com/apache/carbondata/pull/2451#discussion_r200894716
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala ---
    @@ -136,14 +136,33 @@ case class PreAggregateTableHelper(
             parentTable.getTableInfo.getFactTable.getTableProperties.asScala
               .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD,
                 CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD_DEFAULT))
    -    tableProperties
    -      .put(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE,
    -        parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    -          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE, ""))
    -    tableProperties
    -      .put(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE,
    -        parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    -          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE, ""))
    +    val parentDictInclude = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    +      .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE, "").split(",")
    +
    +    val parentDictExclude = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    +      .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE, "").split(",")
    +
    +    var newDictInclude = Seq[String]()
    +    var newDictExclude = Seq[String]()
    +    parentDictInclude.foreach(parentcol =>
    +      fields.filter(col => fieldRelationMap(col).aggregateFunction.isEmpty &&
    +                           parentcol.equals(fieldRelationMap(col).
    +                             columnTableRelationList.get.head.parentColumnName))
    +        .map(cols => newDictInclude :+= cols.column))
    +    parentDictExclude.foreach(parentcol =>
    +      fields.filter(col => fieldRelationMap(col).aggregateFunction.isEmpty &&
    +                           parentcol.equals(fieldRelationMap(col).
    +                             columnTableRelationList.get.head.parentColumnName))
    +        .map(cols => newDictExclude :+= cols.column))
    --- End diff --
    
    change .filter(..).map(..) to .collect


---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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


---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata pull request #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local d...

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

    https://github.com/apache/carbondata/pull/2451#discussion_r200894679
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala ---
    @@ -136,14 +136,33 @@ case class PreAggregateTableHelper(
             parentTable.getTableInfo.getFactTable.getTableProperties.asScala
               .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD,
                 CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD_DEFAULT))
    -    tableProperties
    -      .put(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE,
    -        parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    -          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE, ""))
    -    tableProperties
    -      .put(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE,
    -        parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    -          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE, ""))
    +    val parentDictInclude = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    +      .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE, "").split(",")
    +
    +    val parentDictExclude = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    +      .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE, "").split(",")
    +
    +    var newDictInclude = Seq[String]()
    +    var newDictExclude = Seq[String]()
    +    parentDictInclude.foreach(parentcol =>
    +      fields.filter(col => fieldRelationMap(col).aggregateFunction.isEmpty &&
    --- End diff --
    
    change .filter(..).map(..) to .collect


---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata pull request #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local d...

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

    https://github.com/apache/carbondata/pull/2451#discussion_r200255313
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala ---
    @@ -116,14 +116,33 @@ case class PreAggregateTableHelper(
             parentTable.getTableInfo.getFactTable.getTableProperties.asScala
               .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD,
                 CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD_DEFAULT))
    -    tableProperties
    -      .put(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE,
    -        parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    -          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE, ""))
    -    tableProperties
    -      .put(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE,
    -        parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    -          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE, ""))
    +    val parentdict_include = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    +      .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE, "").split(",")
    +
    +    val parentdict_exclude = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    +      .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE, "").split(",")
    +
    +    var newdict_include = Seq[String]()
    +    var newdict_exclude = Seq[String]()
    +    parentdict_include.foreach(parentcol =>
    --- End diff --
    
    just useļ¼š
    val newDictInclude = XXXX.filter().map(_.column)


---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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


---

[GitHub] carbondata pull request #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local d...

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

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


---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---

[GitHub] carbondata pull request #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local d...

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

    https://github.com/apache/carbondata/pull/2451#discussion_r200250993
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala ---
    @@ -116,14 +116,33 @@ case class PreAggregateTableHelper(
             parentTable.getTableInfo.getFactTable.getTableProperties.asScala
               .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD,
                 CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD_DEFAULT))
    -    tableProperties
    -      .put(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE,
    -        parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    -          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE, ""))
    -    tableProperties
    -      .put(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE,
    -        parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    -          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE, ""))
    +    val parentdict_include = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
    --- End diff --
    
    need to optimize the variable's name
    
    same problem in belowing lines


---

[GitHub] carbondata pull request #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local d...

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

    https://github.com/apache/carbondata/pull/2451#discussion_r200247028
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java ---
    @@ -521,6 +521,7 @@ public void write(DataOutput out) throws IOException {
           }
         }
         out.writeBoolean(isDimensionColumn);
    +    out.writeBoolean(isLocalDictColumn);
    --- End diff --
    
    Have you tested the upgrade scenario since you add the new member in the middle during serialization/deserialization?


---

[GitHub] carbondata issue #2451: [CARBONDATA-2585][CARBONDATA-2586]Fix local dictiona...

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

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



---