Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,12 @@
import com.facebook.airlift.log.Logger;
import com.facebook.plugin.arrow.testingServer.TestingArrowFlightRequest;
import com.facebook.plugin.arrow.testingServer.TestingArrowFlightResponse;
import com.facebook.presto.Session;
import com.facebook.presto.common.function.OperatorType;
import com.facebook.presto.common.type.ArrayType;
import com.facebook.presto.common.type.MapType;
import com.facebook.presto.common.type.RowType;
import com.facebook.presto.common.type.TimeZoneKey;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.testing.MaterializedResult;
import com.facebook.presto.testing.QueryRunner;
Expand All @@ -47,8 +49,11 @@
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.memory.RootAllocator;
import org.apache.arrow.vector.BigIntVector;
import org.apache.arrow.vector.DateDayVector;
import org.apache.arrow.vector.FieldVector;
import org.apache.arrow.vector.IntVector;
import org.apache.arrow.vector.TimeMilliVector;
import org.apache.arrow.vector.TimeStampMilliVector;
import org.apache.arrow.vector.VarCharVector;
import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.arrow.vector.complex.ListVector;
Expand Down Expand Up @@ -83,6 +88,7 @@
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.time.LocalDateTime;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
Expand All @@ -92,19 +98,25 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TimeZone;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;

import static com.facebook.airlift.json.JsonCodec.jsonCodec;
import static com.facebook.presto.SystemSessionProperties.LEGACY_TIMESTAMP;
import static com.facebook.presto.common.block.MethodHandleUtil.compose;
import static com.facebook.presto.common.block.MethodHandleUtil.nativeValueGetter;
import static com.facebook.presto.common.type.BigintType.BIGINT;
import static com.facebook.presto.common.type.DateType.DATE;
import static com.facebook.presto.common.type.IntegerType.INTEGER;
import static com.facebook.presto.common.type.TimeType.TIME;
import static com.facebook.presto.common.type.TimestampType.TIMESTAMP;
import static com.facebook.presto.common.type.VarcharType.VARCHAR;
import static com.facebook.presto.testing.MaterializedResult.resultBuilder;
import static com.facebook.presto.testing.TestingEnvironment.getOperatorMethodHandle;
import static com.facebook.presto.testing.assertions.Assert.assertEquals;
import static com.facebook.presto.util.DateTimeUtils.parseTimestampWithoutTimeZone;
import static java.lang.String.format;
import static java.nio.channels.Channels.newChannel;

Expand Down Expand Up @@ -159,6 +171,57 @@ protected QueryRunner createQueryRunner()
return ArrowFlightQueryRunner.createQueryRunner(serverPort);
}

@Test
public void testDateTimeVectors() throws Exception
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

suggestion (testing): Missing test cases for null values in date, time, and timestamp vectors.

Please add tests with null entries in the date, time, and timestamp vectors to verify correct handling of missing values.

