Skip to content

Commit

Permalink
Remove ArrayType.toStackRepresentation
Browse files Browse the repository at this point in the history
  • Loading branch information
cberner committed Aug 19, 2015
1 parent 3e1887e commit 545cd65
Show file tree
Hide file tree
Showing 12 changed files with 83 additions and 54 deletions.
Expand Up @@ -20,6 +20,8 @@
import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.Block;
import com.facebook.presto.spi.block.BlockBuilder;
import com.facebook.presto.spi.block.BlockBuilderStatus;
import com.facebook.presto.spi.type.StandardTypes; import com.facebook.presto.spi.type.StandardTypes;
import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.TypeManager; import com.facebook.presto.spi.type.TypeManager;
Expand All @@ -34,9 +36,9 @@
import static com.facebook.presto.metadata.FunctionRegistry.operatorInfo; import static com.facebook.presto.metadata.FunctionRegistry.operatorInfo;
import static com.facebook.presto.metadata.Signature.typeParameter; import static com.facebook.presto.metadata.Signature.typeParameter;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_CAST_ARGUMENT; import static com.facebook.presto.spi.StandardErrorCode.INVALID_CAST_ARGUMENT;
import static com.facebook.presto.type.ArrayType.toStackRepresentation;
import static com.facebook.presto.type.TypeJsonUtils.canCastFromJson; import static com.facebook.presto.type.TypeJsonUtils.canCastFromJson;
import static com.facebook.presto.type.TypeJsonUtils.stackRepresentationToObject; import static com.facebook.presto.type.TypeJsonUtils.stackRepresentationToObject;
import static com.facebook.presto.type.TypeUtils.appendToBlockBuilder;
import static com.facebook.presto.type.TypeUtils.parameterizedTypeName; import static com.facebook.presto.type.TypeUtils.parameterizedTypeName;
import static com.facebook.presto.util.Failures.checkCondition; import static com.facebook.presto.util.Failures.checkCondition;
import static com.facebook.presto.util.Reflection.methodHandle; import static com.facebook.presto.util.Reflection.methodHandle;
Expand Down Expand Up @@ -67,11 +69,16 @@ public FunctionInfo specialize(Map<String, Type> types, int arity, TypeManager t
public static Block toArray(Type arrayType, ConnectorSession connectorSession, Slice json) public static Block toArray(Type arrayType, ConnectorSession connectorSession, Slice json)
{ {
try { try {
Object array = stackRepresentationToObject(connectorSession, json, arrayType); List<?> array = (List<?>) stackRepresentationToObject(connectorSession, json, arrayType);
if (array == null) { if (array == null) {
return null; return null;
} }
return toStackRepresentation((List<?>) array, ((ArrayType) arrayType).getElementType()); Type elementType = ((ArrayType) arrayType).getElementType();
BlockBuilder blockBuilder = elementType.createBlockBuilder(new BlockBuilderStatus(), array.size());
for (Object element : array) {
appendToBlockBuilder(elementType, element, blockBuilder);
}
return blockBuilder.build();
} }
catch (RuntimeException e) { catch (RuntimeException e) {
throw new PrestoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to " + arrayType, e); throw new PrestoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to " + arrayType, e);
Expand Down
13 changes: 0 additions & 13 deletions presto-main/src/main/java/com/facebook/presto/type/ArrayType.java
Expand Up @@ -29,7 +29,6 @@
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;


import static com.facebook.presto.type.TypeUtils.appendToBlockBuilder;
import static com.facebook.presto.type.TypeUtils.checkElementNotNull; import static com.facebook.presto.type.TypeUtils.checkElementNotNull;
import static com.facebook.presto.type.TypeUtils.parameterizedTypeName; import static com.facebook.presto.type.TypeUtils.parameterizedTypeName;
import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkNotNull;
Expand All @@ -51,18 +50,6 @@ public Type getElementType()
return elementType; return elementType;
} }


/**
* Takes a list of stack types and converts them to the stack representation of an array
*/
public static Block toStackRepresentation(List<?> values, Type elementType)
{
BlockBuilder blockBuilder = elementType.createBlockBuilder(new BlockBuilderStatus(), values.size());
for (Object element : values) {
appendToBlockBuilder(elementType, element, blockBuilder);
}
return blockBuilder.build();
}

@Override @Override
public boolean isComparable() public boolean isComparable()
{ {
Expand Down
Expand Up @@ -37,6 +37,7 @@
import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.testing.MaterializedResult.resultBuilder; import static com.facebook.presto.testing.MaterializedResult.resultBuilder;
import static com.facebook.presto.testing.TestingTaskContext.createTaskContext; import static com.facebook.presto.testing.TestingTaskContext.createTaskContext;
import static com.facebook.presto.util.StructuralTestUtil.arrayBlockOf;
import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static java.lang.Double.NEGATIVE_INFINITY; import static java.lang.Double.NEGATIVE_INFINITY;
import static java.lang.Double.NaN; import static java.lang.Double.NaN;
Expand Down Expand Up @@ -74,11 +75,11 @@ public void testUnnest()
Type mapType = metadata.getType(parseTypeSignature("map<bigint,bigint>")); Type mapType = metadata.getType(parseTypeSignature("map<bigint,bigint>"));


List<Page> input = rowPagesBuilder(BIGINT, arrayType, mapType) List<Page> input = rowPagesBuilder(BIGINT, arrayType, mapType)
.row(1, ArrayType.toStackRepresentation(ImmutableList.of(2, 3), BIGINT), MapType.toStackRepresentation(ImmutableMap.of(4, 5), BIGINT, BIGINT)) .row(1, arrayBlockOf(BIGINT, 2, 3), MapType.toStackRepresentation(ImmutableMap.of(4, 5), BIGINT, BIGINT))
.row(2, ArrayType.toStackRepresentation(ImmutableList.of(99), BIGINT), null) .row(2, arrayBlockOf(BIGINT, 99), null)
.row(3, null, null) .row(3, null, null)
.pageBreak() .pageBreak()
.row(6, ArrayType.toStackRepresentation(ImmutableList.of(7, 8), BIGINT), MapType.toStackRepresentation(ImmutableMap.of(9, 10, 11, 12), BIGINT, BIGINT)) .row(6, arrayBlockOf(BIGINT, 7, 8), MapType.toStackRepresentation(ImmutableMap.of(9, 10, 11, 12), BIGINT, BIGINT))
.build(); .build();


OperatorFactory operatorFactory = new UnnestOperator.UnnestOperatorFactory( OperatorFactory operatorFactory = new UnnestOperator.UnnestOperatorFactory(
Expand Down Expand Up @@ -107,14 +108,14 @@ public void testUnnestWithArray()
List<Page> input = rowPagesBuilder(BIGINT, arrayType, mapType) List<Page> input = rowPagesBuilder(BIGINT, arrayType, mapType)
.row( .row(
1, 1,
ArrayType.toStackRepresentation(ImmutableList.of(ImmutableList.of(2, 4), ImmutableList.of(3, 6)), new ArrayType(BIGINT)), arrayBlockOf(new ArrayType(BIGINT), ImmutableList.of(2, 4), ImmutableList.of(3, 6)),
MapType.toStackRepresentation(ImmutableMap.of(ImmutableList.of(4, 8), ImmutableList.of(5, 10)), new ArrayType(BIGINT), new ArrayType(BIGINT))) MapType.toStackRepresentation(ImmutableMap.of(ImmutableList.of(4, 8), ImmutableList.of(5, 10)), new ArrayType(BIGINT), new ArrayType(BIGINT)))
.row(2, ArrayType.toStackRepresentation(ImmutableList.of(ImmutableList.of(99, 198)), new ArrayType(BIGINT)), null) .row(2, arrayBlockOf(new ArrayType(BIGINT), ImmutableList.of(99, 198)), null)
.row(3, null, null) .row(3, null, null)
.pageBreak() .pageBreak()
.row( .row(
6, 6,
ArrayType.toStackRepresentation(ImmutableList.of(ImmutableList.of(7, 14), ImmutableList.of(8, 16)), new ArrayType(BIGINT)), arrayBlockOf(new ArrayType(BIGINT), ImmutableList.of(7, 14), ImmutableList.of(8, 16)),
MapType.toStackRepresentation(ImmutableMap.of(ImmutableList.of(9, 18), ImmutableList.of(10, 20), ImmutableList.of(11, 22), ImmutableList.of(12, 24)), new ArrayType(BIGINT), new ArrayType(BIGINT))) MapType.toStackRepresentation(ImmutableMap.of(ImmutableList.of(9, 18), ImmutableList.of(10, 20), ImmutableList.of(11, 22), ImmutableList.of(12, 24)), new ArrayType(BIGINT), new ArrayType(BIGINT)))
.build(); .build();


Expand Down Expand Up @@ -142,11 +143,11 @@ public void testUnnestWithOrdinality()
Type mapType = metadata.getType(parseTypeSignature("map<bigint,bigint>")); Type mapType = metadata.getType(parseTypeSignature("map<bigint,bigint>"));


List<Page> input = rowPagesBuilder(BIGINT, arrayType, mapType) List<Page> input = rowPagesBuilder(BIGINT, arrayType, mapType)
.row(1, ArrayType.toStackRepresentation(ImmutableList.of(2, 3), BIGINT), MapType.toStackRepresentation(ImmutableMap.of(4, 5), BIGINT, BIGINT)) .row(1, arrayBlockOf(BIGINT, 2, 3), MapType.toStackRepresentation(ImmutableMap.of(4, 5), BIGINT, BIGINT))
.row(2, ArrayType.toStackRepresentation(ImmutableList.of(99), BIGINT), null) .row(2, arrayBlockOf(BIGINT, 99), null)
.row(3, null, null) .row(3, null, null)
.pageBreak() .pageBreak()
.row(6, ArrayType.toStackRepresentation(ImmutableList.of(7, 8), BIGINT), MapType.toStackRepresentation(ImmutableMap.of(9, 10, 11, 12), BIGINT, BIGINT)) .row(6, arrayBlockOf(BIGINT, 7, 8), MapType.toStackRepresentation(ImmutableMap.of(9, 10, 11, 12), BIGINT, BIGINT))
.build(); .build();


OperatorFactory operatorFactory = new UnnestOperator.UnnestOperatorFactory( OperatorFactory operatorFactory = new UnnestOperator.UnnestOperatorFactory(
Expand All @@ -173,7 +174,7 @@ public void testUnnestNonNumericDoubles()
Type mapType = metadata.getType(parseTypeSignature("map<bigint,double>")); Type mapType = metadata.getType(parseTypeSignature("map<bigint,double>"));


List<Page> input = rowPagesBuilder(BIGINT, arrayType, mapType) List<Page> input = rowPagesBuilder(BIGINT, arrayType, mapType)
.row(1, ArrayType.toStackRepresentation(ImmutableList.of(NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, NaN), DOUBLE), .row(1, arrayBlockOf(DOUBLE, NEGATIVE_INFINITY, POSITIVE_INFINITY, NaN),
MapType.toStackRepresentation(ImmutableMap.of(1, NEGATIVE_INFINITY, 2, POSITIVE_INFINITY, 3, NaN), BIGINT, DOUBLE)) MapType.toStackRepresentation(ImmutableMap.of(1, NEGATIVE_INFINITY, 2, POSITIVE_INFINITY, 3, NaN), BIGINT, DOUBLE))
.build(); .build();


Expand Down
Expand Up @@ -22,7 +22,7 @@
import java.util.List; import java.util.List;


import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.type.ArrayType.toStackRepresentation; import static com.facebook.presto.util.StructuralTestUtil.arrayBlockOf;


public class TestArrayMinAggregation public class TestArrayMinAggregation
extends AbstractTestAggregationFunction extends AbstractTestAggregationFunction
Expand All @@ -33,7 +33,7 @@ public Block[] getSequenceBlocks(int start, int length)
ArrayType arrayType = new ArrayType(BIGINT); ArrayType arrayType = new ArrayType(BIGINT);
BlockBuilder blockBuilder = arrayType.createBlockBuilder(new BlockBuilderStatus(), length); BlockBuilder blockBuilder = arrayType.createBlockBuilder(new BlockBuilderStatus(), length);
for (int i = start; i < start + length; i++) { for (int i = start; i < start + length; i++) {
arrayType.writeObject(blockBuilder, toStackRepresentation(ImmutableList.of(i), BIGINT)); arrayType.writeObject(blockBuilder, arrayBlockOf(BIGINT, i));
} }
return new Block[] {blockBuilder.build()}; return new Block[] {blockBuilder.build()};
} }
Expand Down
Expand Up @@ -45,8 +45,8 @@
import static com.facebook.presto.spi.type.TimeZoneKey.getTimeZoneKey; import static com.facebook.presto.spi.type.TimeZoneKey.getTimeZoneKey;
import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR; import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.facebook.presto.type.ArrayType.toStackRepresentation;
import static com.facebook.presto.util.DateTimeZoneIndex.getDateTimeZone; import static com.facebook.presto.util.DateTimeZoneIndex.getDateTimeZone;
import static com.facebook.presto.util.StructuralTestUtil.arrayBlockOf;


public class TestHistogram public class TestHistogram
{ {
Expand Down Expand Up @@ -125,13 +125,13 @@ public void testArrayHistograms()
PageBuilder builder = new PageBuilder(ImmutableList.of(arrayType)); PageBuilder builder = new PageBuilder(ImmutableList.of(arrayType));


builder.declarePosition(); builder.declarePosition();
arrayType.writeObject(builder.getBlockBuilder(0), toStackRepresentation(ImmutableList.of("a", "b", "c"), VARCHAR)); arrayType.writeObject(builder.getBlockBuilder(0), arrayBlockOf(VARCHAR, "a", "b", "c"));


builder.declarePosition(); builder.declarePosition();
arrayType.writeObject(builder.getBlockBuilder(0), toStackRepresentation(ImmutableList.of("d", "e", "f"), VARCHAR)); arrayType.writeObject(builder.getBlockBuilder(0), arrayBlockOf(VARCHAR, "d", "e", "f"));


builder.declarePosition(); builder.declarePosition();
arrayType.writeObject(builder.getBlockBuilder(0), toStackRepresentation(ImmutableList.of("c", "b", "a"), VARCHAR)); arrayType.writeObject(builder.getBlockBuilder(0), arrayBlockOf(VARCHAR, "c", "b", "a"));


assertAggregation( assertAggregation(
aggregationFunction, aggregationFunction,
Expand Down
Expand Up @@ -40,7 +40,7 @@
import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; import static com.facebook.presto.spi.type.BooleanType.BOOLEAN;
import static com.facebook.presto.spi.type.DoubleType.DOUBLE; import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR; import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.facebook.presto.type.ArrayType.toStackRepresentation; import static com.facebook.presto.util.StructuralTestUtil.arrayBlockOf;


public class TestMapAggAggregation public class TestMapAggAggregation
{ {
Expand Down Expand Up @@ -139,15 +139,15 @@ public void testDoubleArrayMap()


builder.declarePosition(); builder.declarePosition();
DOUBLE.writeDouble(builder.getBlockBuilder(0), 1.0); DOUBLE.writeDouble(builder.getBlockBuilder(0), 1.0);
arrayType.writeObject(builder.getBlockBuilder(1), toStackRepresentation(ImmutableList.of("a", "b"), VARCHAR)); arrayType.writeObject(builder.getBlockBuilder(1), arrayBlockOf(VARCHAR, "a", "b"));


builder.declarePosition(); builder.declarePosition();
DOUBLE.writeDouble(builder.getBlockBuilder(0), 2.0); DOUBLE.writeDouble(builder.getBlockBuilder(0), 2.0);
arrayType.writeObject(builder.getBlockBuilder(1), toStackRepresentation(ImmutableList.of("c", "d"), VARCHAR)); arrayType.writeObject(builder.getBlockBuilder(1), arrayBlockOf(VARCHAR, "c", "d"));


builder.declarePosition(); builder.declarePosition();
DOUBLE.writeDouble(builder.getBlockBuilder(0), 3.0); DOUBLE.writeDouble(builder.getBlockBuilder(0), 3.0);
arrayType.writeObject(builder.getBlockBuilder(1), toStackRepresentation(ImmutableList.of("e", "f"), VARCHAR)); arrayType.writeObject(builder.getBlockBuilder(1), arrayBlockOf(VARCHAR, "e", "f"));


assertAggregation( assertAggregation(
aggFunc, aggFunc,
Expand Down Expand Up @@ -242,15 +242,15 @@ public void testArrayDoubleMap()
PageBuilder builder = new PageBuilder(ImmutableList.of(arrayType, DOUBLE)); PageBuilder builder = new PageBuilder(ImmutableList.of(arrayType, DOUBLE));


builder.declarePosition(); builder.declarePosition();
arrayType.writeObject(builder.getBlockBuilder(0), toStackRepresentation(ImmutableList.of("a", "b"), VARCHAR)); arrayType.writeObject(builder.getBlockBuilder(0), arrayBlockOf(VARCHAR, "a", "b"));
DOUBLE.writeDouble(builder.getBlockBuilder(1), 1.0); DOUBLE.writeDouble(builder.getBlockBuilder(1), 1.0);


builder.declarePosition(); builder.declarePosition();
arrayType.writeObject(builder.getBlockBuilder(0), toStackRepresentation(ImmutableList.of("c", "d"), VARCHAR)); arrayType.writeObject(builder.getBlockBuilder(0), arrayBlockOf(VARCHAR, "c", "d"));
DOUBLE.writeDouble(builder.getBlockBuilder(1), 2.0); DOUBLE.writeDouble(builder.getBlockBuilder(1), 2.0);


builder.declarePosition(); builder.declarePosition();
arrayType.writeObject(builder.getBlockBuilder(0), toStackRepresentation(ImmutableList.of("e", "f"), VARCHAR)); arrayType.writeObject(builder.getBlockBuilder(0), arrayBlockOf(VARCHAR, "e", "f"));
DOUBLE.writeDouble(builder.getBlockBuilder(1), 3.0); DOUBLE.writeDouble(builder.getBlockBuilder(1), 3.0);


assertAggregation( assertAggregation(
Expand Down
Expand Up @@ -28,6 +28,7 @@
import static com.facebook.presto.spi.type.DateTimeEncoding.packDateTimeWithZone; import static com.facebook.presto.spi.type.DateTimeEncoding.packDateTimeWithZone;
import static com.facebook.presto.spi.type.TimeZoneKey.getTimeZoneKeyForOffset; import static com.facebook.presto.spi.type.TimeZoneKey.getTimeZoneKeyForOffset;
import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
import static com.facebook.presto.util.StructuralTestUtil.arrayBlockOf;
import static org.testng.Assert.assertTrue; import static org.testng.Assert.assertTrue;


public class TestFieldSetFilteringRecordSet public class TestFieldSetFilteringRecordSet
Expand All @@ -51,8 +52,8 @@ public void test()
packDateTimeWithZone(100, getTimeZoneKeyForOffset(123)), packDateTimeWithZone(100, getTimeZoneKeyForOffset(123)),
packDateTimeWithZone(100, getTimeZoneKeyForOffset(234)), packDateTimeWithZone(100, getTimeZoneKeyForOffset(234)),
// test structural type // test structural type
ArrayType.toStackRepresentation(ImmutableList.of(12, 34, 56), BIGINT), arrayBlockOf(BIGINT, 12, 34, 56),
ArrayType.toStackRepresentation(ImmutableList.of(12, 34, 56), BIGINT) arrayBlockOf(BIGINT, 12, 34, 56)
))), ))),
ImmutableList.of(ImmutableSet.of(0, 1), ImmutableSet.of(2, 3), ImmutableSet.of(4, 5))); ImmutableList.of(ImmutableSet.of(0, 1), ImmutableSet.of(2, 3), ImmutableSet.of(4, 5)));
RecordCursor recordCursor = fieldSetFilteringRecordSet.cursor(); RecordCursor recordCursor = fieldSetFilteringRecordSet.cursor();
Expand Down
Expand Up @@ -18,7 +18,6 @@
import com.facebook.presto.spi.block.BlockBuilder; import com.facebook.presto.spi.block.BlockBuilder;
import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.block.BlockBuilderStatus;
import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.Type;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.airlift.slice.DynamicSliceOutput; import io.airlift.slice.DynamicSliceOutput;
import io.airlift.slice.Slice; import io.airlift.slice.Slice;
Expand All @@ -40,6 +39,7 @@
import static com.facebook.presto.testing.TestingConnectorSession.SESSION; import static com.facebook.presto.testing.TestingConnectorSession.SESSION;
import static com.facebook.presto.type.TypeUtils.hashPosition; import static com.facebook.presto.type.TypeUtils.hashPosition;
import static com.facebook.presto.type.TypeUtils.positionEqualsPosition; import static com.facebook.presto.type.TypeUtils.positionEqualsPosition;
import static com.facebook.presto.util.StructuralTestUtil.arrayBlockOf;
import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkNotNull;
import static io.airlift.testing.Assertions.assertInstanceOf; import static io.airlift.testing.Assertions.assertInstanceOf;
import static java.util.Collections.unmodifiableSortedMap; import static java.util.Collections.unmodifiableSortedMap;
Expand Down Expand Up @@ -488,7 +488,7 @@ private static Object getNonNullValueForType(Type type)
ArrayType arrayType = (ArrayType) type; ArrayType arrayType = (ArrayType) type;
Type elementType = arrayType.getElementType(); Type elementType = arrayType.getElementType();
Object elementNonNullValue = getNonNullValueForType(elementType); Object elementNonNullValue = getNonNullValueForType(elementType);
return ArrayType.toStackRepresentation(ImmutableList.of(elementNonNullValue), elementType); return arrayBlockOf(elementType, elementNonNullValue);
} }
if (type instanceof MapType) { if (type instanceof MapType) {
MapType mapType = (MapType) type; MapType mapType = (MapType) type;
Expand Down
Expand Up @@ -39,9 +39,9 @@
import static com.facebook.presto.spi.type.DoubleType.DOUBLE; import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR; import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static com.facebook.presto.type.ArrayType.toStackRepresentation;
import static com.facebook.presto.type.JsonType.JSON; import static com.facebook.presto.type.JsonType.JSON;
import static com.facebook.presto.type.UnknownType.UNKNOWN; import static com.facebook.presto.type.UnknownType.UNKNOWN;
import static com.facebook.presto.util.StructuralTestUtil.arrayBlockOf;
import static java.lang.Double.NEGATIVE_INFINITY; import static java.lang.Double.NEGATIVE_INFINITY;
import static java.lang.Double.NaN; import static java.lang.Double.NaN;
import static java.lang.Double.POSITIVE_INFINITY; import static java.lang.Double.POSITIVE_INFINITY;
Expand All @@ -63,9 +63,7 @@ public TestArrayOperators()
public void testStackRepresentation() public void testStackRepresentation()
throws Exception throws Exception
{ {
Block actualBlock = toStackRepresentation(ImmutableList.of( Block actualBlock = arrayBlockOf(new ArrayType(BIGINT), arrayBlockOf(BIGINT, 1L, 2L), arrayBlockOf(BIGINT, 3L));
toStackRepresentation(ImmutableList.of(1L, 2L), BIGINT),
toStackRepresentation(ImmutableList.of(3L), BIGINT)), new ArrayType(BIGINT));
DynamicSliceOutput actualSliceOutput = new DynamicSliceOutput(100); DynamicSliceOutput actualSliceOutput = new DynamicSliceOutput(100);
writeBlock(actualSliceOutput, actualBlock); writeBlock(actualSliceOutput, actualBlock);


Expand Down
Expand Up @@ -17,13 +17,12 @@
import com.facebook.presto.spi.block.BlockBuilder; import com.facebook.presto.spi.block.BlockBuilder;
import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.block.BlockBuilderStatus;
import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.Type;
import com.google.common.collect.ImmutableList;


import java.util.List; import java.util.List;


import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.type.ArrayType.toStackRepresentation; import static com.facebook.presto.util.StructuralTestUtil.arrayBlockOf;


public class TestBigintArrayType public class TestBigintArrayType
extends AbstractTestType extends AbstractTestType
Expand All @@ -36,10 +35,10 @@ public TestBigintArrayType()
public static Block createTestBlock(Type arrayType) public static Block createTestBlock(Type arrayType)
{ {
BlockBuilder blockBuilder = arrayType.createBlockBuilder(new BlockBuilderStatus(), 4); BlockBuilder blockBuilder = arrayType.createBlockBuilder(new BlockBuilderStatus(), 4);
arrayType.writeObject(blockBuilder, toStackRepresentation(ImmutableList.of(1, 2), BIGINT)); arrayType.writeObject(blockBuilder, arrayBlockOf(BIGINT, 1, 2));
arrayType.writeObject(blockBuilder, toStackRepresentation(ImmutableList.of(1, 2, 3), BIGINT)); arrayType.writeObject(blockBuilder, arrayBlockOf(BIGINT, 1, 2, 3));
arrayType.writeObject(blockBuilder, toStackRepresentation(ImmutableList.of(1, 2, 3), BIGINT)); arrayType.writeObject(blockBuilder, arrayBlockOf(BIGINT, 1, 2, 3));
arrayType.writeObject(blockBuilder, toStackRepresentation(ImmutableList.of(100, 200, 300), BIGINT)); arrayType.writeObject(blockBuilder, arrayBlockOf(BIGINT, 100, 200, 300));
return blockBuilder.build(); return blockBuilder.build();
} }


Expand Down
Expand Up @@ -43,6 +43,7 @@
import static com.facebook.presto.type.JsonType.JSON; import static com.facebook.presto.type.JsonType.JSON;
import static com.facebook.presto.type.MapType.toStackRepresentation; import static com.facebook.presto.type.MapType.toStackRepresentation;
import static com.facebook.presto.type.UnknownType.UNKNOWN; import static com.facebook.presto.type.UnknownType.UNKNOWN;
import static com.facebook.presto.util.StructuralTestUtil.arrayBlockOf;
import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertEquals;


public class TestMapOperators public class TestMapOperators
Expand All @@ -64,7 +65,7 @@ public static Slice uncheckedToJson(@SqlType(StandardTypes.VARCHAR) Slice slice)
public void testStackRepresentation() public void testStackRepresentation()
throws Exception throws Exception
{ {
Block array = ArrayType.toStackRepresentation(ImmutableList.of(1L, 2L), BIGINT); Block array = arrayBlockOf(BIGINT, 1L, 2L);
Block actualBlock = toStackRepresentation(ImmutableMap.of(1.0, array), DOUBLE, new ArrayType(BIGINT)); Block actualBlock = toStackRepresentation(ImmutableMap.of(1.0, array), DOUBLE, new ArrayType(BIGINT));
DynamicSliceOutput actualSliceOutput = new DynamicSliceOutput(100); DynamicSliceOutput actualSliceOutput = new DynamicSliceOutput(100);
writeBlock(actualSliceOutput, actualBlock); writeBlock(actualSliceOutput, actualBlock);
Expand Down

0 comments on commit 545cd65

Please sign in to comment.