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

[GitHub] carbondata pull request #2670: [WIP] Support binary datatype

GitHub user QiangCai opened a pull request:

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

    [WIP] Support binary datatype

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


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

    $ git pull https://github.com/QiangCai/carbondata binary

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

    https://github.com/apache/carbondata/pull/2670.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 #2670
    
----
commit cc39b9d7ec4670549a12084c7248d7d2730caaea
Author: QiangCai <qi...@...>
Date:   2018-08-29T11:05:27Z

    support binary datatype

----


---

[GitHub] carbondata pull request #2670: [WIP] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670#discussion_r213662016
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/BinaryFieldConverterImpl.java ---
    @@ -0,0 +1,90 @@
    +/*
    + * 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.processing.loading.converter.impl;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.datastore.row.CarbonRow;
    +import org.apache.carbondata.core.metadata.datatype.DataType;
    +import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
    +import org.apache.carbondata.processing.loading.DataField;
    +import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
    +import org.apache.carbondata.processing.loading.converter.FieldConverter;
    +import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
    +import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
    +
    +import org.apache.commons.codec.DecoderException;
    +import org.apache.commons.codec.binary.Hex;
    +/**
    + * Converter for binary
    + */
    +public class BinaryFieldConverterImpl implements FieldConverter {
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(BinaryFieldConverterImpl.class.getName());
    +  private int index;
    +  private DataType dataType;
    +  private CarbonMeasure measure;
    +  private String nullformat;
    +  private boolean isEmptyBadRecord;
    +  private DataField dataField;
    +  public BinaryFieldConverterImpl(DataField dataField, String nullformat, int index,
    +      boolean isEmptyBadRecord) {
    +    this.dataType = dataField.getColumn().getDataType();
    +    this.measure = (CarbonMeasure) dataField.getColumn();
    +    this.nullformat = nullformat;
    +    this.index = index;
    +    this.isEmptyBadRecord = isEmptyBadRecord;
    +    this.dataField = dataField;
    +  }
    +  @Override public void convert(CarbonRow row, BadRecordLogHolder logHolder)
    +      throws CarbonDataLoadingException {
    +    String value = row.getString(index);
    +    if (value != null) {
    +      try {
    +        byte[] bytes = Hex.decodeHex(value.toCharArray());
    --- End diff --
    
    So it seems the 'Binary' datatype is actually 'HEX' encoded, not 0101..


---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Build Failed with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/434/



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/535/



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/266/



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Build Failed  with Spark 2.3.1, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.3/8505/



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/228/



---

[GitHub] carbondata issue #2670: [WIP] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Problems that may be ignored are that during loading
    1. We use a buffer to store one row and the row is 2MB fow now
    2. For a column page, we compress it as a byte array and its length is of 'INT' datatype, which means that a page at most 2GB.
    So do we have any restrictions on the binary data type?


---

[GitHub] carbondata issue #2670: [WIP] Support binary datatype

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

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



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

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



---

[GitHub] carbondata issue #2670: [WIP] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    @xuchuanyin 
    Yes, better to check the limitation. 
    What's your opinion about how to fix it? 


---

[GitHub] carbondata pull request #2670: [WIP] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670#discussion_r214566361
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/BinaryFieldConverterImpl.java ---
    @@ -0,0 +1,90 @@
    +/*
    + * 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.processing.loading.converter.impl;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.datastore.row.CarbonRow;
    +import org.apache.carbondata.core.metadata.datatype.DataType;
    +import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
    +import org.apache.carbondata.processing.loading.DataField;
    +import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
    +import org.apache.carbondata.processing.loading.converter.FieldConverter;
    +import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
    +import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
    +
    +import org.apache.commons.codec.DecoderException;
    +import org.apache.commons.codec.binary.Hex;
    +/**
    + * Converter for binary
    + */
    +public class BinaryFieldConverterImpl implements FieldConverter {
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(BinaryFieldConverterImpl.class.getName());
    +  private int index;
    +  private DataType dataType;
    +  private CarbonMeasure measure;
    +  private String nullformat;
    +  private boolean isEmptyBadRecord;
    +  private DataField dataField;
    +  public BinaryFieldConverterImpl(DataField dataField, String nullformat, int index,
    +      boolean isEmptyBadRecord) {
    +    this.dataType = dataField.getColumn().getDataType();
    +    this.measure = (CarbonMeasure) dataField.getColumn();
    +    this.nullformat = nullformat;
    +    this.index = index;
    +    this.isEmptyBadRecord = isEmptyBadRecord;
    +    this.dataField = dataField;
    +  }
    +  @Override public void convert(CarbonRow row, BadRecordLogHolder logHolder)
    +      throws CarbonDataLoadingException {
    +    String value = row.getString(index);
    +    if (value != null) {
    +      try {
    +        byte[] bytes = Hex.decodeHex(value.toCharArray());
    --- End diff --
    
    Hex string is smaller than binary string


---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Build Failed  with Spark 2.3.1, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.3/8504/



---

[GitHub] carbondata pull request #2670: [WIP] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670#discussion_r213660172
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/BinaryFieldConverterImpl.java ---
    @@ -0,0 +1,90 @@
    +/*
    + * 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.processing.loading.converter.impl;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.datastore.row.CarbonRow;
    +import org.apache.carbondata.core.metadata.datatype.DataType;
    +import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
    +import org.apache.carbondata.processing.loading.DataField;
    +import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
    +import org.apache.carbondata.processing.loading.converter.FieldConverter;
    +import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
    +import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
    +
    +import org.apache.commons.codec.DecoderException;
    +import org.apache.commons.codec.binary.Hex;
    +/**
    + * Converter for binary
    + */
    +public class BinaryFieldConverterImpl implements FieldConverter {
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(BinaryFieldConverterImpl.class.getName());
    +  private int index;
    +  private DataType dataType;
    +  private CarbonMeasure measure;
    +  private String nullformat;
    +  private boolean isEmptyBadRecord;
    +  private DataField dataField;
    +  public BinaryFieldConverterImpl(DataField dataField, String nullformat, int index,
    +      boolean isEmptyBadRecord) {
    +    this.dataType = dataField.getColumn().getDataType();
    +    this.measure = (CarbonMeasure) dataField.getColumn();
    +    this.nullformat = nullformat;
    +    this.index = index;
    +    this.isEmptyBadRecord = isEmptyBadRecord;
    +    this.dataField = dataField;
    +  }
    +  @Override public void convert(CarbonRow row, BadRecordLogHolder logHolder)
    +      throws CarbonDataLoadingException {
    +    String value = row.getString(index);
    +    if (value != null) {
    +      try {
    +        byte[] bytes = Hex.decodeHex(value.toCharArray());
    +        row.update(bytes, index);
    +      } catch (DecoderException e) {
    +        if (LOGGER.isDebugEnabled()) {
    +          LOGGER.debug("Can not convert value to Binary type value. Value considered as null.");
    +        }
    +        logHolder.setReason(
    +            CarbonDataProcessorUtil.prepareFailureReason(measure.getColName(), dataType));
    +        row.update(null, index);
    +      }
    +    }
    +  }
    +
    +  @Override public Object convert(Object value, BadRecordLogHolder logHolder)
    --- End diff --
    
    Move 'Override' to the previous line


---

[GitHub] carbondata pull request #2670: [WIP] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670#discussion_r213660098
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/BinaryFieldConverterImpl.java ---
    @@ -0,0 +1,90 @@
    +/*
    + * 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.processing.loading.converter.impl;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.datastore.row.CarbonRow;
    +import org.apache.carbondata.core.metadata.datatype.DataType;
    +import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
    +import org.apache.carbondata.processing.loading.DataField;
    +import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
    +import org.apache.carbondata.processing.loading.converter.FieldConverter;
    +import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
    +import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
    +
    +import org.apache.commons.codec.DecoderException;
    +import org.apache.commons.codec.binary.Hex;
    +/**
    + * Converter for binary
    + */
    +public class BinaryFieldConverterImpl implements FieldConverter {
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(BinaryFieldConverterImpl.class.getName());
    +  private int index;
    +  private DataType dataType;
    +  private CarbonMeasure measure;
    +  private String nullformat;
    +  private boolean isEmptyBadRecord;
    +  private DataField dataField;
    +  public BinaryFieldConverterImpl(DataField dataField, String nullformat, int index,
    +      boolean isEmptyBadRecord) {
    +    this.dataType = dataField.getColumn().getDataType();
    +    this.measure = (CarbonMeasure) dataField.getColumn();
    +    this.nullformat = nullformat;
    +    this.index = index;
    +    this.isEmptyBadRecord = isEmptyBadRecord;
    +    this.dataField = dataField;
    +  }
    +  @Override public void convert(CarbonRow row, BadRecordLogHolder logHolder)
    --- End diff --
    
    why not call the next method internally?


---

[GitHub] carbondata issue #2670: [WIP] Support binary datatype

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

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



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Build Failed  with Spark 2.3.1, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.3/8467/



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    @QiangCai I'm not sure about how to solve this. Maybe you can try origin spark and impala and observe how they solve this.


---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

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



---

[GitHub] carbondata issue #2670: [WIP] Support binary datatype

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

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



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

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



---

[GitHub] carbondata pull request #2670: [CARBONDATA-2917] Support binary datatype

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

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


---

[GitHub] carbondata issue #2670: [WIP] Support binary datatype

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

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



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/265/



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Build Failed  with Spark 2.3.1, Please check CI http://136.243.101.176:8080/job/carbondataprbuilder2.3/8780/



---

[GitHub] carbondata issue #2670: [CARBONDATA-2917] Support binary datatype

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

    https://github.com/apache/carbondata/pull/2670
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/13/



---

[GitHub] carbondata issue #2670: [WIP] Support binary datatype

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

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



---