Skip to content

Commit

Permalink
Rename SingleRowBlock to SqlRow
Browse files Browse the repository at this point in the history
  • Loading branch information
dain committed Oct 12, 2023
1 parent 4251f01 commit 1703cb3
Show file tree
Hide file tree
Showing 22 changed files with 113 additions and 105 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
import io.trino.cost.ComposableStatsCalculator.Rule;
import io.trino.matching.Pattern;
import io.trino.security.AllowAllAccessControl;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.type.RowType;
import io.trino.spi.type.Type;
import io.trino.sql.PlannerContext;
Expand Down Expand Up @@ -96,7 +96,7 @@ private List<Object> getSymbolValues(ValuesNode valuesNode, int symbolId, Sessio
return valuesNode.getRows().get().stream()
.map(row -> {
Object rowValue = evaluateConstantExpression(row, rowType, plannerContext, session, new AllowAllAccessControl(), ImmutableMap.of());
return readNativeValue(symbolType, (SingleRowBlock) rowValue, symbolId);
return readNativeValue(symbolType, (SqlRow) rowValue, symbolId);
})
.collect(toList());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import io.trino.spi.block.RowBlockEncoding;
import io.trino.spi.block.RunLengthBlockEncoding;
import io.trino.spi.block.ShortArrayBlockEncoding;
import io.trino.spi.block.SingleRowBlockEncoding;
import io.trino.spi.block.SqlRowBlockEncoding;
import io.trino.spi.block.VariableWidthBlockEncoding;

import java.util.Map;
Expand All @@ -53,7 +53,7 @@ public BlockEncodingManager()
addBlockEncoding(new ArrayBlockEncoding());
addBlockEncoding(new MapBlockEncoding());
addBlockEncoding(new RowBlockEncoding());
addBlockEncoding(new SingleRowBlockEncoding());
addBlockEncoding(new SqlRowBlockEncoding());
addBlockEncoding(new RunLengthBlockEncoding());
addBlockEncoding(new LazyBlockEncoding());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import io.trino.spi.block.ArrayBlockBuilder;
import io.trino.spi.block.Block;
import io.trino.spi.block.RowBlockBuilder;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.block.VariableWidthBlockBuilder;
import io.trino.spi.type.ArrayType;

Expand Down Expand Up @@ -221,10 +221,10 @@ private void writeValue(byte[] records, int recordOffset, VariableWidthBlockBuil
@Override
public void merge(ListaggAggregationState other)
{
SingleRowBlock serializedState = ((SingleListaggAggregationState) other).removeTempSerializedState();
SqlRow sqlRow = ((SingleListaggAggregationState) other).removeTempSerializedState();

List<Block> fields = serializedState.getChildren();
int index = serializedState.getRowIndex();
List<Block> fields = sqlRow.getChildren();
int index = sqlRow.getRowIndex();
Slice separator = VARCHAR.getSlice(fields.get(0), index);
boolean overflowError = BOOLEAN.getBoolean(fields.get(1), index);
Slice overflowFiller = VARCHAR.getSlice(fields.get(2), index);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.block.RowBlockBuilder;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.function.AccumulatorStateSerializer;
import io.trino.spi.type.ArrayType;
import io.trino.spi.type.RowType;
Expand Down Expand Up @@ -51,7 +51,7 @@ public void serialize(ListaggAggregationState state, BlockBuilder out)
@Override
public void deserialize(Block block, int index, ListaggAggregationState state)
{
SingleRowBlock mapBlock = (SingleRowBlock) serializedType.getObject(block, index);
((SingleListaggAggregationState) state).setTempSerializedState(mapBlock);
SqlRow sqlRow = (SqlRow) serializedType.getObject(block, index);
((SingleListaggAggregationState) state).setTempSerializedState(sqlRow);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
package io.trino.operator.aggregation.listagg;

import io.airlift.slice.SliceOutput;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.block.VariableWidthBlockBuilder;
import io.trino.spi.function.AccumulatorState;

Expand All @@ -25,7 +25,7 @@
public class SingleListaggAggregationState
extends AbstractListaggAggregationState
{
private SingleRowBlock tempSerializedState;
private SqlRow tempSerializedState;

public SingleListaggAggregationState()
{
Expand Down Expand Up @@ -80,16 +80,16 @@ public AccumulatorState copy()
return new SingleListaggAggregationState(this);
}

void setTempSerializedState(SingleRowBlock tempSerializedState)
void setTempSerializedState(SqlRow tempSerializedState)
{
this.tempSerializedState = tempSerializedState;
}

SingleRowBlock removeTempSerializedState()
SqlRow removeTempSerializedState()
{
SingleRowBlock block = tempSerializedState;
checkState(block != null, "tempDeserializeBlock is null");
SqlRow sqlRow = tempSerializedState;
checkState(sqlRow != null, "tempDeserializeBlock is null");
tempSerializedState = null;
return block;
return sqlRow;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.block.RowBlockBuilder;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.function.AccumulatorState;
import io.trino.spi.function.GroupedAccumulatorState;
import io.trino.spi.type.ArrayType;
Expand All @@ -41,14 +41,14 @@ private abstract static class AbstractMinMaxByNState
@Override
public final void merge(MinMaxByNState other)
{
SingleRowBlock serializedState = ((SingleMinMaxByNState) other).removeTempSerializedState();
SqlRow sqlRow = ((SingleMinMaxByNState) other).removeTempSerializedState();

int capacity = toIntExact(BIGINT.getLong(serializedState, 0));
int capacity = toIntExact(BIGINT.getLong(sqlRow, 0));
initialize(capacity);
TypedKeyValueHeap typedKeyValueHeap = getTypedKeyValueHeap();

Block keys = new ArrayType(typedKeyValueHeap.getKeyType()).getObject(serializedState, 1);
Block values = new ArrayType(typedKeyValueHeap.getValueType()).getObject(serializedState, 2);
Block keys = new ArrayType(typedKeyValueHeap.getKeyType()).getObject(sqlRow, 1);
Block values = new ArrayType(typedKeyValueHeap.getValueType()).getObject(sqlRow, 2);
typedKeyValueHeap.addAll(keys, values);
}

Expand Down Expand Up @@ -160,7 +160,7 @@ public abstract static class SingleMinMaxByNState
private final LongFunction<TypedKeyValueHeap> heapFactory;

private TypedKeyValueHeap typedHeap;
private SingleRowBlock tempSerializedState;
private SqlRow tempSerializedState;

public SingleMinMaxByNState(LongFunction<TypedKeyValueHeap> heapFactory)
{
Expand Down Expand Up @@ -224,17 +224,17 @@ final TypedKeyValueHeap getTypedKeyValueHeap()
return typedHeap;
}

void setTempSerializedState(SingleRowBlock tempSerializedState)
void setTempSerializedState(SqlRow tempSerializedState)
{
this.tempSerializedState = tempSerializedState;
}

SingleRowBlock removeTempSerializedState()
SqlRow removeTempSerializedState()
{
SingleRowBlock block = tempSerializedState;
checkState(block != null, "tempDeserializeBlock is null");
SqlRow sqlRow = tempSerializedState;
checkState(sqlRow != null, "tempDeserializeBlock is null");
tempSerializedState = null;
return block;
return sqlRow;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@

import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.function.AccumulatorStateSerializer;
import io.trino.spi.type.Type;

Expand Down Expand Up @@ -44,7 +44,7 @@ public void serialize(T state, BlockBuilder out)
@Override
public void deserialize(Block block, int index, T state)
{
SingleRowBlock rowBlock = (SingleRowBlock) serializedType.getObject(block, index);
((MinMaxByNStateFactory.SingleMinMaxByNState) state).setTempSerializedState(rowBlock);
SqlRow sqlRow = (SqlRow) serializedType.getObject(block, index);
((MinMaxByNStateFactory.SingleMinMaxByNState) state).setTempSerializedState(sqlRow);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.block.RowBlockBuilder;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.function.AccumulatorState;
import io.trino.spi.function.GroupedAccumulatorState;
import io.trino.spi.type.ArrayType;
Expand All @@ -44,13 +44,13 @@ private abstract static class AbstractMinMaxNState
@Override
public final void merge(MinMaxNState other)
{
SingleRowBlock serializedState = ((SingleMinMaxNState) other).removeTempSerializedState();
SqlRow sqlRow = ((SingleMinMaxNState) other).removeTempSerializedState();

int capacity = toIntExact(BIGINT.getLong(serializedState, 0));
int capacity = toIntExact(BIGINT.getLong(sqlRow, 0));
initialize(capacity);
TypedHeap typedHeap = getTypedHeap();

Block values = new ArrayType(typedHeap.getElementType()).getObject(serializedState, 1);
Block values = new ArrayType(typedHeap.getElementType()).getObject(sqlRow, 1);
typedHeap.addAll(values);
}

Expand Down Expand Up @@ -158,7 +158,7 @@ public abstract static class SingleMinMaxNState
private final LongFunction<TypedHeap> heapFactory;

private TypedHeap typedHeap;
private SingleRowBlock tempSerializedState;
private SqlRow tempSerializedState;

public SingleMinMaxNState(LongFunction<TypedHeap> heapFactory)
{
Expand Down Expand Up @@ -221,17 +221,17 @@ final TypedHeap getTypedHeap()
return typedHeap;
}

void setTempSerializedState(SingleRowBlock tempSerializedState)
void setTempSerializedState(SqlRow tempSerializedState)
{
this.tempSerializedState = tempSerializedState;
}

SingleRowBlock removeTempSerializedState()
SqlRow removeTempSerializedState()
{
SingleRowBlock block = tempSerializedState;
checkState(block != null, "tempDeserializeBlock is null");
SqlRow sqlRow = tempSerializedState;
checkState(sqlRow != null, "tempDeserializeBlock is null");
tempSerializedState = null;
return block;
return sqlRow;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
import io.trino.operator.aggregation.minmaxn.MinMaxNStateFactory.SingleMinMaxNState;
import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.function.AccumulatorStateSerializer;
import io.trino.spi.type.Type;

Expand Down Expand Up @@ -50,7 +50,7 @@ public void deserialize(Block block, int index, T state)
// the aggregation framework uses a scratch single state for deserialization, and then calls the combine function
// for typed heap is is simpler to store the deserialized row block in the state and then add the row block
// directly to the heap in the combine
SingleRowBlock rowBlock = (SingleRowBlock) serializedType.getObject(block, index);
((SingleMinMaxNState) state).setTempSerializedState(rowBlock);
SqlRow sqlRow = (SqlRow) serializedType.getObject(block, index);
((SingleMinMaxNState) state).setTempSerializedState(sqlRow);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
import io.trino.Session;
import io.trino.metadata.Metadata;
import io.trino.spi.block.Block;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.connector.ColumnHandle;
import io.trino.spi.predicate.Domain;
import io.trino.spi.predicate.TupleDomain;
Expand Down Expand Up @@ -413,7 +413,7 @@ public Expression visitValues(ValuesNode node, Void context)
}
for (int i = 0; i < node.getOutputSymbols().size(); i++) {
Type type = types.get(node.getOutputSymbols().get(i));
Object item = readNativeValue(type, (SingleRowBlock) evaluated, i);
Object item = readNativeValue(type, (SqlRow) evaluated, i);
if (item == null) {
hasNull[i] = true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
import io.trino.spi.TrinoException;
import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.connector.ConnectorSession;
import io.trino.spi.function.CatalogSchemaFunctionName;
import io.trino.spi.function.FunctionNullability;
Expand Down Expand Up @@ -1549,7 +1549,7 @@ protected Object visitSubscriptExpression(SubscriptExpression node, Object conte
}

// Subscript on Row hasn't got a dedicated operator. It is interpreted by hand.
if (base instanceof SingleRowBlock row) {
if (base instanceof SqlRow row) {
int position = toIntExact((long) index - 1);
if (position < 0 || position >= row.getPositionCount()) {
throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "ROW index out of bounds: " + (position + 1));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@
import io.trino.spi.Page;
import io.trino.spi.PageBuilder;
import io.trino.spi.TrinoException;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.connector.ColumnHandle;
import io.trino.spi.connector.ConnectorIndex;
import io.trino.spi.connector.ConnectorSession;
Expand Down Expand Up @@ -2207,7 +2207,7 @@ public PhysicalOperation visitValues(ValuesNode node, LocalExecutionPlanContext
Object result = new ExpressionInterpreter(row, plannerContext, session, types).evaluate();
for (int j = 0; j < outputTypes.size(); j++) {
// divide row into fields
writeNativeValue(outputTypes.get(j), pageBuilder.getBlockBuilder(j), readNativeValue(outputTypes.get(j), (SingleRowBlock) result, j));
writeNativeValue(outputTypes.get(j), pageBuilder.getBlockBuilder(j), readNativeValue(outputTypes.get(j), (SqlRow) result, j));
}
}
}
Expand Down
12 changes: 6 additions & 6 deletions core/trino-main/src/test/java/io/trino/block/TestRowBlock.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.block.ByteArrayBlock;
import io.trino.spi.block.RowBlockBuilder;
import io.trino.spi.block.SingleRowBlock;
import io.trino.spi.block.SqlRow;
import io.trino.spi.type.Type;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -177,20 +177,20 @@ private void assertValue(Block rowBlock, int position, List<?> row)
requireNonNull(row, "row is null");

assertFalse(rowBlock.isNull(position));
SingleRowBlock singleRowBlock = (SingleRowBlock) rowBlock.getObject(position, Block.class);
assertEquals(singleRowBlock.getPositionCount(), row.size());
SqlRow sqlRow = (SqlRow) rowBlock.getObject(position, Block.class);
assertEquals(sqlRow.getPositionCount(), row.size());

for (int i = 0; i < row.size(); i++) {
Object fieldValue = row.get(i);
if (fieldValue == null) {
assertTrue(singleRowBlock.isNull(i));
assertTrue(sqlRow.isNull(i));
}
else {
if (fieldValue instanceof Long) {
assertEquals(BIGINT.getLong(singleRowBlock, i), ((Long) fieldValue).longValue());
assertEquals(BIGINT.getLong(sqlRow, i), ((Long) fieldValue).longValue());
}
else if (fieldValue instanceof String) {
assertEquals(VARCHAR.getSlice(singleRowBlock, i), utf8Slice((String) fieldValue));
assertEquals(VARCHAR.getSlice(sqlRow, i), utf8Slice((String) fieldValue));
}
else {
throw new IllegalArgumentException();
Expand Down
8 changes: 8 additions & 0 deletions core/trino-spi/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -418,6 +418,14 @@
<old>method io.trino.spi.block.Block io.trino.spi.type.MapType::getObject(io.trino.spi.block.Block, int)</old>
<new>method io.trino.spi.block.SqlMap io.trino.spi.type.MapType::getObject(io.trino.spi.block.Block, int)</new>
</item>
<item>
<code>java.class.removed</code>
<old>class io.trino.spi.block.SingleRowBlock</old>
</item>
<item>
<code>java.class.removed</code>
<old>class io.trino.spi.block.SingleRowBlockEncoding</old>
</item>
</differences>
</revapi.differences>
</analysisConfiguration>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ public <E extends Throwable> Block build(RowValueBuilder<E> builder)
List<Block> blocks = fieldBuilders.stream()
.map(field -> field.build().getRegion(startSize, 1))
.toList();
return new SingleRowBlock(0, blocks.toArray(new Block[0]));
return new SqlRow(0, blocks.toArray(new Block[0]));
}

private boolean equalizeBlockBuilders()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -535,7 +535,7 @@ public <T> T getObject(int position, Class<T> clazz)
}
checkReadablePosition(this, position);

return clazz.cast(new SingleRowBlock(getFieldBlockOffset(position), fieldBlocks));
return clazz.cast(new SqlRow(getFieldBlockOffset(position), fieldBlocks));
}

@Override
Expand Down
Loading

0 comments on commit 1703cb3

Please sign in to comment.