You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by jackylk <gi...@git.apache.org> on 2017/08/09 17:45:26 UTC

[GitHub] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

GitHub user jackylk opened a pull request:

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

    [CARBONDATA-1371] Support creating decoder based on encoding metadata in carbondata file

    When creating decoder for column page, it should base on metadata in the DataChunk2 in  carbondata file 
    
    This PR should be merged after #1240 

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

    $ git pull https://github.com/jackylk/incubator-carbondata rle_integration

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

    https://github.com/apache/carbondata/pull/1248.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 #1248
    
----
commit 924c43bb8856560350722e47aa60a3a64634fecf
Author: Jacky Li <ja...@qq.com>
Date:   2017-08-06T16:02:13Z

    add RLE codec

commit c43e5df5d07eb2b26f3b50e387d19e7e858a014d
Author: Jacky Li <ja...@qq.com>
Date:   2017-08-06T18:06:03Z

    fix style

commit a8b911561277c93267923f72dec2a60a3228714c
Author: Jacky Li <ja...@qq.com>
Date:   2017-08-07T03:21:34Z

    add comment

commit 97fdba24ef86eb50c942cf39142cd027bcab9f3c
Author: Jacky Li <ja...@qq.com>
Date:   2017-08-08T17:45:09Z

    add rle integration

commit b24fcf22468d8bd06e04f97bc90869afeefa27fb
Author: Jacky Li <ja...@qq.com>
Date:   2017-08-09T17:40:35Z

    fix testcase

