You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by ashokblend <gi...@git.apache.org> on 2016/12/03 12:23:44 UTC

[GitHub] incubator-carbondata pull request #388: [WIP]bigdecimal compression.

GitHub user ashokblend opened a pull request:

    https://github.com/apache/incubator-carbondata/pull/388

    [WIP]bigdecimal compression.

    This pr will do compression for bigdecimal data type as carbon does for double datatype. It is in progress.

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

    $ git pull https://github.com/ashokblend/incubator-carbondata bigdecimal_compression

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

    https://github.com/apache/incubator-carbondata/pull/388.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 #388
    
----
commit 0515b412d437611602885ffe322d2751d0494d59
Author: ashok.blend <as...@gmail.com>
Date:   2016-12-03T21:05:38Z

    bigdecimal compression.

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #388: [CARBONDATA-431]bigdecimal compressi...

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

    https://github.com/apache/incubator-carbondata/pull/388#discussion_r92089924
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java ---
    @@ -196,6 +198,31 @@ private static CompressionFinder getDoubleCompressorFinder(Object maxValue, Obje
         }
       }
     
    +  private static CompressionFinder getBigDecimalCompressorFinder(
    +      Object maxValue, Object minValue, int decimal, byte dataTypeSelected) {
    --- End diff --
    
    change decimal to mantissa, keep same with others


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #388: [CARBONDATA-431]bigdecimal compressi...

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

    https://github.com/apache/incubator-carbondata/pull/388#discussion_r92078130
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/compression/BigDecimalCompressor.java ---
    @@ -0,0 +1,77 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.carbondata.core.compression;
    +
    +import org.apache.carbondata.core.datastorage.store.dataholder.CarbonWriteDataHolder;
    +import org.apache.carbondata.core.util.BigDecimalCompressionFinder;
    +import org.apache.carbondata.core.util.CompressionFinder;
    +import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
    +
    +/**
    + * Bigdecimal data type compressor
    + *
    + */
    +public class BigDecimalCompressor extends BigIntCompressor {
    +
    +  private boolean readLeft = true;
    +
    +  @Override
    +  public Object getCompressedValues(CompressionFinder compressionFinder,
    +      CarbonWriteDataHolder dataHolder, Object maxValue, int decimal) {
    +    BigDecimalCompressionFinder bigdCompressionFinder =
    +        (BigDecimalCompressionFinder) compressionFinder;
    +    Long[] maxValues = (Long[]) maxValue;
    +    Object leftCompressedValue = getCompressedValues(
    +        bigdCompressionFinder.getLeftCompType(), dataHolder,
    +        bigdCompressionFinder.getLeftChangedDataType(), maxValues[0], 0);
    +    readLeft = false;
    +    Object rightCompressedValue = getCompressedValues(
    +        bigdCompressionFinder.getRightCompType(), dataHolder,
    +        bigdCompressionFinder.getRightChangedDataType(), maxValues[1], 0);
    +    return new Object[] { leftCompressedValue, rightCompressedValue };
    +
    --- End diff --
    
    remove empty 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #388: [CARBONDATA-431]bigdecimal compressi...

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

    https://github.com/apache/incubator-carbondata/pull/388


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #388: [CARBONDATA-431]bigdecimal compressi...

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

    https://github.com/apache/incubator-carbondata/pull/388#discussion_r92079674
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/compression/BigIntCompressor.java ---
    @@ -44,6 +44,11 @@ protected Object compressMaxMin(DataType changedDataType, CarbonWriteDataHolder
           Object max) {
         long maxValue = (long) max;
         long[] value = dataHolder.getWritableLongValues();
    +    return compressMaxMin(changedDataType, maxValue, value);
    +  }
    +
    +  protected Object compressMaxMin(DataType changedDataType, long maxValue,
    --- End diff --
    
    please add description to these compress functions, make it easier to understand by reader


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #388: [CARBONDATA-431]bigdecimal compression.

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

    https://github.com/apache/incubator-carbondata/pull/388
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/134/



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #388: [CARBONDATA-431][WIP]bigdecimal compression...

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

    https://github.com/apache/incubator-carbondata/pull/388
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/70/



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #388: [CARBONDATA-431]bigdecimal compressi...

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

    https://github.com/apache/incubator-carbondata/pull/388#discussion_r92091360
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java ---
    @@ -243,46 +270,64 @@ public static Object getCompressedValues(COMPRESSION_TYPE compType, double[] val
        * @param actualDataType
        * @return compressor based on actualdatatype
        */
    -  public static ValueCompressor getValueCompressor(DataType actualDataType) {
    -    switch (actualDataType) {
    -      case DATA_BIGINT:
    -        return new BigIntCompressor();
    -      default:
    -        return new DoubleCompressor();
    +  public static ValueCompressor getValueCompressor(
    +      CompressionFinder compressionFinder) {
    +    if (compressionFinder instanceof BigDecimalCompressionFinder) {
    +      return new BigDecimalCompressor();
    +    } else {
    +      switch (compressionFinder.getActualDataType()) {
    +        case DATA_BIGINT:
    +          return new BigIntCompressor();
    +        default:
    +          return new DoubleCompressor();
    +      }
         }
       }
     
    -  private static ValueCompressonHolder.UnCompressValue[] getUncompressedValues(
    -      COMPRESSION_TYPE[] compType, DataType[] actualDataType, DataType[] changedDataType) {
    -
    -    ValueCompressonHolder.UnCompressValue[] compressValue =
    -        new ValueCompressonHolder.UnCompressValue[changedDataType.length];
    -    for (int i = 0; i < changedDataType.length; i++) {
    -      switch (compType[i]) {
    -        case ADAPTIVE:
    -          compressValue[i] = getUnCompressNone(changedDataType[i], actualDataType[i]);
    -          break;
    -
    -        case DELTA_DOUBLE:
    -          compressValue[i] = getUnCompressDecimalMaxMin(changedDataType[i], actualDataType[i]);
    -          break;
    -
    -        case BIGINT:
    -          compressValue[i] = getUnCompressNonDecimal(changedDataType[i]);
    -          break;
    -
    -        case BIGDECIMAL:
    -          compressValue[i] = new UnCompressByteArray(UnCompressByteArray.ByteArrayType.BIG_DECIMAL);
    -          break;
    +  /**
    +   * get uncompressed object
    +   * @param compressionFinders : Compression types for measures
    +   * @return
    +   */
    +  private static UnCompressValue[] getUncompressedValues(
    +      CompressionFinder[] compressionFinders) {
    +    UnCompressValue[] unCompressedValues = new UnCompressValue[compressionFinders.length];
    +    for (int i=0; i< compressionFinders.length; i++) {
    +      unCompressedValues[i] = getUncompressedValue(compressionFinders[i]);
    +    }
    +    return unCompressedValues;
    +  }
    +  /**
    +   *
    +   * @param compressionFinder bigdecimal compression finder
    +   * @return Uncompressed measure object
    +   */
    +  private static UnCompressValue getUncompressedValue(
    +      BigDecimalCompressionFinder compressionFinder) {
    +    UnCompressValue leftPart = getUncompressedValue(compressionFinder.getLeftCompType(),
    +        compressionFinder.getLeftActualDataType(), compressionFinder.getLeftChangedDataType());
    +    UnCompressValue rightPart = getUncompressedValue(compressionFinder.getRightCompType(),
    +        compressionFinder.getRightActualDataType(), compressionFinder.getRightChangedDataType());
    +    return new UnCompressBigDecimal<>(compressionFinder, leftPart, rightPart);
    +  }
     
    -        default:
    -          compressValue[i] = getUnCompressNonDecimalMaxMin(changedDataType[i]);
    -      }
    +  /**
    +   *
    +   * @param compressionFinder for measure other then bigdecimal
    +   * @return
    +   */
    +  private static UnCompressValue getUncompressedValue(
    +      CompressionFinder compressionFinder) {
    +    if (compressionFinder instanceof BigDecimalCompressionFinder) {
    +      return getUncompressedValue(
    --- End diff --
    
    same instanceof 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #388: [CARBONDATA-431][WIP]bigdecimal compression...

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

    https://github.com/apache/incubator-carbondata/pull/388
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/69/



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #388: [WIP]bigdecimal compression.

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

    https://github.com/apache/incubator-carbondata/pull/388
  
    @ashokblend  please create dev mailing list discussion and raise Apache JIRA ISSUE in advance.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #388: [CARBONDATA-431]bigdecimal compression.

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

    https://github.com/apache/incubator-carbondata/pull/388
  
    LGTM
    Thanks for working for this


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #388: [CARBONDATA-431][WIP]bigdecimal compression...

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

    https://github.com/apache/incubator-carbondata/pull/388
  
    Build finished. 



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #388: [CARBONDATA-431]bigdecimal compression.

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

    https://github.com/apache/incubator-carbondata/pull/388
  
    review comments are handled, please check


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #388: [CARBONDATA-431]bigdecimal compression.

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

    https://github.com/apache/incubator-carbondata/pull/388
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/139/



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #388: [CARBONDATA-431]bigdecimal compressi...

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

    https://github.com/apache/incubator-carbondata/pull/388#discussion_r92088924
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/util/BigDecimalCompressionFinder.java ---
    @@ -0,0 +1,79 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.carbondata.core.util;
    +
    +import org.apache.carbondata.core.util.ValueCompressionUtil.COMPRESSION_TYPE;
    +import org.apache.carbondata.core.util.ValueCompressionUtil.DataType;
    +
    +public class BigDecimalCompressionFinder extends CompressionFinder {
    +
    +  private COMPRESSION_TYPE leftCompType;
    +
    +  private COMPRESSION_TYPE rightCompType;
    +
    +  private DataType leftActualDataType;
    +
    +  private DataType rightActualDataType;
    --- End diff --
    
    what does the ActualDataType and ChangedDataType mean?     Does actualDataType mean original data type and changedDataType is the type we store?  suggest we rename the variables more readable


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #388: [CARBONDATA-431]bigdecimal compressi...

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

    https://github.com/apache/incubator-carbondata/pull/388#discussion_r92089699
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java ---
    @@ -138,8 +141,7 @@ public static CompressionFinder getCompressionFinder(Object maxValue, Object min
         // ''b' for decimal, 'l' for long, 'n' for double
         switch (measureStoreType) {
           case 'b':
    -        return new CompressionFinder(COMPRESSION_TYPE.BIGDECIMAL, DataType.DATA_BYTE,
    -            DataType.DATA_BYTE);
    +        return getBigDecimalCompressorFinder(maxValue, minValue, 0, dataTypeSelected);
    --- End diff --
    
    if mantissa equals to 0 hear for big decimal? if it is, keep passing mantissa to getBigDecimalCompressorFinder


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #388: [CARBONDATA-431]bigdecimal compression.

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

    https://github.com/apache/incubator-carbondata/pull/388
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/75/



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #388: [CARBONDATA-431]bigdecimal compression.

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

    https://github.com/apache/incubator-carbondata/pull/388
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/137/



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #388: [CARBONDATA-431]bigdecimal compressi...

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

    https://github.com/apache/incubator-carbondata/pull/388#discussion_r92090267
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/util/ValueCompressionUtil.java ---
    @@ -243,46 +270,64 @@ public static Object getCompressedValues(COMPRESSION_TYPE compType, double[] val
        * @param actualDataType
        * @return compressor based on actualdatatype
        */
    -  public static ValueCompressor getValueCompressor(DataType actualDataType) {
    -    switch (actualDataType) {
    -      case DATA_BIGINT:
    -        return new BigIntCompressor();
    -      default:
    -        return new DoubleCompressor();
    +  public static ValueCompressor getValueCompressor(
    +      CompressionFinder compressionFinder) {
    +    if (compressionFinder instanceof BigDecimalCompressionFinder) {
    --- End diff --
    
    instanceof is a heavy operation, we should avoid to call it.
    we can add a function to CompressionFinder to detect if a compression is BigDecimal 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---