Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[improve]Compatible with reading doris date/datetime arrow type modification #351

Merged
merged 3 commits into from
Mar 22, 2024
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -33,6 +33,8 @@
import org.apache.arrow.vector.VarBinaryVector;
import org.apache.arrow.vector.VarCharVector;
import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.arrow.vector.DateDayVector;
import org.apache.arrow.vector.TimeStampMicroVector;
import org.apache.arrow.vector.complex.ListVector;
import org.apache.arrow.vector.complex.MapVector;
import org.apache.arrow.vector.complex.StructVector;
Expand All @@ -50,16 +52,24 @@
import java.io.IOException;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.charset.StandardCharsets;
import java.sql.Date;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.ZoneId;
import java.time.format.DateTimeFormatter;
import java.time.format.DateTimeFormatterBuilder;
import java.time.temporal.ChronoField;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;

/** row batch data container. */
/**
* row batch data container.
*/
public class RowBatch {
private static Logger logger = LoggerFactory.getLogger(RowBatch.class);

Expand Down Expand Up @@ -89,12 +99,10 @@ public void put(Object o) {
private List<FieldVector> fieldVectors;
private RootAllocator rootAllocator;
private final Schema schema;
private static final String DATETIME_PATTERN = "yyyy-MM-dd HH:mm:ss";
private static final String DATETIMEV2_PATTERN = "yyyy-MM-dd HH:mm:ss.SSSSSS";
private final DateTimeFormatter dateTimeFormatter =
DateTimeFormatter.ofPattern(DATETIME_PATTERN);
private final DateTimeFormatter dateTimeV2Formatter =
DateTimeFormatter.ofPattern(DATETIMEV2_PATTERN);
private static final DateTimeFormatter DATE_TIME_FORMATTER = new DateTimeFormatterBuilder()
.appendPattern("yyyy-MM-dd HH:mm:ss")
.appendFraction(ChronoField.MICRO_OF_SECOND, 0, 6, true)
.toFormatter();
private final DateTimeFormatter dateFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd");

public List<Row> getRowBatch() {
Expand Down Expand Up @@ -274,44 +282,62 @@ private boolean doConvert(
break;
case "DATE":
case "DATEV2":
if (!minorType.equals(Types.MinorType.VARCHAR)) {
if (!minorType.equals(Types.MinorType.DATEDAY)
&& !minorType.equals(Types.MinorType.VARCHAR)) {
return false;
}
VarCharVector date = (VarCharVector) fieldVector;
if (date.isNull(rowIndex)) {
addValueToRow(rowIndex, null);
break;
if (minorType.equals(Types.MinorType.VARCHAR)) {
VarCharVector date = (VarCharVector) fieldVector;
if (date.isNull(rowIndex)) {
addValueToRow(rowIndex, null);
break;
}
String stringValue = new String(date.get(rowIndex));
LocalDate localDate = LocalDate.parse(stringValue, dateFormatter);
addValueToRow(rowIndex, localDate);
} else {
DateDayVector date = (DateDayVector) fieldVector;
if (date.isNull(rowIndex)) {
addValueToRow(rowIndex, null);
break;
}
LocalDate localDate = LocalDate.ofEpochDay(date.get(rowIndex));
addValueToRow(rowIndex, Date.valueOf(localDate));
}
String stringValue = new String(date.get(rowIndex));
LocalDate localDate = LocalDate.parse(stringValue, dateFormatter);
addValueToRow(rowIndex, localDate);
break;
case "DATETIME":
if (!minorType.equals(Types.MinorType.VARCHAR)) {
return false;
}
VarCharVector timeStampSecVector = (VarCharVector) fieldVector;
if (timeStampSecVector.isNull(rowIndex)) {
addValueToRow(rowIndex, null);
break;
}
stringValue = new String(timeStampSecVector.get(rowIndex));
LocalDateTime parse = LocalDateTime.parse(stringValue, dateTimeFormatter);
addValueToRow(rowIndex, parse);
break;
case "DATETIMEV2":
if (!minorType.equals(Types.MinorType.VARCHAR)) {
if (!minorType.equals(Types.MinorType.TIMESTAMPMICRO)
&& !minorType.equals(Types.MinorType.VARCHAR)) {
return false;
}
VarCharVector timeStampV2SecVector = (VarCharVector) fieldVector;
if (timeStampV2SecVector.isNull(rowIndex)) {
addValueToRow(rowIndex, null);
break;
if (minorType.equals(Types.MinorType.VARCHAR)) {
VarCharVector varCharVector = (VarCharVector) fieldVector;
if (varCharVector.isNull(rowIndex)) {
addValueToRow(rowIndex, null);
break;
}
String stringValue = new String(varCharVector.get(rowIndex), StandardCharsets.UTF_8);
addValueToRow(rowIndex, stringValue);
} else {
TimeStampMicroVector vector = (TimeStampMicroVector) fieldVector;
if (vector.isNull(rowIndex)) {
addValueToRow(rowIndex, null);
break;
}
long time = vector.get(rowIndex);
Instant instant;
if (time / 10000000000L == 0) { // datetime(0)
instant = Instant.ofEpochSecond(time);
} else if (time / 10000000000000L == 0) { // datetime(3)
instant = Instant.ofEpochMilli(time);
} else { // datetime(6)
instant = Instant.ofEpochSecond(time / 1000000, time % 1000000 * 1000);
}
LocalDateTime dateTime = LocalDateTime.ofInstant(instant, ZoneId.systemDefault());
String formatted = DATE_TIME_FORMATTER.format(dateTime);
addValueToRow(rowIndex, formatted);
}
stringValue = new String(timeStampV2SecVector.get(rowIndex));
stringValue = completeMilliseconds(stringValue);
parse = LocalDateTime.parse(stringValue, dateTimeV2Formatter);
addValueToRow(rowIndex, parse);
break;
case "LARGEINT":
if (!minorType.equals(Types.MinorType.FIXEDSIZEBINARY)
Expand Down Expand Up @@ -342,7 +368,7 @@ private boolean doConvert(
addValueToRow(rowIndex, null);
break;
}
stringValue = new String(largeIntVector.get(rowIndex));
String stringValue = new String(largeIntVector.get(rowIndex));
BigInteger largeInt = new BigInteger(stringValue);
addValueToRow(rowIndex, largeInt);
break;
Expand All @@ -359,7 +385,7 @@ private boolean doConvert(
addValueToRow(rowIndex, null);
break;
}
stringValue = new String(varCharVector.get(rowIndex));
String stringValue = new String(varCharVector.get(rowIndex));
addValueToRow(rowIndex, stringValue);
break;
case "ARRAY":
Expand Down Expand Up @@ -409,19 +435,6 @@ private boolean doConvert(
return true;
}

private String completeMilliseconds(String stringValue) {
if (stringValue.length() == DATETIMEV2_PATTERN.length()) {
return stringValue;
}
StringBuilder sb = new StringBuilder(stringValue);
if (stringValue.length() == DATETIME_PATTERN.length()) {
sb.append(".");
}
while (sb.toString().length() < DATETIMEV2_PATTERN.length()) {
sb.append(0);
}
return sb.toString();
}

public List<Object> next() {
if (!hasNext()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,24 +17,15 @@

package org.apache.doris.flink.serialization;

import org.apache.arrow.vector.*;
import org.apache.arrow.vector.types.DateUnit;
import org.apache.commons.lang3.ArrayUtils;
Copy link
Member

Choose a reason for hiding this comment

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

not import *

import org.apache.flink.table.data.DecimalData;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.arrow.memory.ArrowBuf;
import org.apache.arrow.memory.RootAllocator;
import org.apache.arrow.vector.BigIntVector;
import org.apache.arrow.vector.BitVector;
import org.apache.arrow.vector.DecimalVector;
import org.apache.arrow.vector.FieldVector;
import org.apache.arrow.vector.Float4Vector;
import org.apache.arrow.vector.Float8Vector;
import org.apache.arrow.vector.IntVector;
import org.apache.arrow.vector.SmallIntVector;
import org.apache.arrow.vector.TinyIntVector;
import org.apache.arrow.vector.VarBinaryVector;
import org.apache.arrow.vector.VarCharVector;
import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.arrow.vector.complex.MapVector;
import org.apache.arrow.vector.complex.StructVector;
import org.apache.arrow.vector.complex.impl.NullableStructWriter;
Expand Down Expand Up @@ -62,7 +53,9 @@
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.charset.StandardCharsets;
import java.sql.Date;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.util.ArrayList;
Expand All @@ -75,7 +68,8 @@
public class TestRowBatch {
private static Logger logger = LoggerFactory.getLogger(TestRowBatch.class);

@Rule public ExpectedException thrown = ExpectedException.none();
@Rule
public ExpectedException thrown = ExpectedException.none();

@Test
public void testRowBatch() throws Exception {
Expand Down Expand Up @@ -622,4 +616,162 @@ public void testStruct() throws IOException, DorisException {
Assert.assertTrue(
ImmutableMap.of("a", new Text("a1"), "b", 1).equals(rowBatch.next().get(0)));
}
}

@Test
public void testDate() throws DorisException, IOException {

ImmutableList.Builder<Field> childrenBuilder = ImmutableList.builder();
childrenBuilder.add(new Field("k1", FieldType.nullable(new ArrowType.Utf8()), null));
childrenBuilder.add(new Field("k2", FieldType.nullable(new ArrowType.Utf8()), null));
childrenBuilder.add(new Field("k3", FieldType.nullable(new ArrowType.Date(DateUnit.DAY)), null));

VectorSchemaRoot root = VectorSchemaRoot.create(
new org.apache.arrow.vector.types.pojo.Schema(childrenBuilder.build(), null),
new RootAllocator(Integer.MAX_VALUE));
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
ArrowStreamWriter arrowStreamWriter = new ArrowStreamWriter(
root,
new DictionaryProvider.MapDictionaryProvider(),
outputStream);

arrowStreamWriter.start();
root.setRowCount(1);

FieldVector vector = root.getVector("k1");
VarCharVector dateVector = (VarCharVector) vector;
dateVector.setInitialCapacity(1);
dateVector.allocateNew();
dateVector.setIndexDefined(0);
dateVector.setValueLengthSafe(0, 10);
dateVector.setSafe(0, "2023-08-09".getBytes());
vector.setValueCount(1);


vector = root.getVector("k2");
VarCharVector dateV2Vector = (VarCharVector) vector;
dateV2Vector.setInitialCapacity(1);
dateV2Vector.allocateNew();
dateV2Vector.setIndexDefined(0);
dateV2Vector.setValueLengthSafe(0, 10);
dateV2Vector.setSafe(0, "2023-08-10".getBytes());
vector.setValueCount(1);

vector = root.getVector("k3");
DateDayVector dateNewVector = (DateDayVector) vector;
dateNewVector.setInitialCapacity(1);
dateNewVector.allocateNew();
dateNewVector.setIndexDefined(0);
dateNewVector.setSafe(0, 19802);
vector.setValueCount(1);

arrowStreamWriter.writeBatch();

arrowStreamWriter.end();
arrowStreamWriter.close();

TStatus status = new TStatus();
status.setStatusCode(TStatusCode.OK);
TScanBatchResult scanBatchResult = new TScanBatchResult();
scanBatchResult.setStatus(status);
scanBatchResult.setEos(false);
scanBatchResult.setRows(outputStream.toByteArray());


String schemaStr = "{\"properties\":[" +
"{\"type\":\"DATE\",\"name\":\"k1\",\"comment\":\"\"}, " +
"{\"type\":\"DATEV2\",\"name\":\"k2\",\"comment\":\"\"}, " +
"{\"type\":\"DATEV2\",\"name\":\"k3\",\"comment\":\"\"}" +
"], \"status\":200}";

Schema schema = RestService.parseSchema(schemaStr, logger);

RowBatch rowBatch = new RowBatch(scanBatchResult, schema).readArrow();

Assert.assertTrue(rowBatch.hasNext());
List<Object> actualRow0 = rowBatch.next();
Assert.assertEquals(LocalDate.of(2023, 8, 9), actualRow0.get(0));
Assert.assertEquals(LocalDate.of(2023,8,10), actualRow0.get(1));
Assert.assertEquals(Date.valueOf("2024-03-20"), actualRow0.get(2));

Assert.assertFalse(rowBatch.hasNext());
thrown.expect(NoSuchElementException.class);
thrown.expectMessage(startsWith("Get row offset:"));
rowBatch.next();

}

@Test
public void testLargeInt() throws DorisException, IOException {

ImmutableList.Builder<Field> childrenBuilder = ImmutableList.builder();
childrenBuilder.add(new Field("k1", FieldType.nullable(new ArrowType.Utf8()), null));
childrenBuilder.add(new Field("k2", FieldType.nullable(new ArrowType.FixedSizeBinary(16)), null));

VectorSchemaRoot root = VectorSchemaRoot.create(
new org.apache.arrow.vector.types.pojo.Schema(childrenBuilder.build(), null),
new RootAllocator(Integer.MAX_VALUE));
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
ArrowStreamWriter arrowStreamWriter = new ArrowStreamWriter(
root,
new DictionaryProvider.MapDictionaryProvider(),
outputStream);

arrowStreamWriter.start();
root.setRowCount(1);

FieldVector vector = root.getVector("k1");
VarCharVector lageIntVector = (VarCharVector) vector;
lageIntVector.setInitialCapacity(1);
lageIntVector.allocateNew();
lageIntVector.setIndexDefined(0);
lageIntVector.setValueLengthSafe(0, 19);
lageIntVector.setSafe(0, "9223372036854775808".getBytes());
vector.setValueCount(1);


vector = root.getVector("k2");
FixedSizeBinaryVector lageIntVector1 = (FixedSizeBinaryVector) vector;
lageIntVector1.setInitialCapacity(1);
lageIntVector1.allocateNew();
lageIntVector1.setIndexDefined(0);
byte[] bytes = new BigInteger("9223372036854775809").toByteArray();
byte[] fixedBytes = new byte[16];
System.arraycopy(bytes, 0, fixedBytes, 16 - bytes.length, bytes.length);
ArrayUtils.reverse(fixedBytes);
lageIntVector1.setSafe(0, fixedBytes);
vector.setValueCount(1);

arrowStreamWriter.writeBatch();

arrowStreamWriter.end();
arrowStreamWriter.close();

TStatus status = new TStatus();
status.setStatusCode(TStatusCode.OK);
TScanBatchResult scanBatchResult = new TScanBatchResult();
scanBatchResult.setStatus(status);
scanBatchResult.setEos(false);
scanBatchResult.setRows(outputStream.toByteArray());

String schemaStr = "{\"properties\":[" +
"{\"type\":\"LARGEINT\",\"name\":\"k1\",\"comment\":\"\"}, " +
"{\"type\":\"LARGEINT\",\"name\":\"k2\",\"comment\":\"\"}" +
"], \"status\":200}";

Schema schema = RestService.parseSchema(schemaStr, logger);

RowBatch rowBatch = new RowBatch(scanBatchResult, schema).readArrow();

Assert.assertTrue(rowBatch.hasNext());
List<Object> actualRow0 = rowBatch.next();

Assert.assertEquals(new BigInteger("9223372036854775808"), actualRow0.get(0));
Assert.assertEquals(new BigInteger("9223372036854775809"), actualRow0.get(1));

Assert.assertFalse(rowBatch.hasNext());
thrown.expect(NoSuchElementException.class);
thrown.expectMessage(startsWith("Get row offset:"));
rowBatch.next();

}
}
Loading