Skip to content

Commit

Permalink
Introduce DecimalType and SqlDecimal
Browse files Browse the repository at this point in the history
  • Loading branch information
Lukasz Osipiuk authored and losipiuk committed Nov 10, 2015
1 parent a94890a commit 4005239
Show file tree
Hide file tree
Showing 12 changed files with 835 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import static com.facebook.presto.spi.type.StandardTypes.BIGINT;
import static com.facebook.presto.spi.type.StandardTypes.BOOLEAN;
import static com.facebook.presto.spi.type.StandardTypes.DATE;
import static com.facebook.presto.spi.type.StandardTypes.DECIMAL;
import static com.facebook.presto.spi.type.StandardTypes.DOUBLE;
import static com.facebook.presto.spi.type.StandardTypes.INTERVAL_DAY_TO_SECOND;
import static com.facebook.presto.spi.type.StandardTypes.INTERVAL_YEAR_TO_MONTH;
Expand Down Expand Up @@ -272,6 +273,7 @@ private static Object fixValue(String type, Object value)
case DATE:
case INTERVAL_YEAR_TO_MONTH:
case INTERVAL_DAY_TO_SECOND:
case DECIMAL:
return String.class.cast(value);
default:
// for now we assume that only the explicit types above are passed
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* Licensed 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 com.facebook.presto.type;

import com.facebook.presto.spi.type.DecimalType;
import com.facebook.presto.spi.type.ShortDecimalType;
import com.facebook.presto.spi.type.StandardTypes;
import com.facebook.presto.spi.type.Type;
import org.jetbrains.annotations.NotNull;

import java.util.List;

import static com.google.common.base.Preconditions.checkArgument;

public class DecimalParametricType

This comment has been minimized.

Copy link
@martint

martint Nov 16, 2015

This should be just DecimalType

This comment has been minimized.

Copy link
@losipiuk

losipiuk Nov 17, 2015

This is implementation of ParametrizedType while DecimalType is implementation of Type.

I could not make DecimalType implement both ParametricType and Type as ParametricType is not part of SPI.

Actually I am not sure why it is not there. It seems it should be. Probably merged into Type interface itself. I think it makes sense to do this refactoring as part of type/literal parameter refactoring @pnowojski is doing on prestodb#3938.

