Skip to content

Commit

Permalink
[CARBONDATA-3261] support float and byte data type reading from prest…
Browse files Browse the repository at this point in the history
…o carbon

[CARBONDATA-3261] support float and byte data type reading from presto

problem: support float and byte reading from presto

cause: currently float and byte cannot be read in presto due to code issue. It was going as double data type. Hence array out of bound issue used to come as float/byte read from double stream reader.

solution: Implement a new stream reader for float and byte.

This closes #3085
  • Loading branch information
ajantha-bhat authored and ravipesala committed Jan 21, 2019
1 parent e485e14 commit 556e220
Show file tree
Hide file tree
Showing 5 changed files with 273 additions and 20 deletions.
Expand Up @@ -28,8 +28,10 @@
import org.apache.carbondata.core.metadata.datatype.StructField;
import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
import org.apache.carbondata.presto.readers.BooleanStreamReader;
import org.apache.carbondata.presto.readers.ByteStreamReader;
import org.apache.carbondata.presto.readers.DecimalSliceStreamReader;
import org.apache.carbondata.presto.readers.DoubleStreamReader;
import org.apache.carbondata.presto.readers.FloatStreamReader;
import org.apache.carbondata.presto.readers.IntegerStreamReader;
import org.apache.carbondata.presto.readers.LongStreamReader;
import org.apache.carbondata.presto.readers.ObjectStreamReader;
Expand Down Expand Up @@ -89,6 +91,10 @@ public static CarbonColumnVectorImpl createDirectStreamReader(int batchSize, Dat
return new LongStreamReader(batchSize, field.getDataType(), dictionary);
} else if (dataType == DataTypes.DOUBLE) {
return new DoubleStreamReader(batchSize, field.getDataType(), dictionary);
} else if (dataType == DataTypes.FLOAT) {
return new FloatStreamReader(batchSize, field.getDataType(), dictionary);
} else if (dataType == DataTypes.BYTE) {
return new ByteStreamReader(batchSize, field.getDataType(), dictionary);
} else if (dataType == DataTypes.STRING) {
return new SliceStreamReader(batchSize, field.getDataType(), dictionary);
} else if (DataTypes.isDecimal(dataType)) {
Expand Down
Expand Up @@ -26,7 +26,6 @@
import org.apache.carbondata.core.datastore.block.TableBlockInfo;
import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.core.metadata.datatype.StructField;
import org.apache.carbondata.core.metadata.encoder.Encoding;
Expand Down Expand Up @@ -201,17 +200,8 @@ private void initBatch() {
}

for (ProjectionMeasure msr : queryMeasures) {
DataType dataType = msr.getMeasure().getDataType();
if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.SHORT || dataType == DataTypes.INT
|| dataType == DataTypes.LONG) {
fields[msr.getOrdinal()] =
new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
} else if (DataTypes.isDecimal(dataType)) {
fields[msr.getOrdinal()] =
new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
} else {
fields[msr.getOrdinal()] = new StructField(msr.getColumnName(), DataTypes.DOUBLE);
}
fields[msr.getOrdinal()] =
new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
}

