Skip to content

Commit

Permalink
Add support for StdFloat, StdDouble, and StdBinary (#46)
Browse files Browse the repository at this point in the history
* Introduce StdFloat, StdDouble, and StdBinary interfaces
* Add implementations of those interfaces in Avro, Hive, Presto, Spark, and Generic type systems
* Add examples of transport UDFs on those new types, and add tests for those UDFs
* Update documentation
  • Loading branch information
khaitranq authored Jun 23, 2020
1 parent 5c1b84a commit b08e633
Show file tree
Hide file tree
Showing 84 changed files with 1,656 additions and 30 deletions.
19 changes: 15 additions & 4 deletions docs/transport-udfs-api.md
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,8 @@ The `StdType` interface is the parent class of all type objects that
are used to describe the schema of the data objects that can be
manipulated by `StdUDFs`. Sub-interfaces of this interface include
`StdIntegerType`, `StdBooleanType`, `StdLongType`, `StdStringType`,
`StdArrayType`, `StdMapType`, `StdStructType`. Each sub-interface is
`StdDoubleType`, `StdFloatType`, `StdBinaryType`, `StdArrayType`,
`StdMapType`, and `StdStructType`. Each sub-interface is
defined by methods that are specific to the corresponding type. For
example, `StdMapType` interface is defined by the two methods shown
below. The `keyType()` and `valueType()` methods can be used to obtain
Expand Down Expand Up @@ -39,9 +40,10 @@ public interface StdStructType extends StdType {
manipulated by Transport UDFs. As a top-level interface, `StdData`
itself does not contain any methods. A number of type-specific
interfaces extend `StdData`, such as `StdInteger`, `StdLong`,
`StdBoolean`, `StdString`, `StdArray`, `StdMap`, `StdStruct` to
represent `INTEGER`, `LONG`, `BOOLEAN`, `VARCHAR`, `ARRAY`, `MAP`,
`STRUCT` SQL types respectively. Each of those interfaces exposes
`StdBoolean`, `StdString`, `StdDouble`, `StdFloat`, `StdBinary`,
`StdArray`, `StdMap`, and `StdStruct` to represent `INTEGER`,
`LONG`, `BOOLEAN`, `VARCHAR`, `DOUBLE`, `REAL`, `VARBINARY`, `ARRAY`, `MAP`,
and `STRUCT` SQL types respectively. Each of those interfaces exposes
operations that can manipulate that type of data. For example,
`StdMap` interface is defined by the following methods:

Expand Down Expand Up @@ -108,6 +110,12 @@ definition:
is StdInteger.
* `"boolean"`: to represent SQL Boolean type. The respective Standard
Type is StdBoolean.
* `"double"`: to represent SQL Double type. The respective Standard
Type is StdDouble.
* `"real"`: to represent SQL Real type. The respective Standard
Type is StdFloat.
* `"varbinary"`: to represent SQL Binary type. The respective Standard
Type is StdBinary.
* `"array(T)"`: to represent SQL Array type, with elements of type
T. The respective Standard Type is StdArray.
* `"map(K,V)"`: to represent SQL Map type, with keys of type K and
Expand All @@ -132,6 +140,9 @@ public interface StdFactory {
StdLong createLong(long value);
StdBoolean createBoolean(boolean value);
StdString createString(String value);
StdDouble createDouble(double value);
StdFloat createFloat(float value);
StdBinary createBinary(ByteBuffer value);
StdArray createArray(StdType stdType, int expectedSize);
StdArray createArray(StdType stdType);
StdMap createMap(StdType stdType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,10 @@

import com.linkedin.transport.api.data.StdArray;
import com.linkedin.transport.api.data.StdBoolean;
import com.linkedin.transport.api.data.StdBinary;
import com.linkedin.transport.api.data.StdData;
import com.linkedin.transport.api.data.StdDouble;
import com.linkedin.transport.api.data.StdFloat;
import com.linkedin.transport.api.data.StdInteger;
import com.linkedin.transport.api.data.StdLong;
import com.linkedin.transport.api.data.StdMap;
Expand All @@ -19,6 +22,7 @@
import com.linkedin.transport.api.types.StdType;
import com.linkedin.transport.api.udf.StdUDF;
import java.io.Serializable;
import java.nio.ByteBuffer;
import java.util.List;


Expand Down Expand Up @@ -63,6 +67,30 @@ public interface StdFactory extends Serializable {
*/
StdString createString(String value);

/**
* Creates a {@link StdFloat} representing a given float value.
*
* @param value the input float value
* @return {@link StdFloat} with the given float value
*/
StdFloat createFloat(float value);

/**
* Creates a {@link StdDouble} representing a given double value.
*
* @param value the input double value
* @return {@link StdDouble} with the given double value
*/
StdDouble createDouble(double value);

/**
* Creates a {@link StdBinary} representing a given {@link ByteBuffer} value.
*
* @param value the input {@link ByteBuffer} value
* @return {@link StdBinary} with the given {@link ByteBuffer} value
*/
StdBinary createBinary(ByteBuffer value);

/**
* Creates an empty {@link StdArray} whose type is given by the given {@link StdType}.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
/**
* Copyright 2018 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
package com.linkedin.transport.api.data;

import java.nio.ByteBuffer;

/** A Standard UDF data type for representing binary objects. */
public interface StdBinary extends StdData {

/** Returns the underlying {@link ByteBuffer} value. */
ByteBuffer get();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
/**
* Copyright 2018 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
package com.linkedin.transport.api.data;

/** A Standard UDF data type for representing doubles. */
public interface StdDouble extends StdData {

/** Returns the underlying double value. */
double get();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
/**
* Copyright 2018 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
package com.linkedin.transport.api.data;

/** A Standard UDF data type for representing floats. */
public interface StdFloat extends StdData {

/** Returns the underlying float value. */
float get();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,10 @@
/**
* Copyright 2018 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
package com.linkedin.transport.api.types;

/** A {@link StdType} representing a {@link java.nio.ByteBuffer} type. */
public interface StdBinaryType extends StdType {
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,10 @@
/**
* Copyright 2018 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
package com.linkedin.transport.api.types;

/** A {@link StdType} representing a double type. */
public interface StdDoubleType extends StdType {
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,10 @@
/**
* Copyright 2018 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
package com.linkedin.transport.api.types;

/** A {@link StdType} representing a float type. */
public interface StdFloatType extends StdType {
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,9 @@
import com.linkedin.transport.api.StdFactory;
import com.linkedin.transport.api.data.StdArray;
import com.linkedin.transport.api.data.StdBoolean;
import com.linkedin.transport.api.data.StdBinary;
import com.linkedin.transport.api.data.StdDouble;
import com.linkedin.transport.api.data.StdFloat;
import com.linkedin.transport.api.data.StdInteger;
import com.linkedin.transport.api.data.StdLong;
import com.linkedin.transport.api.data.StdMap;
Expand All @@ -16,6 +19,9 @@
import com.linkedin.transport.api.types.StdType;
import com.linkedin.transport.avro.data.AvroArray;
import com.linkedin.transport.avro.data.AvroBoolean;
import com.linkedin.transport.avro.data.AvroBinary;
import com.linkedin.transport.avro.data.AvroDouble;
import com.linkedin.transport.avro.data.AvroFloat;
import com.linkedin.transport.avro.data.AvroInteger;
import com.linkedin.transport.avro.data.AvroLong;
import com.linkedin.transport.avro.data.AvroMap;
Expand All @@ -24,6 +30,7 @@
import com.linkedin.transport.avro.typesystem.AvroTypeFactory;
import com.linkedin.transport.typesystem.AbstractBoundVariables;
import com.linkedin.transport.typesystem.TypeSignature;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -64,6 +71,21 @@ public StdString createString(String value) {
return new AvroString(new Utf8(value));
}

@Override
public StdFloat createFloat(float value) {
return new AvroFloat(value);
}

@Override
public StdDouble createDouble(double value) {
return new AvroDouble(value);
}

@Override
public StdBinary createBinary(ByteBuffer value) {
return new AvroBinary(value);
}

@Override
public StdArray createArray(StdType stdType, int size) {
return new AvroArray((Schema) stdType.underlyingType(), size);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,18 +9,25 @@
import com.linkedin.transport.api.types.StdType;
import com.linkedin.transport.avro.data.AvroArray;
import com.linkedin.transport.avro.data.AvroBoolean;
import com.linkedin.transport.avro.data.AvroBinary;
import com.linkedin.transport.avro.data.AvroDouble;
import com.linkedin.transport.avro.data.AvroFloat;
import com.linkedin.transport.avro.data.AvroInteger;
import com.linkedin.transport.avro.data.AvroLong;
import com.linkedin.transport.avro.data.AvroMap;
import com.linkedin.transport.avro.data.AvroString;
import com.linkedin.transport.avro.data.AvroStruct;
import com.linkedin.transport.avro.types.AvroArrayType;
import com.linkedin.transport.avro.types.AvroBooleanType;
import com.linkedin.transport.avro.types.AvroBinaryType;
import com.linkedin.transport.avro.types.AvroDoubleType;
import com.linkedin.transport.avro.types.AvroFloatType;
import com.linkedin.transport.avro.types.AvroIntegerType;
import com.linkedin.transport.avro.types.AvroLongType;
import com.linkedin.transport.avro.types.AvroMapType;
import com.linkedin.transport.avro.types.AvroStringType;
import com.linkedin.transport.avro.types.AvroStructType;
import java.nio.ByteBuffer;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericArray;
Expand All @@ -43,6 +50,12 @@ public static StdData createStdData(Object avroData, Schema avroSchema) {
return new AvroBoolean((Boolean) avroData);
case STRING:
return new AvroString((Utf8) avroData);
case FLOAT:
return new AvroFloat((Float) avroData);
case DOUBLE:
return new AvroDouble((Double) avroData);
case BYTES:
return new AvroBinary((ByteBuffer) avroData);
case ARRAY:
return new AvroArray((GenericArray<Object>) avroData, avroSchema);
case MAP:
Expand All @@ -66,6 +79,12 @@ public static StdType createStdType(Schema avroSchema) {
return new AvroBooleanType(avroSchema);
case STRING:
return new AvroStringType(avroSchema);
case FLOAT:
return new AvroFloatType(avroSchema);
case DOUBLE:
return new AvroDoubleType(avroSchema);
case BYTES:
return new AvroBinaryType(avroSchema);
case ARRAY:
return new AvroArrayType(avroSchema);
case MAP:
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/**
* Copyright 2018 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
package com.linkedin.transport.avro.data;

import com.linkedin.transport.api.data.PlatformData;
import com.linkedin.transport.api.data.StdBinary;
import java.nio.ByteBuffer;


public class AvroBinary implements StdBinary, PlatformData {
private ByteBuffer _byteBuffer;

public AvroBinary(ByteBuffer aByteBuffer) {
_byteBuffer = aByteBuffer;
}

@Override
public Object getUnderlyingData() {
return _byteBuffer;
}

@Override
public void setUnderlyingData(Object value) {
_byteBuffer = (ByteBuffer) value;
}

@Override
public ByteBuffer get() {
return _byteBuffer;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/**
* Copyright 2018 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
package com.linkedin.transport.avro.data;

import com.linkedin.transport.api.data.PlatformData;
import com.linkedin.transport.api.data.StdDouble;


public class AvroDouble implements StdDouble, PlatformData {
private Double _double;

public AvroDouble(Double aDouble) {
_double = aDouble;
}

@Override
public Object getUnderlyingData() {
return _double;
}

@Override
public void setUnderlyingData(Object value) {
_double = (Double) value;
}

@Override
public double get() {
return _double;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/**
* Copyright 2018 LinkedIn Corporation. All rights reserved.
* Licensed under the BSD-2 Clause license.
* See LICENSE in the project root for license information.
*/
package com.linkedin.transport.avro.data;

import com.linkedin.transport.api.data.PlatformData;
import com.linkedin.transport.api.data.StdFloat;


public class AvroFloat implements StdFloat, PlatformData {
private Float _float;

public AvroFloat(Float aFloat) {
_float = aFloat;
}

@Override
public Object getUnderlyingData() {
return _float;
}

@Override
public void setUnderlyingData(Object value) {
_float = (Float) value;
}

@Override
public float get() {
return _float;
}
}
Loading

0 comments on commit b08e633

Please sign in to comment.