----


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134121778
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java ---
    @@ -194,72 +202,118 @@ public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
       /**
        * Below method will be used to convert the compressed dimension chunk raw data to actual data
        *
    -   * @param dimensionRawColumnChunk dimension raw chunk
    +   * @param rawColumnPage dimension raw chunk
        * @param pageNumber              number
        * @return DimensionColumnDataChunk
        */
       @Override public DimensionColumnDataChunk convertToDimensionChunk(
    -      DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException {
    -    byte[] dataPage = null;
    -    int[] invertedIndexes = null;
    -    int[] invertedIndexesReverse = null;
    -    int[] rlePage = null;
    -    // data chunk of page
    -    DataChunk2 dimensionColumnChunk = null;
    +      DimensionRawColumnChunk rawColumnPage, int pageNumber) throws IOException, MemoryException {
         // data chunk of blocklet column
    -    DataChunk3 dataChunk3 = dimensionRawColumnChunk.getDataChunkV3();
    +    DataChunk3 dataChunk3 = rawColumnPage.getDataChunkV3();
         // get the data buffer
    -    ByteBuffer rawData = dimensionRawColumnChunk.getRawData();
    -    dimensionColumnChunk = dataChunk3.getData_chunk_list().get(pageNumber);
    +    ByteBuffer rawData = rawColumnPage.getRawData();
    +    DataChunk2 pageMetadata = dataChunk3.getData_chunk_list().get(pageNumber);
         // calculating the start point of data
         // as buffer can contain multiple column data, start point will be datachunkoffset +
         // data chunk length + page offset
    -    int copySourcePoint = dimensionRawColumnChunk.getOffSet() + dimensionChunksLength
    -        .get(dimensionRawColumnChunk.getBlockletId()) + dataChunk3.getPage_offset().get(pageNumber);
    +    int offset = rawColumnPage.getOffSet() + dimensionChunksLength
    +        .get(rawColumnPage.getBlockletId()) + dataChunk3.getPage_offset().get(pageNumber);
         // first read the data and uncompressed it
    -    dataPage = COMPRESSOR
    -        .unCompressByte(rawData.array(), copySourcePoint, dimensionColumnChunk.data_page_length);
    -    copySourcePoint += dimensionColumnChunk.data_page_length;
    +    return decodeDimension(rawColumnPage, rawData, pageMetadata, offset);
    +  }
    +
    +  private DimensionColumnDataChunk decodeDimensionByMeta(DataChunk2 pageMetadata,
    +      ByteBuffer pageData, int offset)
    +      throws IOException, MemoryException {
    +    List<Encoding> encodings = pageMetadata.getEncoders();
    +    List<ByteBuffer> encoderMetas = pageMetadata.getEncoder_meta();
    +    assert (encodings.size() == 1);
    +    assert (encoderMetas.size() == 1);
    +    Encoding encoding = encodings.get(0);
    +    ColumnPageEncoderMeta metadata = null;
    +    ByteArrayInputStream stream = new ByteArrayInputStream(encoderMetas.get(0).array());
    +    DataInputStream in = new DataInputStream(stream);
    +    switch (encoding) {
    --- End diff --
    
    I think switch case can be moved to a common factory of encoding


---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

    https://github.com/apache/carbondata/pull/1248
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/272/



---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134145165
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java ---
    @@ -49,6 +59,8 @@
      */
     public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkReaderV2V3Format {
     
    +  private EncodingStrategy strategy = new DefaultEncodingStrategy();
    --- End diff --
    
    fixed


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r133890711
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java ---
    @@ -55,7 +55,7 @@ MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int blockIndex)
        * @return
        * @throws IOException
        */
    -  MeasureColumnDataChunk convertToMeasureChunk(MeasureRawColumnChunk measureRawColumnChunk,
    +  ColumnPage convertToMeasureChunk(MeasureRawColumnChunk measureRawColumnChunk,
    --- End diff --
    
    fixed


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134147748
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java ---
    @@ -194,72 +202,118 @@ public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
       /**
        * Below method will be used to convert the compressed dimension chunk raw data to actual data
        *
    -   * @param dimensionRawColumnChunk dimension raw chunk
    +   * @param rawColumnPage dimension raw chunk
        * @param pageNumber              number
        * @return DimensionColumnDataChunk
        */
       @Override public DimensionColumnDataChunk convertToDimensionChunk(
    -      DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException {
    -    byte[] dataPage = null;
    -    int[] invertedIndexes = null;
    -    int[] invertedIndexesReverse = null;
    -    int[] rlePage = null;
    -    // data chunk of page
    -    DataChunk2 dimensionColumnChunk = null;
    +      DimensionRawColumnChunk rawColumnPage, int pageNumber) throws IOException, MemoryException {
         // data chunk of blocklet column
    -    DataChunk3 dataChunk3 = dimensionRawColumnChunk.getDataChunkV3();
    +    DataChunk3 dataChunk3 = rawColumnPage.getDataChunkV3();
         // get the data buffer
    -    ByteBuffer rawData = dimensionRawColumnChunk.getRawData();
    -    dimensionColumnChunk = dataChunk3.getData_chunk_list().get(pageNumber);
    +    ByteBuffer rawData = rawColumnPage.getRawData();
    +    DataChunk2 pageMetadata = dataChunk3.getData_chunk_list().get(pageNumber);
         // calculating the start point of data
         // as buffer can contain multiple column data, start point will be datachunkoffset +
         // data chunk length + page offset
    -    int copySourcePoint = dimensionRawColumnChunk.getOffSet() + dimensionChunksLength
    -        .get(dimensionRawColumnChunk.getBlockletId()) + dataChunk3.getPage_offset().get(pageNumber);
    +    int offset = rawColumnPage.getOffSet() + dimensionChunksLength
    +        .get(rawColumnPage.getBlockletId()) + dataChunk3.getPage_offset().get(pageNumber);
         // first read the data and uncompressed it
    -    dataPage = COMPRESSOR
    -        .unCompressByte(rawData.array(), copySourcePoint, dimensionColumnChunk.data_page_length);
    -    copySourcePoint += dimensionColumnChunk.data_page_length;
    +    return decodeDimension(rawColumnPage, rawData, pageMetadata, offset);
    +  }
    +
    +  private DimensionColumnDataChunk decodeDimensionByMeta(DataChunk2 pageMetadata,
    +      ByteBuffer pageData, int offset)
    +      throws IOException, MemoryException {
    +    List<Encoding> encodings = pageMetadata.getEncoders();
    +    List<ByteBuffer> encoderMetas = pageMetadata.getEncoder_meta();
    +    assert (encodings.size() == 1);
    +    assert (encoderMetas.size() == 1);
    +    Encoding encoding = encodings.get(0);
    +    ColumnPageEncoderMeta metadata = null;
    +    ByteArrayInputStream stream = new ByteArrayInputStream(encoderMetas.get(0).array());
    +    DataInputStream in = new DataInputStream(stream);
    +    switch (encoding) {
    --- End diff --
    
    fixed. I added a concrete method in EncodingStrategy to create decoder based on meta


---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

    https://github.com/apache/carbondata/pull/1248
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/168/



---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134121709
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java ---
    @@ -194,72 +202,118 @@ public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
       /**
        * Below method will be used to convert the compressed dimension chunk raw data to actual data
        *
    -   * @param dimensionRawColumnChunk dimension raw chunk
    +   * @param rawColumnPage dimension raw chunk
        * @param pageNumber              number
        * @return DimensionColumnDataChunk
        */
       @Override public DimensionColumnDataChunk convertToDimensionChunk(
    -      DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException {
    -    byte[] dataPage = null;
    -    int[] invertedIndexes = null;
    -    int[] invertedIndexesReverse = null;
    -    int[] rlePage = null;
    -    // data chunk of page
    -    DataChunk2 dimensionColumnChunk = null;
    +      DimensionRawColumnChunk rawColumnPage, int pageNumber) throws IOException, MemoryException {
         // data chunk of blocklet column
    -    DataChunk3 dataChunk3 = dimensionRawColumnChunk.getDataChunkV3();
    +    DataChunk3 dataChunk3 = rawColumnPage.getDataChunkV3();
         // get the data buffer
    -    ByteBuffer rawData = dimensionRawColumnChunk.getRawData();
    -    dimensionColumnChunk = dataChunk3.getData_chunk_list().get(pageNumber);
    +    ByteBuffer rawData = rawColumnPage.getRawData();
    +    DataChunk2 pageMetadata = dataChunk3.getData_chunk_list().get(pageNumber);
         // calculating the start point of data
         // as buffer can contain multiple column data, start point will be datachunkoffset +
         // data chunk length + page offset
    -    int copySourcePoint = dimensionRawColumnChunk.getOffSet() + dimensionChunksLength
    -        .get(dimensionRawColumnChunk.getBlockletId()) + dataChunk3.getPage_offset().get(pageNumber);
    +    int offset = rawColumnPage.getOffSet() + dimensionChunksLength
    +        .get(rawColumnPage.getBlockletId()) + dataChunk3.getPage_offset().get(pageNumber);
         // first read the data and uncompressed it
    -    dataPage = COMPRESSOR
    -        .unCompressByte(rawData.array(), copySourcePoint, dimensionColumnChunk.data_page_length);
    -    copySourcePoint += dimensionColumnChunk.data_page_length;
    +    return decodeDimension(rawColumnPage, rawData, pageMetadata, offset);
    +  }
    +
    +  private DimensionColumnDataChunk decodeDimensionByMeta(DataChunk2 pageMetadata,
    +      ByteBuffer pageData, int offset)
    +      throws IOException, MemoryException {
    +    List<Encoding> encodings = pageMetadata.getEncoders();
    +    List<ByteBuffer> encoderMetas = pageMetadata.getEncoder_meta();
    +    assert (encodings.size() == 1);
    +    assert (encoderMetas.size() == 1);
    +    Encoding encoding = encodings.get(0);
    +    ColumnPageEncoderMeta metadata = null;
    +    ByteArrayInputStream stream = new ByteArrayInputStream(encoderMetas.get(0).array());
    +    DataInputStream in = new DataInputStream(stream);
    +    switch (encoding) {
    +      case DIRECT_COMPRESS:
    +        DirectCompressorEncoderMeta meta = new DirectCompressorEncoderMeta();
    --- End diff --
    
    I think it can be `metadata  = new DirectCompressorEncoderMeta();`


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134121323
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java ---
    @@ -33,7 +33,7 @@
      */
     public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
     
    -  private MeasureColumnDataChunk[] dataChunks;
    +  private ColumnPage[] dataChunks;
    --- End diff --
    
    rename to columnPages


---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

    https://github.com/apache/carbondata/pull/1248
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/219/



---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134121379
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java ---
    @@ -46,17 +46,17 @@ public MeasureRawColumnChunk(int blockId, ByteBuffer rawData, int offSet, int le
       }
     
       /**
    -   * Convert all raw data with all pages to processed MeasureColumnDataChunk's
    +   * Convert all raw data with all pages to processed ColumnPage
        * @return
        */
    -  public MeasureColumnDataChunk[] convertToMeasureColDataChunks() {
    +  public ColumnPage[] convertToMeasureColDataChunks() {
    --- End diff --
    
    rename method name


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134379783
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.datastore.chunk.store;
    +
    +import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
    +import org.apache.carbondata.core.datastore.page.ColumnPage;
    +import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
    +import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
    +
    +public class ColumnPageWrapper implements DimensionColumnDataChunk {
    +
    +  private ColumnPage columnPage;
    +
    +  public ColumnPageWrapper(ColumnPage columnPage) {
    +    this.columnPage = columnPage;
    +  }
    +
    +  @Override
    +  public int fillChunkData(byte[] data, int offset, int columnIndex,
    +      KeyStructureInfo restructuringInfo) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
    +      KeyStructureInfo restructuringInfo) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
    +      KeyStructureInfo restructuringInfo) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int column,
    +      KeyStructureInfo restructuringInfo) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public byte[] getChunkData(int columnIndex) {
    +    return columnPage.getBytes(columnIndex);
    +  }
    +
    +  @Override
    +  public int getInvertedIndex(int rowId) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public boolean isNoDicitionaryColumn() {
    +    return true;
    +  }
    +
    +  @Override
    +  public int getColumnValueSize() {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public boolean isExplicitSorted() {
    +    return false;
    +  }
    +
    +  @Override
    +  public int compareTo(int index, byte[] compareValue) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public void freeMemory() {
    --- End diff --
    
    It will be done in #1265 


---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

    https://github.com/apache/carbondata/pull/1248
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/176/



---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r133776149
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java ---
    @@ -55,7 +55,7 @@ MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int blockIndex)
        * @return
        * @throws IOException
        */
    -  MeasureColumnDataChunk convertToMeasureChunk(MeasureRawColumnChunk measureRawColumnChunk,
    +  ColumnPage convertToMeasureChunk(MeasureRawColumnChunk measureRawColumnChunk,
    --- End diff --
    
    rename  the method name `convertToColumnPage`


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134135896
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java ---
    @@ -33,7 +33,7 @@
      */
     public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
     
    -  private MeasureColumnDataChunk[] dataChunks;
    +  private ColumnPage[] dataChunks;
    --- End diff --
    
    fixed


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

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


---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

    https://github.com/apache/carbondata/pull/1248
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/253/



---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134379724
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.datastore.chunk.store;
    +
    +import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
    +import org.apache.carbondata.core.datastore.page.ColumnPage;
    +import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
    +import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
    +
    +public class ColumnPageWrapper implements DimensionColumnDataChunk {
    --- End diff --
    
    Actually ColumnPageWrapper is not used in this PR, because DefaultEncodingStrategy.createEncoder will call createEncoderForDimensionLegacy for dimension columns, so CompressedDimensionChunkFileBasedReaderV3.isEncodedWithMeta will return false. 
    ColumnPageWrapper will be used in #1265 


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134381229
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java ---
    @@ -68,8 +74,38 @@ public DataType getDataType() {
         return dataType;
       }
     
    -  public Object getStatistics() {
    -    return statsCollector.getPageStats();
    +  public SimpleStatsResult getStatistics() {
    +    if (statsCollector != null) {
    +      return statsCollector.getPageStats();
    +    } else {
    +      // return a dummy result, for complex column
    +      return new SimpleStatsResult() {
    --- End diff --
    
    fixed


---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

    https://github.com/apache/carbondata/pull/1248
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/209/



---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

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


---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

    https://github.com/apache/carbondata/pull/1248
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/256/



---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

    https://github.com/apache/carbondata/pull/1248
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/243/



---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134373768
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.datastore.chunk.store;
    +
    +import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
    +import org.apache.carbondata.core.datastore.page.ColumnPage;
    +import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
    +import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
    +
    +public class ColumnPageWrapper implements DimensionColumnDataChunk {
    --- End diff --
    
    This wrapper has many default implementations like `isExplicitSorted`, doesn't it create a 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134121517
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java ---
    @@ -49,6 +59,8 @@
      */
     public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkReaderV2V3Format {
     
    +  private EncodingStrategy strategy = new DefaultEncodingStrategy();
    --- End diff --
    
    Create this instance from factory and replace in all other places 


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134121391
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java ---
    @@ -67,19 +67,19 @@ public MeasureRawColumnChunk(int blockId, ByteBuffer rawData, int offSet, int le
       }
     
       /**
    -   * Convert raw data with specified page number processed to MeasureColumnDataChunk
    +   * Convert raw data with specified page number processed to ColumnPage
        * @param index
        * @return
        */
    -  public MeasureColumnDataChunk convertToMeasureColDataChunk(int index) {
    +  public ColumnPage convertToMeasureColDataChunk(int index) {
    --- End diff --
    
    Please rename method name


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134144789
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java ---
    @@ -67,19 +67,19 @@ public MeasureRawColumnChunk(int blockId, ByteBuffer rawData, int offSet, int le
       }
     
       /**
    -   * Convert raw data with specified page number processed to MeasureColumnDataChunk
    +   * Convert raw data with specified page number processed to ColumnPage
        * @param index
        * @return
        */
    -  public MeasureColumnDataChunk convertToMeasureColDataChunk(int index) {
    +  public ColumnPage convertToMeasureColDataChunk(int index) {
    --- End diff --
    
    fixed


---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

    https://github.com/apache/carbondata/pull/1248
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/288/



---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134375715
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.datastore.chunk.store;
    +
    +import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
    +import org.apache.carbondata.core.datastore.page.ColumnPage;
    +import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
    +import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
    +
    +public class ColumnPageWrapper implements DimensionColumnDataChunk {
    +
    +  private ColumnPage columnPage;
    +
    +  public ColumnPageWrapper(ColumnPage columnPage) {
    +    this.columnPage = columnPage;
    +  }
    +
    +  @Override
    +  public int fillChunkData(byte[] data, int offset, int columnIndex,
    +      KeyStructureInfo restructuringInfo) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
    +      KeyStructureInfo restructuringInfo) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
    +      KeyStructureInfo restructuringInfo) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int column,
    +      KeyStructureInfo restructuringInfo) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public byte[] getChunkData(int columnIndex) {
    +    return columnPage.getBytes(columnIndex);
    +  }
    +
    +  @Override
    +  public int getInvertedIndex(int rowId) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public boolean isNoDicitionaryColumn() {
    +    return true;
    +  }
    +
    +  @Override
    +  public int getColumnValueSize() {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public boolean isExplicitSorted() {
    +    return false;
    +  }
    +
    +  @Override
    +  public int compareTo(int index, byte[] compareValue) {
    +    throw new UnsupportedOperationException("internal error");
    +  }
    +
    +  @Override
    +  public void freeMemory() {
    --- End diff --
    
    doesn't it require to free column page memory if it is unsafe


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134376519
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/page/ColumnPage.java ---
    @@ -68,8 +74,38 @@ public DataType getDataType() {
         return dataType;
       }
     
    -  public Object getStatistics() {
    -    return statsCollector.getPageStats();
    +  public SimpleStatsResult getStatistics() {
    +    if (statsCollector != null) {
    +      return statsCollector.getPageStats();
    +    } else {
    +      // return a dummy result, for complex column
    +      return new SimpleStatsResult() {
    --- End diff --
    
    Better create class separately and set to ColumnPage instead of creating class instance every time.


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134136239
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java ---
    @@ -46,17 +46,17 @@ public MeasureRawColumnChunk(int blockId, ByteBuffer rawData, int offSet, int le
       }
     
       /**
    -   * Convert all raw data with all pages to processed MeasureColumnDataChunk's
    +   * Convert all raw data with all pages to processed ColumnPage
        * @return
        */
    -  public MeasureColumnDataChunk[] convertToMeasureColDataChunks() {
    +  public ColumnPage[] convertToMeasureColDataChunks() {
    --- End diff --
    
    fixed


---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r134145643
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java ---
    @@ -194,72 +202,118 @@ public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
       /**
        * Below method will be used to convert the compressed dimension chunk raw data to actual data
        *
    -   * @param dimensionRawColumnChunk dimension raw chunk
    +   * @param rawColumnPage dimension raw chunk
        * @param pageNumber              number
        * @return DimensionColumnDataChunk
        */
       @Override public DimensionColumnDataChunk convertToDimensionChunk(
    -      DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException {
    -    byte[] dataPage = null;
    -    int[] invertedIndexes = null;
    -    int[] invertedIndexesReverse = null;
    -    int[] rlePage = null;
    -    // data chunk of page
    -    DataChunk2 dimensionColumnChunk = null;
    +      DimensionRawColumnChunk rawColumnPage, int pageNumber) throws IOException, MemoryException {
         // data chunk of blocklet column
    -    DataChunk3 dataChunk3 = dimensionRawColumnChunk.getDataChunkV3();
    +    DataChunk3 dataChunk3 = rawColumnPage.getDataChunkV3();
         // get the data buffer
    -    ByteBuffer rawData = dimensionRawColumnChunk.getRawData();
    -    dimensionColumnChunk = dataChunk3.getData_chunk_list().get(pageNumber);
    +    ByteBuffer rawData = rawColumnPage.getRawData();
    +    DataChunk2 pageMetadata = dataChunk3.getData_chunk_list().get(pageNumber);
         // calculating the start point of data
         // as buffer can contain multiple column data, start point will be datachunkoffset +
         // data chunk length + page offset
    -    int copySourcePoint = dimensionRawColumnChunk.getOffSet() + dimensionChunksLength
    -        .get(dimensionRawColumnChunk.getBlockletId()) + dataChunk3.getPage_offset().get(pageNumber);
    +    int offset = rawColumnPage.getOffSet() + dimensionChunksLength
    +        .get(rawColumnPage.getBlockletId()) + dataChunk3.getPage_offset().get(pageNumber);
         // first read the data and uncompressed it
    -    dataPage = COMPRESSOR
    -        .unCompressByte(rawData.array(), copySourcePoint, dimensionColumnChunk.data_page_length);
    -    copySourcePoint += dimensionColumnChunk.data_page_length;
    +    return decodeDimension(rawColumnPage, rawData, pageMetadata, offset);
    +  }
    +
    +  private DimensionColumnDataChunk decodeDimensionByMeta(DataChunk2 pageMetadata,
    +      ByteBuffer pageData, int offset)
    +      throws IOException, MemoryException {
    +    List<Encoding> encodings = pageMetadata.getEncoders();
    +    List<ByteBuffer> encoderMetas = pageMetadata.getEncoder_meta();
    +    assert (encodings.size() == 1);
    +    assert (encoderMetas.size() == 1);
    +    Encoding encoding = encodings.get(0);
    +    ColumnPageEncoderMeta metadata = null;
    +    ByteArrayInputStream stream = new ByteArrayInputStream(encoderMetas.get(0).array());
    +    DataInputStream in = new DataInputStream(stream);
    +    switch (encoding) {
    +      case DIRECT_COMPRESS:
    +        DirectCompressorEncoderMeta meta = new DirectCompressorEncoderMeta();
    --- End diff --
    
    ok, fixed


---
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] carbondata issue #1248: [CARBONDATA-1371] Support creating decoder based on ...

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

    https://github.com/apache/carbondata/pull/1248
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/167/



---
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] carbondata pull request #1248: [CARBONDATA-1371] Support creating decoder ba...

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

    https://github.com/apache/carbondata/pull/1248#discussion_r133704869
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/IndexStorageEncoder.java ---
    @@ -0,0 +1,93 @@
    +/*
    + * 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.datastore.page.encoding.dimension.legacy;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.datastore.columnar.IndexStorage;
    +import org.apache.carbondata.core.datastore.page.ColumnPage;
    +import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoder;
    +import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoderMeta;
    +import org.apache.carbondata.core.memory.MemoryException;
    +import org.apache.carbondata.format.DataChunk2;
    +import org.apache.carbondata.format.SortState;
    +
    +public abstract class IndexStorageEncoder extends ColumnPageEncoder {
    +  IndexStorage indexStorage;
    +  byte[] compressedDataPage;
    +
    +  abstract void encodeIndexStorage(ColumnPage inputPage);
    +
    +  @Override
    +  protected byte[] encodeData(ColumnPage input) throws MemoryException, IOException {
    +    assert (indexStorage == null);
    +    assert (compressedDataPage == null);
    +    encodeIndexStorage(input);
    +    assert (indexStorage != null);
    +    assert (compressedDataPage != null);
    +    ByteArrayOutputStream stream = new ByteArrayOutputStream();
    +    DataOutputStream out = new DataOutputStream(stream);
    +    out.write(compressedDataPage);
    +    if (indexStorage.getRowIdPageLengthInBytes() > 0) {
    +      out.writeInt(indexStorage.getRowIdPageLengthInBytes());
    +      short[] rowIdPage = (short[])indexStorage.getRowIdPage();
    +      for (short rowId : rowIdPage) {
    +        out.writeShort(rowId);
    +      }
    +      if (indexStorage.getRowIdRlePageLengthInBytes() > 0) {
    +        short[] rowIdRlePage = (short[])indexStorage.getRowIdRlePage();
    +        for (short rowIdRle : rowIdRlePage) {
    +          out.writeShort(rowIdRle);
    +        }
    +      }
    +    }
    +    if (indexStorage.getDataRlePageLengthInBytes() > 0) {
    +      short[] dataRlePage = (short[])indexStorage.getDataRlePage();
    +      for (short dataRle : dataRlePage) {
    +        out.writeShort(dataRle);
    +      }
    +    }
    +    return stream.toByteArray();
    +  }
    +
    +  @Override
    +  protected ColumnPageEncoderMeta getEncoderMeta(ColumnPage inputPage) {
    +    return null;
    --- End diff --
    
    I have not changed this behavior, this is the way currently writing dimension column.
    I will add new encoding for no dictionary string column in next PR.


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