This is an automated email from the ASF dual-hosted git repository. jackylk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/carbondata.git
The following commit(s) were added to refs/heads/master by this push: new ebbecb5 [CARBONDATA-3761] Remove redundant conversion for complex type insert ebbecb5 is described below commit ebbecb5285ad4272022b9cd7b2c23406b277987c Author: ajantha-bhat <ajanthab...@gmail.com> AuthorDate: Mon Mar 30 19:08:25 2020 +0530 [CARBONDATA-3761] Remove redundant conversion for complex type insert Why is this PR needed? In PrimitiveDataType#writeByteArray DataTypeUtil.parseValue(**input.toString()**, carbonDimension) Here we convert every complex child element to string and then parse as an object to handle bad records. Which leads to heavy GC DatatypeUtil#getBytesDataDataTypeForNoDictionaryColumn -> double,float, byte, decimal case is missing. so we convert them to string and then convert to bytes. which create more redundant objects What changes were proposed in this PR? For new Insert into flow, no need to handle bad records for complex types as it is already validated in source table. So, use object directly. This can decrease the memory foot print for complex type insert Add a case for double,float, byte, decimal data type. Does this PR introduce any user interface change? No Is any new testcase added? No This closes #3687 --- .../apache/carbondata/core/util/DataTypeUtil.java | 9 + .../apache/carbondata/spark/util/CommonUtil.scala | 9 +- .../processing/datatypes/ArrayDataType.java | 6 +- .../processing/datatypes/GenericDataType.java | 4 +- .../processing/datatypes/PrimitiveDataType.java | 266 +++++++++++---------- .../processing/datatypes/StructDataType.java | 8 +- .../converter/impl/ComplexFieldConverterImpl.java | 2 +- .../InputProcessorStepWithNoConverterImpl.java | 9 +- 8 files changed, 173 insertions(+), 140 deletions(-) diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java index 7c5dc1d..63b634d 100644 --- a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java @@ -476,6 +476,15 @@ public final class DataTypeUtil { return ByteUtil.toXorBytes((Integer) dimensionValue); } else if (actualDataType == DataTypes.LONG) { return ByteUtil.toXorBytes((Long) dimensionValue); + } else if (actualDataType == DataTypes.DOUBLE) { + return ByteUtil.toXorBytes((double) dimensionValue); + } else if (actualDataType == DataTypes.FLOAT) { + return ByteUtil.toXorBytes((float) dimensionValue); + } else if (DataTypes.isDecimal(actualDataType)) { + // Need to make BigDecimal object, else ByteUtil.toBytes will have precision loss + return bigDecimalToByte(new BigDecimal(dimensionValue.toString())); + } else if (actualDataType == DataTypes.BYTE) { + return ByteUtil.toXorBytes((byte) dimensionValue); } else if (actualDataType == DataTypes.TIMESTAMP) { return ByteUtil.toXorBytes((Long) dimensionValue); } else if (actualDataType == DataTypes.BINARY) { diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala index d2c53f6..7e98605 100644 --- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala +++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala @@ -936,7 +936,8 @@ object CommonUtil { dataFieldsWithComplexDataType(fields(i).name).asInstanceOf[ArrayDataType] .writeByteArray(result.asInstanceOf[ArrayObject], dataOutputStream, - badRecordLogHolder) + badRecordLogHolder, + true) dataOutputStream.close() data(i) = byteArray.toByteArray.asInstanceOf[AnyRef] case structType : StructType => @@ -947,7 +948,8 @@ object CommonUtil { dataFieldsWithComplexDataType(fields(i).name).asInstanceOf[StructDataType] .writeByteArray(result.asInstanceOf[StructObject], dataOutputStream, - badRecordLogHolder) + badRecordLogHolder, + true) dataOutputStream.close() data(i) = byteArray.toByteArray.asInstanceOf[AnyRef] case mapType : MapType => @@ -958,7 +960,8 @@ object CommonUtil { dataFieldsWithComplexDataType(fields(i).name).asInstanceOf[ArrayDataType] .writeByteArray(result.asInstanceOf[ArrayObject], dataOutputStream, - badRecordLogHolder) + badRecordLogHolder, + true) dataOutputStream.close() data(i) = byteArray.toByteArray.asInstanceOf[AnyRef] case other => diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java index ad17a55..46e4749 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java +++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java @@ -172,15 +172,15 @@ public class ArrayDataType implements GenericDataType<ArrayObject> { @Override public void writeByteArray(ArrayObject input, DataOutputStream dataOutputStream, - BadRecordLogHolder logHolder) throws IOException { + BadRecordLogHolder logHolder, Boolean isWithoutConverter) throws IOException { if (input == null) { dataOutputStream.writeInt(1); - children.writeByteArray(null, dataOutputStream, logHolder); + children.writeByteArray(null, dataOutputStream, logHolder, isWithoutConverter); } else { Object[] data = input.getData(); dataOutputStream.writeInt(data.length); for (Object eachInput : data) { - children.writeByteArray(eachInput, dataOutputStream, logHolder); + children.writeByteArray(eachInput, dataOutputStream, logHolder, isWithoutConverter); } } } diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java index f1b76f1..5e8ac08 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java +++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java @@ -57,9 +57,11 @@ public interface GenericDataType<T> extends Serializable { /** * writes to byte stream * @param dataOutputStream + * @param isWithoutConverter * @throws IOException */ - void writeByteArray(T input, DataOutputStream dataOutputStream, BadRecordLogHolder logHolder) + void writeByteArray(T input, DataOutputStream dataOutputStream, BadRecordLogHolder logHolder, + Boolean isWithoutConverter) throws IOException; /** diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java index 45d2a83..f1b6f1c 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java +++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java @@ -21,8 +21,6 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.Charset; -import java.text.ParseException; -import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.List; @@ -240,144 +238,164 @@ public class PrimitiveDataType implements GenericDataType<Object> { @Override public void writeByteArray(Object input, DataOutputStream dataOutputStream, - BadRecordLogHolder logHolder) throws IOException { - String parsedValue = - input == null ? null : DataTypeUtil.parseValue(input.toString(), carbonDimension); - String message = logHolder.getColumnMessageMap().get(carbonDimension.getColName()); - if (this.isDictionary) { - Integer surrogateKey; - if (null == parsedValue) { - surrogateKey = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY; - if (null == message) { - message = CarbonDataProcessorUtil - .prepareFailureReason(carbonDimension.getColName(), carbonDimension.getDataType()); - logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message); - logHolder.setReason(message); - } - } else { - if (dictionaryGenerator instanceof DirectDictionary && input instanceof Long) { - surrogateKey = ((DirectDictionary) dictionaryGenerator).generateKey((long) input); - } else { - surrogateKey = dictionaryGenerator.getOrGenerateKey(parsedValue); - } - if (surrogateKey == CarbonCommonConstants.INVALID_SURROGATE_KEY) { - surrogateKey = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY; - message = CarbonDataProcessorUtil - .prepareFailureReason(carbonDimension.getColName(), carbonDimension.getDataType()); - logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message); - logHolder.setReason(message); - } - } - dataOutputStream.writeInt(surrogateKey); - } else { - // Transform into ByteArray for No Dictionary. - // TODO have to refactor and place all the cases present in NonDictionaryFieldConverterImpl - if (null == parsedValue && this.carbonDimension.getDataType() != DataTypes.STRING) { - updateNullValue(dataOutputStream, logHolder); - } else if (null == parsedValue || parsedValue.equals(nullFormat)) { + BadRecordLogHolder logHolder, Boolean isWithoutConverter) throws IOException { + String parsedValue = null; + // write null value + if (null == input || + (this.carbonDimension.getDataType() == DataTypes.STRING && input.equals(nullFormat))) { + updateNullValue(dataOutputStream, logHolder); + return; + } + // write null value after converter + if (!isWithoutConverter) { + parsedValue = DataTypeUtil.parseValue(input.toString(), carbonDimension); + if (null == parsedValue || (this.carbonDimension.getDataType() == DataTypes.STRING + && parsedValue.equals(nullFormat))) { updateNullValue(dataOutputStream, logHolder); - } else { - String dateFormat = null; - if (this.carbonDimension.getDataType() == DataTypes.DATE) { - dateFormat = carbonDimension.getDateFormat(); - } else if (this.carbonDimension.getDataType() == DataTypes.TIMESTAMP) { - dateFormat = carbonDimension.getTimestampFormat(); - } - try { - if (!this.carbonDimension.getUseActualData()) { - byte[] value = null; - if (isDirectDictionary) { - int surrogateKey; - if (!(input instanceof Long) && !(input instanceof Integer)) { - SimpleDateFormat parser = new SimpleDateFormat(getDateFormat(carbonDimension)); - parser.parse(parsedValue); - } - // If the input is a long value then this means that logical type was provided by - // the user using AvroCarbonWriter. In this case directly generate surrogate key - // using dictionaryGenerator. - if (dictionaryGenerator instanceof DirectDictionary && input instanceof Long) { - surrogateKey = ((DirectDictionary) dictionaryGenerator).generateKey((long) input); - } else if (dictionaryGenerator instanceof DirectDictionary - && input instanceof Integer) { - // In case of file format, for complex type date or time type, input data comes as a - // Integer object, so just assign the surrogate key with the input object value - surrogateKey = (int) input; - } else { - surrogateKey = dictionaryGenerator.getOrGenerateKey(parsedValue); - } - if (surrogateKey == CarbonCommonConstants.INVALID_SURROGATE_KEY) { - value = new byte[0]; + return; + } + } + // Transform into ByteArray for No Dictionary. + try { + if (!this.carbonDimension.getUseActualData()) { + byte[] value; + if (isDirectDictionary) { + value = writeDirectDictionary(input, parsedValue, isWithoutConverter); + } else { + // If the input is a long value then this means that logical type was provided by + // the user using AvroCarbonWriter. In this case directly generate Bytes from value. + if (this.carbonDimension.getDataType().equals(DataTypes.DATE) + || this.carbonDimension.getDataType().equals(DataTypes.TIMESTAMP) + && input instanceof Long) { + if (dictionaryGenerator != null) { + value = ByteUtil.toXorBytes(((DirectDictionary) dictionaryGenerator) + .generateKey((long) input)); + } else { + if (isWithoutConverter) { + value = ByteUtil.toXorBytes((Long)input); } else { - value = ByteUtil.toXorBytes(surrogateKey); + value = ByteUtil.toXorBytes(Long.parseLong(parsedValue)); } + } + } else if (this.carbonDimension.getDataType().equals(DataTypes.BINARY)) { + // write binary data type + if (binaryDecoder == null) { + value = DataTypeUtil.getBytesDataDataTypeForNoDictionaryColumn(input, + this.carbonDimension.getDataType()); } else { - // If the input is a long value then this means that logical type was provided by - // the user using AvroCarbonWriter. In this case directly generate Bytes from value. - if (this.carbonDimension.getDataType().equals(DataTypes.DATE) - || this.carbonDimension.getDataType().equals(DataTypes.TIMESTAMP) - && input instanceof Long) { - if (dictionaryGenerator != null) { - value = ByteUtil.toXorBytes(((DirectDictionary) dictionaryGenerator) - .generateKey((long) input)); - } else { - value = ByteUtil.toXorBytes(Long.parseLong(parsedValue)); - } - } else if (this.carbonDimension.getDataType().equals(DataTypes.BINARY)) { - if (binaryDecoder == null) { - value = DataTypeUtil.getBytesDataDataTypeForNoDictionaryColumn(input, - this.carbonDimension.getDataType()); - } else { - value = binaryDecoder.decode(parsedValue); - } + if (isWithoutConverter) { + value = binaryDecoder.decode((String)input); } else { - value = DataTypeUtil.getBytesBasedOnDataTypeForNoDictionaryColumn(parsedValue, - this.carbonDimension.getDataType(), dateFormat); - } - if (this.carbonDimension.getDataType() == DataTypes.STRING - && value.length > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) { - throw new CarbonDataLoadingException("Dataload failed, String size cannot exceed " - + CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes"); + value = binaryDecoder.decode(parsedValue); } } - updateValueToByteStream(dataOutputStream, value); } else { - Object value; - if (dictionaryGenerator instanceof DirectDictionary - && input instanceof Long) { - value = ByteUtil.toXorBytes( - ((DirectDictionary) dictionaryGenerator).generateKey((long) input)); + // write other data types + if (isWithoutConverter) { + value = DataTypeUtil.getBytesDataDataTypeForNoDictionaryColumn(input, + this.carbonDimension.getDataType()); } else { - value = DataTypeUtil.getDataDataTypeForNoDictionaryColumn(parsedValue, - this.carbonDimension.getDataType(), dateFormat); - } - if (this.carbonDimension.getDataType() == DataTypes.STRING - && value.toString().length() > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) { - throw new CarbonDataLoadingException("Dataload failed, String size cannot exceed " - + CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes"); - } - if (parsedValue.length() > 0) { - updateValueToByteStream(dataOutputStream, - parsedValue.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET))); - } else { - updateNullValue(dataOutputStream, logHolder); + value = DataTypeUtil.getBytesBasedOnDataTypeForNoDictionaryColumn(parsedValue, + this.carbonDimension.getDataType(), getDateOrTimeFormat()); } } - } catch (NumberFormatException e) { - // Update logHolder for bad record and put null in dataOutputStream. - updateNullValue(dataOutputStream, logHolder); - } catch (CarbonDataLoadingException e) { - throw e; - } catch (ParseException ex) { - updateNullValue(dataOutputStream, logHolder); - } catch (Throwable ex) { - // TODO have to implemented the Bad Records LogHolder. - // Same like NonDictionaryFieldConverterImpl. - throw ex; + if (this.carbonDimension.getDataType() == DataTypes.STRING + && value.length > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) { + throw new CarbonDataLoadingException("Dataload failed, String size cannot exceed " + + CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes"); + } } + updateValueToByteStream(dataOutputStream, value); + } else { + byte[] value; + if (dictionaryGenerator instanceof DirectDictionary + && input instanceof Long) { + value = ByteUtil.toXorBytes( + ((DirectDictionary) dictionaryGenerator).generateKey((long) input)); + } else { + if (isWithoutConverter) { + value = DataTypeUtil.getBytesDataDataTypeForNoDictionaryColumn(input, + this.carbonDimension.getDataType()); + } else { + value = DataTypeUtil.getBytesBasedOnDataTypeForNoDictionaryColumn(parsedValue, + this.carbonDimension.getDataType(), getDateOrTimeFormat()); + } + } + checkAndWriteByteArray(input, dataOutputStream, logHolder, isWithoutConverter, parsedValue, + value); + } + } catch (NumberFormatException e) { + // Update logHolder for bad record and put null in dataOutputStream. + updateNullValue(dataOutputStream, logHolder); + } + } + + private void checkAndWriteByteArray(Object input, DataOutputStream dataOutputStream, + BadRecordLogHolder logHolder, Boolean isWithoutConverter, String parsedValue, byte[] value) + throws IOException { + if (isWithoutConverter) { + if (this.carbonDimension.getDataType() == DataTypes.STRING && input instanceof String + && ((String)input).length() > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) { + throw new CarbonDataLoadingException("Dataload failed, String size cannot exceed " + + CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes"); + } + updateValueToByteStream(dataOutputStream, value); + } else { + if (this.carbonDimension.getDataType() == DataTypes.STRING + && value.length > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) { + throw new CarbonDataLoadingException("Dataload failed, String size cannot exceed " + + CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes"); + } + if (parsedValue.length() > 0) { + updateValueToByteStream(dataOutputStream, + parsedValue.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET))); + } else { + updateNullValue(dataOutputStream, logHolder); } } } + private String getDateOrTimeFormat() { + if (this.carbonDimension.getDataType() == DataTypes.DATE) { + return carbonDimension.getDateFormat(); + } else if (this.carbonDimension.getDataType() == DataTypes.TIMESTAMP) { + return carbonDimension.getTimestampFormat(); + } else { + return null; + } + } + + private byte[] writeDirectDictionary( + Object input, + String parsedValue, + Boolean isWithoutConverter) { + byte[] value; + int surrogateKey; + // If the input is a long value then this means that logical type was provided by + // the user using AvroCarbonWriter. In this case directly generate surrogate key + // using dictionaryGenerator. + if (input instanceof Long) { + surrogateKey = ((DirectDictionary) dictionaryGenerator).generateKey((long) input); + } else if (input instanceof Integer) { + // In case of file format, for complex type date or time type, input data comes as a + // Integer object, so just assign the surrogate key with the input object value + surrogateKey = (int) input; + } else { + // in case of data frame insert, date can come as string value + if (isWithoutConverter) { + surrogateKey = dictionaryGenerator.getOrGenerateKey(input.toString()); + } else { + surrogateKey = dictionaryGenerator.getOrGenerateKey(parsedValue); + } + } + if (surrogateKey == CarbonCommonConstants.INVALID_SURROGATE_KEY) { + value = new byte[0]; + } else { + value = ByteUtil.toXorBytes(surrogateKey); + } + return value; + } + private void updateValueToByteStream(DataOutputStream dataOutputStream, byte[] value) throws IOException { if (DataTypeUtil.isByteArrayComplexChildColumn(dataType)) { diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java index 701bbd3..ab5e71f 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java +++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java @@ -174,21 +174,21 @@ public class StructDataType implements GenericDataType<StructObject> { @Override public void writeByteArray(StructObject input, DataOutputStream dataOutputStream, - BadRecordLogHolder logHolder) throws IOException { + BadRecordLogHolder logHolder, Boolean isWithoutConverter) throws IOException { dataOutputStream.writeShort(children.size()); if (input == null) { for (int i = 0; i < children.size(); i++) { - children.get(i).writeByteArray(null, dataOutputStream, logHolder); + children.get(i).writeByteArray(null, dataOutputStream, logHolder, isWithoutConverter); } } else { Object[] data = input.getData(); for (int i = 0; i < data.length && i < children.size(); i++) { - children.get(i).writeByteArray(data[i], dataOutputStream, logHolder); + children.get(i).writeByteArray(data[i], dataOutputStream, logHolder, isWithoutConverter); } // For other children elements which don't have data, write empty for (int i = data.length; i < children.size(); i++) { - children.get(i).writeByteArray(null, dataOutputStream, logHolder); + children.get(i).writeByteArray(null, dataOutputStream, logHolder, isWithoutConverter); } } } diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java index 5b55f9c..b00f1fc 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java @@ -53,7 +53,7 @@ public class ComplexFieldConverterImpl implements FieldConverter { ByteArrayOutputStream byteArray = new ByteArrayOutputStream(); DataOutputStream dataOutputStream = new DataOutputStream(byteArray); try { - genericDataType.writeByteArray(value, dataOutputStream, logHolder); + genericDataType.writeByteArray(value, dataOutputStream, logHolder, false); dataOutputStream.close(); return byteArray.toByteArray(); } catch (Exception e) { diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java index 589b49d..eba5b80 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java @@ -383,7 +383,7 @@ public class InputProcessorStepWithNoConverterImpl extends AbstractDataLoadProce if (dataTypes[i].isComplexType() && isHivePartitionTable) { newData[i] = data[orderOfData[i]]; } else if (dataTypes[i].isComplexType()) { - getComplexTypeByteArray(newData, i, data, dataFields[i], orderOfData[i]); + getComplexTypeByteArray(newData, i, data, dataFields[i], orderOfData[i], false); } else { DataType dataType = dataFields[i].getColumn().getDataType(); if (dataType == DataTypes.DATE && data[orderOfData[i]] instanceof Long) { @@ -419,7 +419,7 @@ public class InputProcessorStepWithNoConverterImpl extends AbstractDataLoadProce // keep the no dictionary measure column as original data newData[i] = data[i]; } else if (dataTypes[i].isComplexType()) { - getComplexTypeByteArray(newData, i, data, dataFields[i], i); + getComplexTypeByteArray(newData, i, data, dataFields[i], i, true); } else if (dataTypes[i] == DataTypes.DATE && data[i] instanceof Long) { if (dateDictionaryGenerator == null) { dateDictionaryGenerator = DirectDictionaryKeyGeneratorFactory @@ -435,13 +435,14 @@ public class InputProcessorStepWithNoConverterImpl extends AbstractDataLoadProce } private void getComplexTypeByteArray(Object[] newData, int index, Object[] data, - DataField dataField, int orderedIndex) { + DataField dataField, int orderedIndex, boolean isWithoutConverter) { ByteArrayOutputStream byteArray = new ByteArrayOutputStream(); DataOutputStream dataOutputStream = new DataOutputStream(byteArray); try { GenericDataType complextType = dataFieldsWithComplexDataType.get(dataField.getColumn().getOrdinal()); - complextType.writeByteArray(data[orderedIndex], dataOutputStream, logHolder); + complextType + .writeByteArray(data[orderedIndex], dataOutputStream, logHolder, isWithoutConverter); dataOutputStream.close(); newData[index] = byteArray.toByteArray(); } catch (BadRecordFoundException e) {