columnarBatch =
Expand Down
@@ -0,0 +1,105 @@
/*
* 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.presto.readers;

import org.apache.carbondata.core.cache.dictionary.Dictionary;
import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
import org.apache.carbondata.core.util.DataTypeUtil;

import com.facebook.presto.spi.block.Block;
import com.facebook.presto.spi.block.BlockBuilder;
import com.facebook.presto.spi.type.AbstractType;
import com.facebook.presto.spi.type.TinyintType;

/**
* Class for Reading the Byte(tiny int) value and setting it in Block
*/
public class ByteStreamReader extends CarbonColumnVectorImpl implements PrestoVectorBlockBuilder {

protected int batchSize;

protected AbstractType type = TinyintType.TINYINT;

protected BlockBuilder builder;

private Dictionary dictionary;


public ByteStreamReader(int batchSize, DataType dataType, Dictionary dictionary) {
super(batchSize, dataType);
this.batchSize = batchSize;
this.builder = type.createBlockBuilder(null, batchSize);
this.dictionary = dictionary;
}

@Override public Block buildBlock() {
return builder.build();
}

@Override public void setBatchSize(int batchSize) {
this.batchSize = batchSize;
}

@Override public void putInt(int rowId, int value) {
Object data = DataTypeUtil
.getDataBasedOnDataType(dictionary.getDictionaryValueForKey(value), DataTypes.BYTE);
if (data != null) {
type.writeLong(builder, (byte) data);
} else {
builder.appendNull();
}
}

@Override public void putByte(int rowId, byte value) {
type.writeLong(builder, value);
}

@Override public void putBytes(int rowId, int count, byte[] src, int srcIndex) {
for (int i = srcIndex; i < count; i++) {
type.writeLong(builder, src[i]);
}
}

@Override public void putNull(int rowId) {
builder.appendNull();
}

@Override public void putNulls(int rowId, int count) {
for (int i = 0; i < count; i++) {
builder.appendNull();
}
}

@Override public void reset() {
builder = type.createBlockBuilder(null, batchSize);
}

@Override public void putObject(int rowId, Object value) {
if (value == null) {
putNull(rowId);
} else {
if (dictionary == null) {
putByte(rowId, (byte) value);
} else {
putInt(rowId, (int) value);
}
}
}
}
@@ -0,0 +1,105 @@
/*
* 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.presto.readers;

import org.apache.carbondata.core.cache.dictionary.Dictionary;
import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl;
import org.apache.carbondata.core.util.DataTypeUtil;

import com.facebook.presto.spi.block.Block;
import com.facebook.presto.spi.block.BlockBuilder;
import com.facebook.presto.spi.type.AbstractType;
import com.facebook.presto.spi.type.RealType;

/**
* Class for Reading the Float(real) value and setting it in Block
*/
public class FloatStreamReader extends CarbonColumnVectorImpl implements PrestoVectorBlockBuilder {

protected int batchSize;

protected AbstractType type = RealType.REAL;

protected BlockBuilder builder;

private Dictionary dictionary;


public FloatStreamReader(int batchSize, DataType dataType, Dictionary dictionary) {
super(batchSize, dataType);
this.batchSize = batchSize;
this.builder = type.createBlockBuilder(null, batchSize);
this.dictionary = dictionary;
}

@Override public Block buildBlock() {
return builder.build();
}

@Override public void setBatchSize(int batchSize) {
this.batchSize = batchSize;
}

@Override public void putInt(int rowId, int value) {
Object data = DataTypeUtil
.getDataBasedOnDataType(dictionary.getDictionaryValueForKey(value), DataTypes.FLOAT);
if (data != null) {
type.writeLong(builder, (long)Float.floatToRawIntBits((float)data));
} else {
builder.appendNull();
}
}

@Override public void putFloat(int rowId, float value) {
type.writeLong(builder, (long)Float.floatToRawIntBits(value));
}

@Override public void putFloats(int rowId, int count, float[] src, int srcIndex) {
for (int i = srcIndex; i < count; i++) {
type.writeLong(builder, (long)Float.floatToRawIntBits(src[i]));
}
}

@Override public void putNull(int rowId) {
builder.appendNull();
}

@Override public void putNulls(int rowId, int count) {
for (int i = 0; i < count; i++) {
builder.appendNull();
}
}

@Override public void reset() {
builder = type.createBlockBuilder(null, batchSize);
}

@Override public void putObject(int rowId, Object value) {
if (value == null) {
putNull(rowId);
} else {
if (dictionary == null) {
putFloat(rowId, (float) value);
} else {
putInt(rowId, (int) value);
}
}
}
}
Expand Up @@ -82,7 +82,8 @@ class PrestoTestNonTransactionalTableFiles extends FunSuiteLike with BeforeAndAf
prestoServer.execute("create schema sdk_output")
prestoServer
.execute(
"create table sdk_output.files(name varchar, age int, height double) with" +
"create table sdk_output.files(name varchar, age int, id tinyint, height double, salary " +
"real) with" +
"(format='CARBON') ")
}

