ptupitsyn commented on code in PR #4142: URL: https://github.com/apache/ignite-3/pull/4142#discussion_r1713700047
########## modules/binary-tuple/src/main/java/org/apache/ignite/internal/binarytuple/inlineschema/TupleWithSchemaMarshalling.java: ########## @@ -0,0 +1,333 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.ignite.internal.binarytuple.inlineschema; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.Period; +import java.util.UUID; +import java.util.function.BiConsumer; +import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder; +import org.apache.ignite.internal.binarytuple.BinaryTupleReader; +import org.apache.ignite.sql.ColumnType; +import org.apache.ignite.table.Tuple; +import org.jetbrains.annotations.Nullable; + +/** Tuple with schema marshalling. */ +public final class TupleWithSchemaMarshalling { + private static final ByteOrder BYTE_ORDER = ByteOrder.LITTLE_ENDIAN; + + /** + * Marshal tuple is the following format (LITTLE_ENDIAN): + * marshalledTuple := | size | valuePosition | binaryTupleWithSchema | + * size := int32 + * valuePosition := int32 + * binaryTupleWithSchema := | schemaBinaryTuple | valueBinaryTuple | + * schemaBinaryTuple := | column1 | ... | columnN | + * column := | columnName | columnType | + * columnName := string + * columnType := int32 + * valueBinaryTuple := | value1 | ... | valueN |. + */ + public static byte @Nullable [] marshal(Tuple tup) { + // Allocate all the memory we need upfront. + int size = tup.columnCount(); + Object[] values = new Object[size]; + String[] columns = new String[size]; + ColumnType[] types = new ColumnType[size]; + + // Fill in the values, column names, and types. + for (int i = 0; i < size; i++) { + var value = tup.value(i); + values[i] = value; + columns[i] = tup.columnName(i); + types[i] = inferType(value); + } + + BinaryTupleBuilder schemaBuilder = schemaBuilder(columns, types); + BinaryTupleBuilder valueBuilder = valueBuilder(values, types); + + byte[] schemaBt = schemaBuilder.build().array(); + byte[] valueBt = valueBuilder.build().array(); + // Size: int32 (tuple size), int32 (value offset), schema, value. + byte[] result = new byte[4 + 4 + schemaBt.length + valueBt.length]; + ByteBuffer buff = ByteBuffer.wrap(result).order(BYTE_ORDER); + + // Put the size of the schema in the first 4 bytes. + buff.putInt(0, size); + + // Put the value offset in the second 4 bytes. + int offset = schemaBt.length + 8; + buff.putInt(4, offset); + + System.arraycopy(schemaBt, 0, result, 8, schemaBt.length); + System.arraycopy(valueBt, 0, result, schemaBt.length + 8, valueBt.length); + + return result; + } + + /** + * Unmarshal tuple (LITTLE_ENDIAN). + * + * @param raw byte[] bytes that are marshaled by {@link #marshal(Tuple)}. + */ + public static @Nullable Tuple unmarshal(byte @Nullable [] raw) { + // Read first int32. + ByteBuffer buff = ByteBuffer.wrap(raw).order(BYTE_ORDER); + int size = buff.getInt(0); + + // Read second int32. + int valueOffset = buff.getInt(4); + + ByteBuffer schemaBuff = buff + .position(8).limit(valueOffset) + .slice().order(BYTE_ORDER); + ByteBuffer valueBuff = buff + .position(valueOffset).limit(raw.length) + .slice().order(BYTE_ORDER); + + BinaryTupleReader schemaReader = new BinaryTupleReader(size * 2, schemaBuff); + BinaryTupleReader valueReader = new BinaryTupleReader(size, valueBuff); + + String[] columns = new String[size]; + ColumnType[] types = new ColumnType[size]; + Tuple tup = Tuple.create(size); + + int readerInd = 0; + int i = 0; + while (i < size) { + String colName = schemaReader.stringValue(readerInd++); + int colTypeId = schemaReader.intValue(readerInd++); + + columns[i] = colName; + types[i] = ColumnType.getById(colTypeId); + + i += 1; + } + + int k = 0; + while (k < size) { + setColumnValue(valueReader, tup, columns[k], types[k].id(), k); + k += 1; + } + + return tup; + } + + private static BinaryTupleBuilder schemaBuilder(String[] columns, ColumnType[] types) { + BinaryTupleBuilder builder = new BinaryTupleBuilder(columns.length * 2, columns.length * 2); + + for (int i = 0; i < columns.length; i++) { + builder.appendString(columns[i]); + builder.appendInt(types[i].id()); + } + + return builder; + } + + private static BinaryTupleBuilder valueBuilder(Object[] values, ColumnType[] types) { + BinaryTupleBuilder builder = new BinaryTupleBuilder(values.length, values.length); + + for (int i = 0; i < values.length; i++) { + ColumnType type = types[i]; + Object v = values[i]; + + appender(type).accept(builder, v); + } + + return builder; + } + + private static BiConsumer<BinaryTupleBuilder, Object> appender(ColumnType type) { + switch (type) { + case NULL: + return (b, v) -> b.appendNull(); + case BOOLEAN: + return (b, v) -> b.appendBoolean(checkType(Boolean.class, v)); + case INT8: + return (b, v) -> b.appendByte(checkType(Byte.class, v)); + case INT16: + return (b, v) -> b.appendShort(checkType(Short.class, v)); + case INT32: + return (b, v) -> b.appendInt(checkType(Integer.class, v)); + case INT64: + return (b, v) -> b.appendLong(checkType(Long.class, v)); + case FLOAT: + return (b, v) -> b.appendFloat(checkType(Float.class, v)); + case DOUBLE: + return (b, v) -> b.appendDouble(checkType(Double.class, v)); + case STRING: + return (b, v) -> b.appendString(checkType(String.class, v)); + case DECIMAL: + return (b, v) -> b.appendDecimal(checkType(BigDecimal.class, v), ((BigDecimal) v).scale()); + case DATE: + return (b, v) -> b.appendDate(checkType(LocalDate.class, v)); + case TIME: + return (b, v) -> b.appendTime(checkType(LocalTime.class, v)); + case DATETIME: + return (b, v) -> b.appendDateTime(checkType(LocalDateTime.class, v)); + case TIMESTAMP: + return (b, v) -> b.appendTimestamp(checkType(Instant.class, v)); + case UUID: + return (b, v) -> b.appendUuid(checkType(UUID.class, v)); + case BYTE_ARRAY: + return (b, v) -> b.appendBytes(checkType(byte[].class, v)); + case PERIOD: + return (b, v) -> b.appendPeriod(checkType(Period.class, v)); + case DURATION: + return (b, v) -> b.appendDuration(checkType(Duration.class, v)); + default: + throw new IllegalArgumentException("Unsupported type: " + type); + } + } + + private static <T> T checkType(Class<T> cl, Object o) { + if (o.getClass().isAssignableFrom(cl)) { + return (T) o; + } + + throw new IllegalArgumentException("Unsupported type: " + o.getClass()); + } Review Comment: See `ClientBinaryTupleUtils.appendValue` - we catch `ClassCastException` and rethrow with more details, this is less code and more efficient than calling `checkType` every time. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
