Skip to content

Commit

Permalink
Fix interval type handling in REST API
Browse files Browse the repository at this point in the history
Previously, interval day to second types greater than INT_MAX milliseconds
(~24 days) and interval year to month types greater than INT_MAX months
returned the wrong value due to integer overflow.
  • Loading branch information
cberner committed Jun 10, 2016
1 parent ea77194 commit 86232df
Show file tree
Hide file tree
Showing 5 changed files with 67 additions and 2 deletions.
4 changes: 4 additions & 0 deletions presto-docs/src/main/sphinx/release/release-0.148.rst
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,10 @@ General Changes
* Fix data duplication when ``task.writer-count`` configuration mismatches between coordinator and worker.
* Fix bug where ``node-scheduler.max-pending-splits-per-node-per-task`` config is not always
honored by node scheduler. This bug could stop the cluster from making further progress.
* Fix handling of ``INTERVAL DAY TO SECOND`` type in REST API. Previously, intervals greater than
``2,147,483,647`` milliseconds (about ``24`` days) were returned as the wrong value.
* Fix handling of ``INTERVAL YEAR TO MONTH`` type in REST API. Previously, intervals greater than
``2,147,483,647`` months were returned as the wrong value.
* Add ``colocated-joins-enabled`` to enable colocated joins by default for
connectors that expose node-partitioned data.
* Add support for colocated unions.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public Object getObjectValue(ConnectorSession session, Block block, int position
if (block.isNull(position)) {
return null;
}
return new SqlIntervalDayTime((int) block.getLong(position, 0));
return new SqlIntervalDayTime(block.getLong(position, 0));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public Object getObjectValue(ConnectorSession session, Block block, int position
if (block.isNull(position)) {
return null;
}
return new SqlIntervalYearMonth((int) block.getLong(position, 0));
return new SqlIntervalYearMonth(block.getLong(position, 0));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
import com.facebook.presto.metadata.FunctionListBuilder;
import com.facebook.presto.metadata.SqlFunction;
import com.facebook.presto.spi.session.PropertyMetadata;
import com.facebook.presto.spi.type.SqlIntervalDayTime;
import com.facebook.presto.spi.type.SqlIntervalYearMonth;
import com.facebook.presto.spi.type.TimeZoneKey;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.sql.analyzer.SemanticException;
Expand Down Expand Up @@ -135,6 +137,19 @@ public void testParsingError()
computeActual("SELECT foo FROM");
}

@Test
public void selectLargeInterval()
throws Exception
{
MaterializedResult result = computeActual("SELECT INTERVAL '30' DAY");
assertEquals(result.getRowCount(), 1);
assertEquals(result.getMaterializedRows().get(0).getField(0), new SqlIntervalDayTime(30, 0, 0, 0, 0));

result = computeActual("SELECT INTERVAL '" + Integer.MAX_VALUE + "' YEAR");
assertEquals(result.getRowCount(), 1);
assertEquals(result.getMaterializedRows().get(0).getField(0), new SqlIntervalYearMonth(Integer.MAX_VALUE, 0));
}

@Test
public void selectNull()
throws Exception
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
import com.facebook.presto.client.QueryResults;
import com.facebook.presto.server.testing.TestingPrestoServer;
import com.facebook.presto.spi.type.DecimalType;
import com.facebook.presto.spi.type.SqlIntervalDayTime;
import com.facebook.presto.spi.type.SqlIntervalYearMonth;
import com.facebook.presto.spi.type.TimeZoneKey;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.spi.type.VarcharType;
Expand All @@ -26,6 +28,9 @@
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import io.airlift.log.Logger;
import org.joda.time.Period;
import org.joda.time.format.PeriodFormatter;
import org.joda.time.format.PeriodFormatterBuilder;

import java.math.BigDecimal;
import java.sql.Date;
Expand All @@ -47,6 +52,8 @@
import static com.facebook.presto.spi.type.DateType.DATE;
import static com.facebook.presto.spi.type.DoubleType.DOUBLE;
import static com.facebook.presto.spi.type.IntegerType.INTEGER;
import static com.facebook.presto.spi.type.IntervalDayTimeType.INTERVAL_DAY_TIME;
import static com.facebook.presto.spi.type.IntervalYearMonthType.INTERVAL_YEAR_MONTH;
import static com.facebook.presto.spi.type.SmallintType.SMALLINT;
import static com.facebook.presto.spi.type.TimeType.TIME;
import static com.facebook.presto.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE;
Expand All @@ -71,6 +78,30 @@ public class TestingPrestoClient
{
private static final Logger log = Logger.get("TestQueries");

private static final PeriodFormatter INTERVAL_YEAR_TO_MONTH_FORMATTER = new PeriodFormatterBuilder()
.appendYears()
.appendLiteral("-")
.appendMonths()
.toFormatter();

private static final PeriodFormatter INTERVAL_DAY_TO_SECOND_FORMATTER = new PeriodFormatterBuilder()
.appendDays()
.appendLiteral(" ")
.appendHours()
.appendLiteral(":")
.appendMinutes()
.appendLiteral(":")
.appendSecondsWithOptionalMillis()
.toFormatter();

private static final int YEAR_FIELD = 0;
private static final int MONTH_FIELD = 1;
private static final int DAY_FIELD = 3;
private static final int HOUR_FIELD = 4;
private static final int MINUTE_FIELD = 5;
private static final int SECOND_FIELD = 6;
private static final int MILLIS_FIELD = 7;

public TestingPrestoClient(TestingPrestoServer prestoServer, Session defaultSession)
{
super(prestoServer, defaultSession);
Expand Down Expand Up @@ -203,6 +234,21 @@ else if (TIMESTAMP.equals(type)) {
else if (TIMESTAMP_WITH_TIME_ZONE.equals(type)) {
return new Timestamp(unpackMillisUtc(parseTimestampWithTimeZone(timeZoneKey, (String) value)));
}
else if (INTERVAL_DAY_TIME.equals(type)) {
Period period = INTERVAL_DAY_TO_SECOND_FORMATTER.parsePeriod(String.valueOf(value));
return new SqlIntervalDayTime(
period.getValue(DAY_FIELD),
period.getValue(HOUR_FIELD),
period.getValue(MINUTE_FIELD),
period.getValue(SECOND_FIELD),
period.getValue(MILLIS_FIELD));
}
else if (INTERVAL_YEAR_MONTH.equals(type)) {
Period period = INTERVAL_YEAR_TO_MONTH_FORMATTER.parsePeriod(String.valueOf(value));
return new SqlIntervalYearMonth(
period.getValue(YEAR_FIELD),
period.getValue(MONTH_FIELD));
}
else if (type instanceof ArrayType) {
return ((List<Object>) value).stream()
.map(element -> convertToRowValue(((ArrayType) type).getElementType(), element, timeZoneKey))
Expand Down

0 comments on commit 86232df

Please sign in to comment.