Expand All @@ -98,7 +99,9 @@ class PrestoTestNonTransactionalTableFiles extends FunSuiteLike with BeforeAndAf
.append("[ \n")
.append(" {\"NaMe\":\"string\"},\n")
.append(" {\"age\":\"int\"},\n")
.append(" {\"height\":\"double\"}\n")
.append(" {\"id\":\"byte\"},\n")
.append(" {\"height\":\"double\"},\n")
.append(" {\"salary\":\"float\"}\n")
.append("]")
.toString()

Expand All @@ -124,16 +127,30 @@ class PrestoTestNonTransactionalTableFiles extends FunSuiteLike with BeforeAndAf
while (i < rows) {
if ((options != null) && (i < 3)) {
// writing a bad record
writer.write(Array[String]("robot" + i, String.valueOf(i.toDouble / 2), "robot"))
writer
.write(Array[String]("robot" + i,
String.valueOf(i),
String.valueOf(i.toDouble / 2),
"robot",
String.valueOf(i.toFloat / 2)))
} else {
writer
.write(Array[String]("robot" + i, String.valueOf(i), String.valueOf(i.toDouble / 2)))
.write(Array[String]("robot" + i,
String.valueOf(i),
String.valueOf(i % 128),
String.valueOf(i.toDouble / 2),
String.valueOf(i.toFloat / 2)))
}
i += 1
}
if (options != null) {
//Keep one valid record. else carbon data file will not generate
writer.write(Array[String]("robot" + i, String.valueOf(i), String.valueOf(i.toDouble / 2)))
writer
.write(Array[String]("robot" + i,
String.valueOf(i),
String.valueOf(i),
String.valueOf(i.toDouble / 2),
String.valueOf(i.toFloat / 2)))
}
writer.close()
} catch {
Expand All @@ -143,11 +160,13 @@ class PrestoTestNonTransactionalTableFiles extends FunSuiteLike with BeforeAndAf

// prepare sdk writer output with other schema
def buildTestDataOtherDataType(rows: Int, sortColumns: Array[String]): Any = {
val fields: Array[Field] = new Array[Field](3)
val fields: Array[Field] = new Array[Field](5)
// same column name, but name as boolean type
fields(0) = new Field("name", DataTypes.BOOLEAN)
fields(1) = new Field("age", DataTypes.INT)
fields(2) = new Field("height", DataTypes.DOUBLE)
fields(2) = new Field("id", DataTypes.BYTE)
fields(3) = new Field("height", DataTypes.DOUBLE)
fields(4) = new Field("salary", DataTypes.FLOAT)

try {
val builder = CarbonWriter.builder()
Expand All @@ -157,7 +176,12 @@ class PrestoTestNonTransactionalTableFiles extends FunSuiteLike with BeforeAndAf
.withCsvInput(new Schema(fields)).writtenBy("TestNonTransactionalCarbonTable").build()
var i = 0
while (i < rows) {
writer.write(Array[String]("true", String.valueOf(i), String.valueOf(i.toDouble / 2)))
writer
.write(Array[String]("true",
String.valueOf(i),
String.valueOf(i),
String.valueOf(i.toDouble / 2),
String.valueOf(i.toFloat / 2)))
i += 1
}
writer.close()
Expand Down Expand Up @@ -247,4 +271,27 @@ class PrestoTestNonTransactionalTableFiles extends FunSuiteLike with BeforeAndAf
.contains("No Index files are present in the table location"))
cleanTestData()
}

test("test select all columns") {
buildTestDataSingleFile()
val actualResult: List[Map[String, Any]] = prestoServer
.executeQuery("select * from files ")
val expectedResult: List[Map[String, Any]] = List(Map(
"name" -> "robot0",
"height" -> 0.0,
"age" -> 0,
"salary" -> 0.0,
"id" -> 0),
Map("name" -> "robot1",
"height" -> 0.5,
"age" -> 1,
"salary" -> 0.5,
"id" -> 1),
Map("name" -> "robot2",
"height" -> 1.0,
"age" -> 2,
"salary" -> 1.0,
"id" -> 2))
assert(actualResult.toString() equals expectedResult.toString())
}
}

0 comments on commit 556e220

Please sign in to comment.