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

IGNITE-20478 Sql. Get rid of UNSPECIFIED_VALUE_PLACEHOLDER #2671

Merged
merged 13 commits into from
Oct 19, 2023
Merged
Show file tree
Hide file tree
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 @@ -51,7 +51,7 @@ public class ItSecondaryIndexTest extends ClusterPerClassIntegrationTest {
* Before all.
*/
@BeforeAll
static void initTestData() throws InterruptedException {
static void initTestData() {
sql("CREATE TABLE developer (id INT PRIMARY KEY, name VARCHAR, depid INT, city VARCHAR, age INT)");
sql("CREATE INDEX " + DEPID_IDX + " ON developer (depid)");
sql("CREATE INDEX " + NAME_CITY_IDX + " ON developer (name DESC, city DESC)");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.ignite.internal.sql.engine.exec;

import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;

import java.nio.ByteBuffer;
import java.util.List;
import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
Expand All @@ -31,7 +29,6 @@
import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
import org.apache.ignite.internal.sql.engine.util.TypeUtils;
import org.apache.ignite.internal.type.NativeTypeSpec;
import org.apache.ignite.internal.util.IgniteUtils;

/**
* Helper class provides method to convert binary tuple to rows and vice-versa.
Expand Down Expand Up @@ -59,7 +56,7 @@ public static BinaryTupleSchema createIndexRowSchema(List<String> indexedColumns
* @param <RowT> Row type.
* @return Binary tuple.
*/
public static <RowT> BinaryTuplePrefix toBinaryTuplePrefix(
static <RowT> BinaryTuplePrefix toBinaryTuplePrefix(
ExecutionContext<RowT> ectx,
BinaryTupleSchema binarySchema,
RowHandler.RowFactory<RowT> factory,
Expand All @@ -70,20 +67,11 @@ public static <RowT> BinaryTuplePrefix toBinaryTuplePrefix(
int indexedColumnsCount = binarySchema.elementCount();
int prefixColumnsCount = handler.columnCount(searchRow);

assert prefixColumnsCount == indexedColumnsCount : "Invalid range condition";

int specifiedCols = 0;
for (int i = 0; i < prefixColumnsCount; i++) {
if (handler.get(i, searchRow) == UNSPECIFIED_VALUE_PLACEHOLDER) {
break;
}

specifiedCols++;
}
assert binarySchema.elementCount() >= prefixColumnsCount : "Invalid range condition";

BinaryTuplePrefixBuilder tupleBuilder = new BinaryTuplePrefixBuilder(specifiedCols, indexedColumnsCount);
BinaryTuplePrefixBuilder tupleBuilder = new BinaryTuplePrefixBuilder(prefixColumnsCount, indexedColumnsCount);

return new BinaryTuplePrefix(indexedColumnsCount, toByteBuffer(ectx, binarySchema, handler, tupleBuilder, searchRow));
return new BinaryTuplePrefix(indexedColumnsCount, toByteBuffer(binarySchema, handler, tupleBuilder, searchRow));
}

/**
Expand All @@ -108,21 +96,12 @@ public static <RowT> BinaryTuple toBinaryTuple(

assert rowColumnsCount == binarySchema.elementCount() : "Invalid lookup key.";

if (IgniteUtils.assertionsEnabled()) {
for (int i = 0; i < rowColumnsCount; i++) {
if (handler.get(i, searchRow) == UNSPECIFIED_VALUE_PLACEHOLDER) {
throw new AssertionError("Invalid lookup key.");
}
}
}

BinaryTupleBuilder tupleBuilder = new BinaryTupleBuilder(rowColumnsCount);

return new BinaryTuple(rowColumnsCount, toByteBuffer(ectx, binarySchema, handler, tupleBuilder, searchRow));
return new BinaryTuple(rowColumnsCount, toByteBuffer(binarySchema, handler, tupleBuilder, searchRow));
}

private static <RowT> ByteBuffer toByteBuffer(
ExecutionContext<RowT> ectx,
BinaryTupleSchema binarySchema,
RowHandler<RowT> handler,
BinaryTupleBuilder tupleBuilder,
Expand All @@ -133,10 +112,6 @@ private static <RowT> ByteBuffer toByteBuffer(
for (int i = 0; i < columnsCount; i++) {
Object val = handler.get(i, searchRow);

if (val == UNSPECIFIED_VALUE_PLACEHOLDER) {
break; // No more columns in prefix.
}

Element element = binarySchema.element(i);

val = TypeUtils.fromInternal(val, NativeTypeSpec.toClass(element.typeSpec(), element.nullable()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -183,18 +183,20 @@ public <RowT> Publisher<RowT> indexLookup(
return new TransformingPublisher<>(pub, item -> rowConverter.toRow(ctx, item, rowFactory));
}

private <RowT> @Nullable BinaryTuplePrefix toBinaryTuplePrefix(ExecutionContext<RowT> ctx,
private static <RowT> @Nullable BinaryTuplePrefix toBinaryTuplePrefix(
ExecutionContext<RowT> ctx,
BinaryTupleSchema indexRowSchema,
@Nullable RowT condition, RowFactory<RowT> factory) {

@Nullable RowT condition,
RowFactory<RowT> factory
) {
if (condition == null) {
return null;
}

return RowConverter.toBinaryTuplePrefix(ctx, indexRowSchema, factory, condition);
}

private <RowT> @Nullable BinaryTuple toBinaryTuple(ExecutionContext<RowT> ctx, BinaryTupleSchema indexRowSchema,
private static <RowT> @Nullable BinaryTuple toBinaryTuple(ExecutionContext<RowT> ctx, BinaryTupleSchema indexRowSchema,
@Nullable RowT condition, RowFactory<RowT> factory) {
if (condition == null) {
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.ignite.internal.sql.engine.exec;

import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;

import java.math.BigDecimal;
import java.math.BigInteger;
Expand Down Expand Up @@ -233,8 +232,6 @@ ByteBuffer toByteBuffer() {
for (int i = 0; i < row.length; i++) {
Object value = row[i];

assert value != UNSPECIFIED_VALUE_PLACEHOLDER : "Invalid row value.";

appendValue(tupleBuilder, rowSchema.fields().get(i), value);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,14 +24,14 @@
import java.lang.reflect.Type;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.BitSet;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentMap;
import java.util.function.BiPredicate;
import java.util.function.Function;
Expand Down Expand Up @@ -93,14 +93,6 @@
public class ExpressionFactoryImpl<RowT> implements ExpressionFactory<RowT> {
private static final int CACHE_SIZE = 1024;

/** Placeholder for values, which expressions are not specified. */
public static final Object UNSPECIFIED_VALUE_PLACEHOLDER = Placeholder.UNSPECIFIED_VALUE;

// We use enums for placeholders because enum serialization/deserialization guarantees to preserve object's identity.
private enum Placeholder {
UNSPECIFIED_VALUE
}

private static final ConcurrentMap<String, Scalar> SCALAR_CACHE = Caffeine.newBuilder()
.maximumSize(CACHE_SIZE)
.<String, Scalar>build()
Expand Down Expand Up @@ -155,7 +147,6 @@ public Comparator<RowT> comparator(RelCollation collation) {

return (o1, o2) -> {
RowHandler<RowT> hnd = ctx.rowHandler();
Object unspecifiedVal = UNSPECIFIED_VALUE_PLACEHOLDER;

for (RelFieldCollation field : collation.getFieldCollations()) {
int fieldIdx = field.getFieldIndex();
Expand All @@ -164,11 +155,6 @@ public Comparator<RowT> comparator(RelCollation collation) {
Object c1 = hnd.get(fieldIdx, o1);
Object c2 = hnd.get(fieldIdx, o2);

// If filter for some field is unspecified, assume equality for this field and all subsequent fields.
if (c1 == unspecifiedVal || c2 == unspecifiedVal) {
return 0;
}

int res = (field.direction == RelFieldCollation.Direction.ASCENDING)
?
compare(c1, c2, nullComparison) :
Expand Down Expand Up @@ -354,6 +340,30 @@ public RangeIterable<RowT> ranges(
return new RangeIterableImpl(ranges, comparator);
}

/**
* Transforms input bound, stores only sequential non null elements.
* i.e. (literal1, literal2, null, literal3) -> (literal1, literal2).
* Return transformed bound and appropriate type.
*/
private static Map.Entry<List<RexNode>, RelDataType> shrinkBounds(IgniteTypeFactory factory, List<RexNode> bound, RelDataType rowType) {
List<RexNode> newBound = new ArrayList<>();
List<RelDataType> newTypes = new ArrayList<>();
List<RelDataType> types = RelOptUtil.getFieldTypeList(rowType);
int i = 0;
for (RexNode node : bound) {
if (node != null) {
newTypes.add(types.get(i++));
newBound.add(node);
} else {
break;
}
}
bound = Collections.unmodifiableList(newBound);
rowType = TypeUtils.createRowType(factory, newTypes);

return Map.entry(bound, rowType);
}

/**
* Expand column-oriented {@link SearchBounds} to a row-oriented list of ranges ({@link RangeCondition}).
*
Expand Down Expand Up @@ -381,12 +391,37 @@ private void expandBounds(
if ((fieldIdx >= searchBounds.size())
|| (!lowerInclude && !upperInclude)
|| searchBounds.get(fieldIdx) == null) {
RelDataType lowerType = rowType;
RelDataType upperType = rowType;

RowFactory<RowT> lowerFactory = rowFactory;
RowFactory<RowT> upperFactory = rowFactory;

// we need to shrink bounds here due to the recursive logic for processing lower and upper bounds,
// after division this logic into upper and lower calculation such approach need to be removed.
Entry<List<RexNode>, RelDataType> res = shrinkBounds(ctx.getTypeFactory(), curLower, rowType);
curLower = res.getKey();
lowerType = res.getValue();

res = shrinkBounds(ctx.getTypeFactory(), curUpper, rowType);
curUpper = res.getKey();
upperType = res.getValue();

lowerFactory = ctx.rowHandler()
.factory(TypeUtils.rowSchemaFromRelTypes(RelOptUtil.getFieldTypeList(lowerType)));

upperFactory = ctx.rowHandler()
.factory(TypeUtils.rowSchemaFromRelTypes(RelOptUtil.getFieldTypeList(upperType)));

ranges.add(new RangeConditionImpl(
scalar(curLower, rowType),
scalar(curUpper, rowType),
curLower,
curUpper,
lowerInclude,
upperInclude,
rowFactory
lowerType,
upperType,
lowerFactory,
upperFactory
));

return;
Expand Down Expand Up @@ -457,7 +492,7 @@ private SingleScalar scalar(RexNode node, RelDataType type) {
/**
* Creates {@link SingleScalar}, a code-generated expressions evaluator.
*
* @param nodes Expressions. {@code Null} expressions will be evaluated to {@link ExpressionFactoryImpl#UNSPECIFIED_VALUE_PLACEHOLDER}.
* @param nodes Expressions.
* @param type Row type.
* @return SingleScalar.
*/
Expand Down Expand Up @@ -487,19 +522,10 @@ private Scalar compile(List<RexNode> nodes, RelDataType type, boolean biInParams

RexProgramBuilder programBuilder = new RexProgramBuilder(type, rexBuilder);

BitSet unspecifiedValues = new BitSet(nodes.size());

for (int i = 0; i < nodes.size(); i++) {
RexNode node = nodes.get(i);

if (node != null) {
programBuilder.addProject(node, null);
} else {
unspecifiedValues.set(i);
for (RexNode node : nodes) {
assert node != null : "unexpected nullable node";

programBuilder.addProject(rexBuilder.makeNullLiteral(type == emptyType
? nullType : type.getFieldList().get(i).getType()), null);
}
programBuilder.addProject(node, null);
}

RexProgram program = programBuilder.getProgram();
Expand Down Expand Up @@ -533,12 +559,8 @@ private Scalar compile(List<RexNode> nodes, RelDataType type, boolean biInParams
List<Expression> projects = RexToLixTranslator.translateProjects(program, typeFactory, conformance,
builder, null, null, ctx, inputGetter, correlates);

assert nodes.size() == projects.size();

for (int i = 0; i < projects.size(); i++) {
Expression val = unspecifiedValues.get(i)
? Expressions.field(null, ExpressionFactoryImpl.class, "UNSPECIFIED_VALUE_PLACEHOLDER")
: projects.get(i);
Expression val = projects.get(i);

builder.add(
Expressions.statement(
Expand Down Expand Up @@ -765,36 +787,44 @@ private class RangeConditionImpl implements RangeCondition<RowT> {
/** Upper row. */
private @Nullable RowT upperRow;

/** Cached skip range flag. */
private Boolean skip;
/** Lower bound row factory. */
private final RowFactory<RowT> lowerFactory;

/** Row factory. */
private final RowFactory<RowT> factory;
/** Upper bound row factory. */
private final RowFactory<RowT> upperFactory;

/** Cached skip range flag. */
private @Nullable Boolean skip;

private RangeConditionImpl(
SingleScalar lowerBound,
SingleScalar upperBound,
List<RexNode> lower,
List<RexNode> upper,
boolean lowerInclude,
boolean upperInclude,
RowFactory<RowT> factory
RelDataType rowTypeLower,
RelDataType rowTypeUpper,
RowFactory<RowT> lowerFactory,
RowFactory<RowT> upperFactory
) {
this.lowerBound = lowerBound;
this.upperBound = upperBound;
this.lowerBound = scalar(lower, rowTypeLower);
this.upperBound = scalar(upper, rowTypeUpper);
this.lowerInclude = lowerInclude;
this.upperInclude = upperInclude;
this.factory = factory;

this.lowerFactory = lowerFactory;
this.upperFactory = upperFactory;
}

/** {@inheritDoc} */
@Override
public RowT lower() {
return lowerRow != null ? lowerRow : (lowerRow = getRow(lowerBound));
public @Nullable RowT lower() {
return lowerRow != null ? lowerRow : (lowerRow = getRow(lowerBound, lowerFactory));
}

/** {@inheritDoc} */
@Override
public RowT upper() {
return upperRow != null ? upperRow : (upperRow = getRow(upperBound));
public @Nullable RowT upper() {
return upperRow != null ? upperRow : (upperRow = getRow(upperBound, upperFactory));
}

/** {@inheritDoc} */
Expand All @@ -810,7 +840,7 @@ public boolean upperInclude() {
}

/** Compute row. */
private RowT getRow(SingleScalar scalar) {
private RowT getRow(SingleScalar scalar, RowFactory<RowT> factory) {
RowT res = factory.create();
scalar.execute(ctx, null, res);

Expand All @@ -835,7 +865,8 @@ private RowT getRow(SingleScalar scalar) {

/** Clear cached rows. */
void clearCache() {
lowerRow = upperRow = null;
lowerRow = null;
upperRow = null;
skip = null;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,9 +29,9 @@ public interface RangeCondition<RowT> {
/** Upper search row. */
RowT upper();

/** Inlusive search by lower row. */
/** Inclusive search by lower row. */
boolean lowerInclude();

/** Inlusive search by upper row. */
/** Inclusive search by upper row. */
boolean upperInclude();
}
Loading