Skip to content
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 @@ -42,7 +42,6 @@
import io.trino.spi.predicate.TupleDomain;

import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Optional;
Expand Down Expand Up @@ -338,9 +337,8 @@ private <T> Optional<Set<String>> filterString(TupleDomain<T> constraint, T colu
if (domain.isSingleValue()) {
return Optional.of(ImmutableSet.of(((Slice) domain.getSingleValue()).toStringUtf8()));
}
if (domain.getValues() instanceof EquatableValueSet) {
Collection<Object> values = ((EquatableValueSet) domain.getValues()).getValues();
return Optional.of(values.stream()
if (domain.getValues() instanceof EquatableValueSet set) {
return Optional.of(set.getValues().stream()
.map(Slice.class::cast)
.map(Slice::toStringUtf8)
.collect(toImmutableSet()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -150,8 +150,8 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C
}

SystemTable systemTable = checkAndGetTable(session, table);
if (systemTable instanceof JdbcTable) {
TupleDomain<ColumnHandle> filtered = ((JdbcTable) systemTable).applyFilter(session, effectiveConstraint(oldDomain, constraint, newDomain));
if (systemTable instanceof JdbcTable jdbcTable) {
TupleDomain<ColumnHandle> filtered = jdbcTable.applyFilter(session, effectiveConstraint(oldDomain, constraint, newDomain));
newDomain = newDomain.intersect(filtered);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,8 +63,8 @@ public PlanCostEstimate getCost(PlanNode node)
requireNonNull(node, "node is null");

try {
if (node instanceof GroupReference) {
return getGroupCost((GroupReference) node);
if (node instanceof GroupReference group) {
return getGroupCost(group);
}

PlanCostEstimate cost = cache.get(node);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,8 +75,8 @@ public PlanNodeStatsEstimate getStats(PlanNode node)
requireNonNull(node, "node is null");

try {
if (node instanceof GroupReference) {
return getGroupStats((GroupReference) node);
if (node instanceof GroupReference group) {
return getGroupStats(group);
}

PlanNodeStatsEstimate stats = cache.get(node);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,10 @@ public ComposableStatsCalculator(List<Rule<?>> rules)
{
this.rulesByRootType = rules.stream()
.peek(rule -> {
checkArgument(rule.getPattern() instanceof TypeOfPattern, "Rule pattern must be TypeOfPattern");
Class<?> expectedClass = ((TypeOfPattern<?>) rule.getPattern()).expectedClass();
if (!(rule.getPattern() instanceof TypeOfPattern pattern)) {
throw new IllegalArgumentException("Rule pattern must be TypeOfPattern but was: " + rule.getPattern().getClass().getSimpleName());
}
Class<?> expectedClass = pattern.expectedClass();
checkArgument(!expectedClass.isInterface() && !Modifier.isAbstract(expectedClass.getModifiers()), "Rule must be registered on a concrete class");
})
.collect(toMultimap(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,13 +67,13 @@ protected Optional<PlanNodeStatsEstimate> doCalculate(FilterNode node, Context c
return Optional.empty();
}
PlanNode projectNodeSource = context.lookup().resolve(projectNode.getSource());
if (!(projectNodeSource instanceof AggregationNode)) {
if (!(projectNodeSource instanceof AggregationNode value)) {
return Optional.empty();
}
aggregationNode = (AggregationNode) projectNodeSource;
aggregationNode = value;
}
else if (nodeSource instanceof AggregationNode) {
aggregationNode = (AggregationNode) nodeSource;
else if (nodeSource instanceof AggregationNode value) {
aggregationNode = value;
}
else {
return Optional.empty();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,8 +98,8 @@ private double getOutputSizeForSymbol(SymbolStatsEstimate symbolStatistics, Type
// account for "is null" boolean array
outputSize += outputRowCount;

if (type instanceof FixedWidthType) {
outputSize += numberOfNonNullRows * ((FixedWidthType) type).getFixedSize();
if (type instanceof FixedWidthType fixedType) {
outputSize += numberOfNonNullRows * fixedType.getFixedSize();
}
else {
double averageRowSize = firstNonNaN(symbolStatistics.getAverageRowSize(), DEFAULT_DATA_SIZE_PER_COLUMN);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,8 +116,8 @@ private PlanNodeStatsEstimate adjustStats(
double nullsFraction = firstNonNaN(symbolStatistics.getNullsFraction(), 0d);
double numberOfNonNullRows = runtimeStats.outputRowCountEstimate() * (1.0 - nullsFraction);

if (type instanceof FixedWidthType) {
fixedWidthTypeSize += numberOfNonNullRows * ((FixedWidthType) type).getFixedSize();
if (type instanceof FixedWidthType fixedType) {
fixedWidthTypeSize += numberOfNonNullRows * fixedType.getFixedSize();
}
else {
variableTypeValuesCount += numberOfNonNullRows;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,8 +168,8 @@ private boolean isIntegralType(Type type)
return true;
}

if (type instanceof DecimalType) {
return ((DecimalType) type).getScale() == 0;
if (type instanceof DecimalType decimalType) {
return decimalType.getScale() == 0;
}

return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,13 +71,13 @@ protected Optional<PlanNodeStatsEstimate> doCalculate(FilterNode node, Context c
return Optional.empty();
}
PlanNode projectNodeSource = context.lookup().resolve(projectNode.getSource());
if (!(projectNodeSource instanceof SemiJoinNode)) {
if (!(projectNodeSource instanceof SemiJoinNode value)) {
return Optional.empty();
}
semiJoinNode = (SemiJoinNode) projectNodeSource;
semiJoinNode = value;
}
else if (nodeSource instanceof SemiJoinNode) {
semiJoinNode = (SemiJoinNode) nodeSource;
else if (nodeSource instanceof SemiJoinNode value) {
semiJoinNode = value;
}
else {
return Optional.empty();
Expand Down Expand Up @@ -141,8 +141,7 @@ private Optional<SemiJoinOutputFilter> extractSemiJoinOutputFilter(Expression pr
private static boolean isSemiJoinOutputReference(Expression conjunct, Symbol semiJoinOutput)
{
Reference semiJoinOutputSymbolReference = semiJoinOutput.toSymbolReference();
return conjunct.equals(semiJoinOutputSymbolReference) ||
(conjunct instanceof Not && ((Not) conjunct).value().equals(semiJoinOutputSymbolReference));
return conjunct.equals(semiJoinOutputSymbolReference) || (conjunct instanceof Not not && not.value().equals(semiJoinOutputSymbolReference));
}

private static class SemiJoinOutputFilter
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -170,8 +170,8 @@ private double maxDistinctValuesByLowHigh(SymbolStatsEstimate symbolStats, Type
return NaN;
}

if (type instanceof DecimalType) {
length *= pow(10, ((DecimalType) type).getScale());
if (type instanceof DecimalType decimalType) {
length *= pow(10, decimalType.getScale());
}
return floor(length + 1);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -319,7 +319,7 @@ private void safeExecute(Runnable command)
executor.execute(command);
}
catch (RejectedExecutionException e) {
if ((executor instanceof ExecutorService) && ((ExecutorService) executor).isShutdown()) {
if (executor instanceof ExecutorService service && service.isShutdown()) {
throw new TrinoException(SERVER_SHUTTING_DOWN, "Server is shutting down", e);
}
throw e;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,8 @@ public static List<Object> unwrapArrays(List<Object> sequence)
{
return sequence.stream()
.flatMap(object -> {
if (object instanceof JsonNode && ((JsonNode) object).getNodeType() == ARRAY) {
return ImmutableList.copyOf(((JsonNode) object).elements()).stream();
if (object instanceof JsonNode node && node.getNodeType() == ARRAY) {
return ImmutableList.copyOf(node.elements()).stream();
}
return Stream.of(object);
})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -156,9 +156,9 @@ protected List<Object> visitIrAbsMethod(IrAbsMethod node, PathEvaluationContext
ImmutableList.Builder<Object> outputSequence = ImmutableList.builder();
for (Object object : sequence) {
TypedValue value;
if (object instanceof JsonNode) {
value = getNumericTypedValue((JsonNode) object)
.orElseThrow(() -> itemTypeError("NUMBER", ((JsonNode) object).getNodeType().name()));
if (object instanceof JsonNode jsonNode) {
value = getNumericTypedValue(jsonNode)
.orElseThrow(() -> itemTypeError("NUMBER", jsonNode.getNodeType().name()));
}
else {
value = (TypedValue) object;
Expand Down Expand Up @@ -243,8 +243,8 @@ private static TypedValue getAbsoluteValue(TypedValue typedValue)
}
return new TypedValue(type, floatToRawIntBits(Math.abs(value)));
}
if (type instanceof DecimalType) {
if (((DecimalType) type).isShort()) {
if (type instanceof DecimalType decimalType) {
if (decimalType.isShort()) {
long value = typedValue.getLongValue();
if (value > 0) {
return typedValue;
Expand Down Expand Up @@ -285,19 +285,19 @@ protected List<Object> visitIrArithmeticBinary(IrArithmeticBinary node, PathEval

TypedValue left;
Object leftObject = getOnlyElement(leftSequence);
if (leftObject instanceof JsonNode) {
left = getNumericTypedValue((JsonNode) leftObject)
.orElseThrow(() -> itemTypeError("NUMBER", ((JsonNode) leftObject).getNodeType().name()));
if (leftObject instanceof JsonNode jsonNode) {
left = getNumericTypedValue(jsonNode)
.orElseThrow(() -> itemTypeError("NUMBER", jsonNode.getNodeType().name()));
}
else {
left = (TypedValue) leftObject;
}

TypedValue right;
Object rightObject = getOnlyElement(rightSequence);
if (rightObject instanceof JsonNode) {
right = getNumericTypedValue((JsonNode) rightObject)
.orElseThrow(() -> itemTypeError("NUMBER", ((JsonNode) rightObject).getNodeType().name()));
if (rightObject instanceof JsonNode jsonNode) {
right = getNumericTypedValue(jsonNode)
.orElseThrow(() -> itemTypeError("NUMBER", jsonNode.getNodeType().name()));
}
else {
right = (TypedValue) rightObject;
Expand Down Expand Up @@ -351,9 +351,9 @@ protected List<Object> visitIrArithmeticUnary(IrArithmeticUnary node, PathEvalua
ImmutableList.Builder<Object> outputSequence = ImmutableList.builder();
for (Object object : sequence) {
TypedValue value;
if (object instanceof JsonNode) {
value = getNumericTypedValue((JsonNode) object)
.orElseThrow(() -> itemTypeError("NUMBER", ((JsonNode) object).getNodeType().name()));
if (object instanceof JsonNode jsonNode) {
value = getNumericTypedValue(jsonNode)
.orElseThrow(() -> itemTypeError("NUMBER", jsonNode.getNodeType().name()));
}
else {
value = (TypedValue) object;
Expand Down Expand Up @@ -423,8 +423,8 @@ private static TypedValue negate(TypedValue typedValue)
if (type.equals(REAL)) {
return new TypedValue(type, RealOperators.negate(typedValue.getLongValue()));
}
if (type instanceof DecimalType) {
if (((DecimalType) type).isShort()) {
if (type instanceof DecimalType decimalType) {
if (decimalType.isShort()) {
return new TypedValue(type, -typedValue.getLongValue());
}
Int128 negatedValue;
Expand All @@ -448,9 +448,9 @@ protected List<Object> visitIrArrayAccessor(IrArrayAccessor node, PathEvaluation
ImmutableList.Builder<Object> outputSequence = ImmutableList.builder();
for (Object object : sequence) {
List<Object> elements;
if (object instanceof JsonNode) {
if (((JsonNode) object).isArray()) {
elements = ImmutableList.copyOf(((JsonNode) object).elements());
if (object instanceof JsonNode jsonNode) {
if (jsonNode.isArray()) {
elements = ImmutableList.copyOf(jsonNode.elements());
}
else if (lax) {
elements = ImmutableList.of(object);
Expand Down Expand Up @@ -553,7 +553,7 @@ private static long asArrayIndex(Object object)
if (type instanceof DecimalType decimalType) {
int precision = decimalType.getPrecision();
int scale = decimalType.getScale();
if (((DecimalType) type).isShort()) {
if (decimalType.isShort()) {
long tenToScale = longTenToNth(DecimalConversions.intScale(scale));
return DecimalCasts.shortDecimalToBigint(value.getLongValue(), precision, scale, tenToScale);
}
Expand Down Expand Up @@ -581,9 +581,9 @@ protected List<Object> visitIrCeilingMethod(IrCeilingMethod node, PathEvaluation
ImmutableList.Builder<Object> outputSequence = ImmutableList.builder();
for (Object object : sequence) {
TypedValue value;
if (object instanceof JsonNode) {
value = getNumericTypedValue((JsonNode) object)
.orElseThrow(() -> itemTypeError("NUMBER", ((JsonNode) object).getNodeType().name()));
if (object instanceof JsonNode jsonNode) {
value = getNumericTypedValue(jsonNode)
.orElseThrow(() -> itemTypeError("NUMBER", jsonNode.getNodeType().name()));
}
else {
value = (TypedValue) object;
Expand Down Expand Up @@ -804,7 +804,7 @@ private static TypedValue getFloor(TypedValue typedValue)
if (type instanceof DecimalType decimalType) {
int scale = decimalType.getScale();
DecimalType resultType = DecimalType.createDecimalType(decimalType.getPrecision() - scale + Math.min(scale, 1), 0);
if (((DecimalType) type).isShort()) {
if (decimalType.isShort()) {
return new TypedValue(resultType, floorShort(scale, typedValue.getLongValue()));
}
if (resultType.isShort()) {
Expand Down Expand Up @@ -843,11 +843,11 @@ protected List<Object> visitIrKeyValueMethod(IrKeyValueMethod node, PathEvaluati

ImmutableList.Builder<Object> outputSequence = ImmutableList.builder();
for (Object object : sequence) {
if (!(object instanceof JsonNode)) {
if (!(object instanceof JsonNode jsonNode)) {
throw itemTypeError("OBJECT", ((TypedValue) object).getType().getDisplayName());
}
if (!((JsonNode) object).isObject()) {
throw itemTypeError("OBJECT", ((JsonNode) object).getNodeType().name());
if (!jsonNode.isObject()) {
throw itemTypeError("OBJECT", jsonNode.getNodeType().name());
}

// non-unique keys are not supported. if they were, we should follow the spec here on handling them.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,11 +162,11 @@ protected Boolean visitIrComparisonPredicate(IrComparisonPredicate node, PathEva
boolean leftHasScalar = false;
boolean leftHasNonScalar = false;
for (Object object : leftSequence) {
if (object instanceof JsonNode) {
if (object instanceof JsonNode jsonNode) {
if (object instanceof NullNode) {
leftHasJsonNull = true;
}
else if (((JsonNode) object).isValueNode()) {
else if (jsonNode.isValueNode()) {
leftHasScalar = true;
}
else {
Expand All @@ -182,11 +182,11 @@ else if (((JsonNode) object).isValueNode()) {
boolean rightHasScalar = false;
boolean rightHasNonScalar = false;
for (Object object : rightSequence) {
if (object instanceof JsonNode) {
if (((JsonNode) object).isNull()) {
if (object instanceof JsonNode jsonNode) {
if (jsonNode.isNull()) {
rightHasJsonNull = true;
}
else if (((JsonNode) object).isValueNode()) {
else if (jsonNode.isValueNode()) {
rightHasScalar = true;
}
else {
Expand Down Expand Up @@ -443,8 +443,8 @@ private static List<TypedValue> getScalars(List<Object> sequence)
{
ImmutableList.Builder<TypedValue> scalars = ImmutableList.builder();
for (Object object : sequence) {
if (object instanceof TypedValue) {
scalars.add((TypedValue) object);
if (object instanceof TypedValue typedValue) {
scalars.add(typedValue);
}
else {
JsonNode jsonNode = (JsonNode) object;
Expand All @@ -471,8 +471,8 @@ private static Slice getText(Object object)
{
if (object instanceof TypedValue typedValue) {
if (isCharacterStringType(typedValue.getType())) {
if (typedValue.getType() instanceof CharType) {
return padSpaces((Slice) typedValue.getObjectValue(), (CharType) typedValue.getType());
if (typedValue.getType() instanceof CharType charType) {
return padSpaces((Slice) typedValue.getObjectValue(), charType);
}
return (Slice) typedValue.getObjectValue();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,8 +126,8 @@ public static Optional<JsonNode> getJsonNode(TypedValue typedValue)
if (type.equals(BOOLEAN)) {
return Optional.of(BooleanNode.valueOf(typedValue.getBooleanValue()));
}
if (type instanceof CharType) {
return Optional.of(TextNode.valueOf(padSpaces((Slice) typedValue.getObjectValue(), (CharType) typedValue.getType()).toStringUtf8()));
if (type instanceof CharType charType) {
return Optional.of(TextNode.valueOf(padSpaces((Slice) typedValue.getObjectValue(), charType).toStringUtf8()));
}
if (type instanceof VarcharType) {
return Optional.of(TextNode.valueOf(((Slice) typedValue.getObjectValue()).toStringUtf8()));
Expand All @@ -144,15 +144,15 @@ public static Optional<JsonNode> getJsonNode(TypedValue typedValue)
if (type.equals(TINYINT)) {
return Optional.of(ShortNode.valueOf(Shorts.checkedCast(typedValue.getLongValue())));
}
if (type instanceof DecimalType) {
if (type instanceof DecimalType decimalType) {
BigInteger unscaledValue;
if (((DecimalType) type).isShort()) {
if (decimalType.isShort()) {
unscaledValue = BigInteger.valueOf(typedValue.getLongValue());
}
else {
unscaledValue = ((Int128) typedValue.getObjectValue()).toBigInteger();
}
return Optional.of(DecimalNode.valueOf(new BigDecimal(unscaledValue, ((DecimalType) type).getScale())));
return Optional.of(DecimalNode.valueOf(new BigDecimal(unscaledValue, decimalType.getScale())));
}
if (type.equals(DOUBLE)) {
return Optional.of(DoubleNode.valueOf(typedValue.getDoubleValue()));
Expand Down
Loading