Suggested implementation:

    public void testDateTimeVectors() throws Exception
    {
        try (BufferAllocator bufferAllocator = allocator.newChildAllocator("echo-test-client", 0, Long.MAX_VALUE);
                IntVector intVector = new IntVector("id", bufferAllocator);
                DateDayVector dateVector = new DateDayVector("date", bufferAllocator);
                TimeMilliVector timeVector = new TimeMilliVector("time", bufferAllocator);
                TimeStampMilliVector timestampVector = new TimeStampMilliVector("timestamp", bufferAllocator);
                VectorSchemaRoot root = new VectorSchemaRoot(Arrays.asList(intVector, dateVector, timeVector, timestampVector));
                FlightClient client = createFlightClient(bufferAllocator, serverPort)) {
            MaterializedResult.Builder expectedBuilder = resultBuilder(getSession(), INTEGER, DATE, TIME, TIMESTAMP);

            // Add test data including nulls
            int rowCount = 4;
            root.setRowCount(rowCount);

            intVector.setSafe(0, 1);
            dateVector.setSafe(0, 18262); // 2020-01-01
            timeVector.setSafe(0, 3600000); // 01:00:00.000
            timestampVector.setSafe(0, 1577836800000L); // 2020-01-01T00:00:00.000Z

            intVector.setSafe(1, 2);
            dateVector.setNull(1); // null date
            timeVector.setSafe(1, 7200000); // 02:00:00.000
            timestampVector.setSafe(1, 1577923200000L); // 2020-01-02T00:00:00.000Z

            intVector.setSafe(2, 3);
            dateVector.setSafe(2, 18264); // 2020-01-03
            timeVector.setNull(2); // null time
            timestampVector.setSafe(2, 1578096000000L); // 2020-01-04T00:00:00.000Z

            intVector.setSafe(3, 4);
            dateVector.setSafe(3, 18265); // 2020-01-04
            timeVector.setSafe(3, 10800000); // 03:00:00.000
            timestampVector.setNull(3); // null timestamp

            expectedBuilder.row(1, LocalDate.of(2020, 1, 1), LocalTime.of(1, 0), LocalDateTime.of(2020, 1, 1, 0, 0));
            expectedBuilder.row(2, null, LocalTime.of(2, 0), LocalDateTime.of(2020, 1, 2, 0, 0));
            expectedBuilder.row(3, LocalDate.of(2020, 1, 3), null, LocalDateTime.of(2020, 1, 4, 0, 0));
            expectedBuilder.row(4, LocalDate.of(2020, 1, 4), LocalTime.of(3, 0), null);

            MaterializedResult expected = expectedBuilder.build();

            // Send and receive the data, then assert
            try (ArrowStreamWriter writer = new ArrowStreamWriter(root, null, client.startPut(...))) {
                writer.writeBatch();
                writer.end();
            }

            MaterializedResult actual = ... // fetch result from FlightClient

            assertEquals(actual, expected);

You will need to:

  1. Fill in the code for sending the data via FlightClient and fetching the result (...).
  2. Ensure that the MaterializedResult comparison works with nulls.
  3. Import LocalDate, LocalTime, and LocalDateTime if not already imported.

{
try (BufferAllocator bufferAllocator = allocator.newChildAllocator("echo-test-client", 0, Long.MAX_VALUE);
IntVector intVector = new IntVector("id", bufferAllocator);
DateDayVector dateVector = new DateDayVector("date", bufferAllocator);
TimeMilliVector timeVector = new TimeMilliVector("time", bufferAllocator);
TimeStampMilliVector timestampVector = new TimeStampMilliVector("timestamp", bufferAllocator);
VectorSchemaRoot root = new VectorSchemaRoot(Arrays.asList(intVector, dateVector, timeVector, timestampVector));
FlightClient client = createFlightClient(bufferAllocator, serverPort)) {
MaterializedResult.Builder expectedBuilder = resultBuilder(getSession(), INTEGER, DATE, TIME, TIMESTAMP);

List<String> values = ImmutableList.of(
"1970-01-01T00:00:00",
"2024-01-01T01:01:01",
"2024-01-02T12:00:00",
"2112-12-31T23:58:00",
"1968-07-05T08:15:12.345");

for (int i = 0; i < values.size(); i++) {
intVector.setSafe(i, i);
LocalDateTime dateTime = LocalDateTime.parse(values.get(i));
// First vector value is explicitly set to 0 to ensure no issues with parsing
dateVector.setSafe(i, i == 0 ? 0 : (int) dateTime.toLocalDate().toEpochDay());
timeVector.setSafe(i, i == 0 ? 0 : (int) TimeUnit.NANOSECONDS.toMillis(dateTime.toLocalTime().toNanoOfDay()));
timestampVector.setSafe(i, i == 0 ? 0 : parseTimestampWithoutTimeZone(values.get(i).replace("T", " ")));
expectedBuilder.row(i, dateTime.toLocalDate(), dateTime.toLocalTime(), dateTime);
}

root.setRowCount(values.size());

String tableName = "datetime";
addTableToServer(client, root, tableName);

for (String timeZoneId : ImmutableList.of(TimeZone.getDefault().getID(), "UTC", "America/New_York", "Asia/Tokyo")) {
Session sessionWithTimezone = Session.builder(getSession())
.setTimeZoneKey(TimeZoneKey.getTimeZoneKey(timeZoneId))
.setSystemProperty(LEGACY_TIMESTAMP, "false")
.build();

MaterializedResult actual = computeActual(sessionWithTimezone, format("SELECT * FROM %s", tableName));
MaterializedResult expected = expectedBuilder.build();

assertEquals(actual.getRowCount(), root.getRowCount());
assertEquals(actual, expected);
}

removeTableFromServer(client, tableName);
}
}

@Test
public void testVarCharVector() throws Exception
{
Expand Down
Loading