implements ParametricType
{
public static final DecimalParametricType DECIMAL = new DecimalParametricType();

@Override
public String getName()
{
return StandardTypes.DECIMAL;
}

@Override
public Type createType(List<Type> types, List<Object> literals)
{
checkArgument(types.isEmpty(), "Type parameters not allowed for DECIMAL");
if (literals.size() == 0) {
// we need support for that for support of unparametrized TypeSignatures which are
// used for defining DECIMAL related functions. See e.g. DecimalOperators class.
return ShortDecimalType.createUnparametrizedDecimal();
}

return createParametrizedDecimal(literals);
}

@NotNull
private Type createParametrizedDecimal(List<Object> literals)
{
checkArgument(literals.size() == 2, "Expected 2 literal parameters for DECIMAL");
checkArgument(literals.get(0) instanceof Long &&
literals.get(1) instanceof Long, "Expected both literal parameters for DECIMAL to be numbers");
long precision = (long) literals.get(0);
long scale = (long) literals.get(1);
return DecimalType.createDecimalType((int) precision, (int) scale);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@ public TypeRegistry(Set<Type> types)
addType(COLOR);
addType(JSON);
addParametricType(VarcharParametricType.VARCHAR);
addParametricType(DecimalParametricType.DECIMAL);
addParametricType(ROW);
addParametricType(ARRAY);
addParametricType(MAP);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
/*
* Licensed 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 com.facebook.presto.type;

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.DecimalType;
import com.facebook.presto.spi.type.LongDecimalType;
import com.facebook.presto.spi.type.SqlDecimal;
import io.airlift.slice.Slice;
import io.airlift.slice.Slices;
import org.testng.annotations.Test;

import java.math.BigDecimal;
import java.math.BigInteger;

import static com.facebook.presto.spi.type.LongDecimalType.unscaledValueToSlice;
import static org.testng.Assert.assertEquals;

public class TestLongDecimalType
extends AbstractTestType
{
private static final LongDecimalType LONG_DECIMAL_TYPE = (LongDecimalType) DecimalType.createDecimalType(30, 10);

public TestLongDecimalType()
{
super(LONG_DECIMAL_TYPE, SqlDecimal.class, createTestBlock());
}

@Test
public void testUnscaledValueToSlice()
{
assertEquals(unscaledValueToSlice(0L), unscaledValueToSlice(BigInteger.valueOf(0L)));
assertEquals(unscaledValueToSlice(1L), unscaledValueToSlice(BigInteger.valueOf(1L)));
assertEquals(unscaledValueToSlice(-1L), unscaledValueToSlice(BigInteger.valueOf(-1L)));
assertEquals(unscaledValueToSlice(Long.MAX_VALUE), unscaledValueToSlice(BigInteger.valueOf(Long.MAX_VALUE)));
assertEquals(unscaledValueToSlice(Long.MIN_VALUE), unscaledValueToSlice(BigInteger.valueOf(Long.MIN_VALUE)));
}

public static Block createTestBlock()
{
BlockBuilder blockBuilder = LONG_DECIMAL_TYPE.createBlockBuilder(new BlockBuilderStatus(), 15);
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("-12345678901234567890.1234567890"));
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("-12345678901234567890.1234567890"));
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("-12345678901234567890.1234567890"));
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("22345678901234567890.1234567890"));
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("22345678901234567890.1234567890"));
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("22345678901234567890.1234567890"));
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("22345678901234567890.1234567890"));
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("22345678901234567890.1234567890"));
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("32345678901234567890.1234567890"));
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("32345678901234567890.1234567890"));
LONG_DECIMAL_TYPE.writeBigDecimal(blockBuilder, new BigDecimal("42345678901234567890.1234567890"));
return blockBuilder.build();
}

@Override
protected Object getNonNullValue()
{
return Slices.wrappedBuffer(new byte[16]);
}

@Override
protected Object getGreaterValue(Object value)
{
Slice slice = (Slice) value;
BigDecimal decimal = LongDecimalType.toBigDecimal(slice, 10);
BigDecimal greaterDecimal = decimal.add(BigDecimal.ONE);
return LONG_DECIMAL_TYPE.bigDecimalToSlice(greaterDecimal);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* Licensed 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 com.facebook.presto.type;

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.DecimalType;
import com.facebook.presto.spi.type.ShortDecimalType;
import com.facebook.presto.spi.type.SqlDecimal;

public class TestShortDecimalType
extends AbstractTestType
{
private static final ShortDecimalType SHORT_DECIMAL_TYPE = (ShortDecimalType) DecimalType.createDecimalType(4, 2);

public TestShortDecimalType()
{
super(SHORT_DECIMAL_TYPE, SqlDecimal.class, createTestBlock());
}

public static Block createTestBlock()
{
BlockBuilder blockBuilder = SHORT_DECIMAL_TYPE.createBlockBuilder(new BlockBuilderStatus(), 15);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, -1234);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, -1234);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, -1234);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, 2321);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, 2321);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, 2321);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, 2321);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, 2321);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, 3321);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, 3321);
SHORT_DECIMAL_TYPE.writeLong(blockBuilder, 4321);
return blockBuilder.build();
}

@Override
protected Object getGreaterValue(Object value)
{
return ((long) value) + 1;
}
}
151 changes: 151 additions & 0 deletions presto-spi/src/main/java/com/facebook/presto/spi/type/DecimalType.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
/*
* Licensed 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 com.facebook.presto.spi.type;

import com.facebook.presto.spi.PrestoException;

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

import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
import static java.util.Collections.emptyList;
import static java.util.Collections.unmodifiableList;

public abstract class DecimalType

This comment has been minimized.

Copy link
@martint

martint Nov 16, 2015

Why is this a separate class from DecimalParametricType?

extends AbstractFixedWidthType
{
protected static final int UNSET = -1;
protected static final int TEN_TO_NTH_TABLE_LENGTH = 100;
public static final int MAX_PRECISION = 38;
public static final int MAX_SHORT_PRECISION = 17;

public static DecimalType createDecimalType(int precision, int scale)
{
if (precision <= MAX_SHORT_PRECISION) {
return new ShortDecimalType(precision, scale);
}
else {
return new LongDecimalType(precision, scale);
}
}

public static DecimalType createDecimalType(int precision)
{
return createDecimalType(precision, 0);
}

public static DecimalType createUnparametrizedDecimal()
{
return new UnparametrizedDecimalType();
}

protected final int precision;
protected final int scale;

protected DecimalType(int precision, int scale, Class<?> javaType, int fixedSize)
{
super(new TypeSignature(StandardTypes.DECIMAL, emptyList(), buildPrecisionScaleList(precision, scale)), javaType, fixedSize);
this.precision = precision;
this.scale = scale;
}

protected DecimalType()
{
super(new TypeSignature(StandardTypes.DECIMAL, emptyList(), emptyList()), long.class, 0);
this.precision = UNSET;
this.scale = UNSET;
}

public static Object unscaledValueToObject(String unscaledValue, int precision)
{
Object value;
if (precision <= MAX_SHORT_PRECISION) {
value = Long.parseLong(unscaledValue);
}
else {
value = LongDecimalType.unscaledValueToSlice(unscaledValue);
}
return value;
}

@Override
public boolean isComparable()
{
return true;
}

@Override
public boolean isOrderable()
{
return true;
}

protected void validatePrecisionScale(int precision, int scale, int maxPrecision)
{
if (precision < 0 || precision > maxPrecision) {
throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Invalid DECIMAL precision " + precision);
}

if (scale < 0 || scale > precision) {
throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Invalid DECIMAL scale " + scale);
}
}

public int getPrecision()
{
return precision;
}

public int getScale()
{
return scale;
}

public boolean isShort()
{
return precision <= MAX_SHORT_PRECISION;
}

private static List<Object> buildPrecisionScaleList(int precision, int scale)
{
List<Object> literalArguments = new ArrayList<>();
literalArguments.add((long) precision);
literalArguments.add((long) scale);
return unmodifiableList(literalArguments);
}

public static String toString(String unscaledValueString, int precision, int scale)
{
int positiveUnscaledValueStringStart = unscaledValueString.startsWith("-") ? 1 : 0;
StringBuilder unscaledValueWithLeadingZerosBuilder = new StringBuilder();
for (int i = 0; i < precision - unscaledValueString.length() + positiveUnscaledValueStringStart; ++i) {
unscaledValueWithLeadingZerosBuilder.append('0');
}

unscaledValueWithLeadingZerosBuilder.append(unscaledValueString.substring(positiveUnscaledValueStringStart));
String unscaledValueWithLeadingZeros = unscaledValueWithLeadingZerosBuilder.toString();
StringBuilder resultBuilder = new StringBuilder();
if (positiveUnscaledValueStringStart > 0) {
resultBuilder.append("-");
}
resultBuilder.append(unscaledValueWithLeadingZeros, 0, precision - scale);
if (scale != 0) {
resultBuilder.append('.');
resultBuilder.append(unscaledValueWithLeadingZeros, precision - scale, precision);
}

return resultBuilder.toString();
}
}
Loading

0 comments on commit 4005239

Please sign in to comment.