diff --git a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaMetadata.java b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaMetadata.java index 14c8bea286c7..f11fa451544d 100644 --- a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaMetadata.java +++ b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaMetadata.java @@ -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; @@ -338,9 +337,8 @@ private Optional> filterString(TupleDomain constraint, T colu if (domain.isSingleValue()) { return Optional.of(ImmutableSet.of(((Slice) domain.getSingleValue()).toStringUtf8())); } - if (domain.getValues() instanceof EquatableValueSet) { - Collection 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())); diff --git a/core/trino-main/src/main/java/io/trino/connector/system/SystemTablesMetadata.java b/core/trino-main/src/main/java/io/trino/connector/system/SystemTablesMetadata.java index 82da42b69658..2473f976c70f 100644 --- a/core/trino-main/src/main/java/io/trino/connector/system/SystemTablesMetadata.java +++ b/core/trino-main/src/main/java/io/trino/connector/system/SystemTablesMetadata.java @@ -150,8 +150,8 @@ public Optional> applyFilter(C } SystemTable systemTable = checkAndGetTable(session, table); - if (systemTable instanceof JdbcTable) { - TupleDomain filtered = ((JdbcTable) systemTable).applyFilter(session, effectiveConstraint(oldDomain, constraint, newDomain)); + if (systemTable instanceof JdbcTable jdbcTable) { + TupleDomain filtered = jdbcTable.applyFilter(session, effectiveConstraint(oldDomain, constraint, newDomain)); newDomain = newDomain.intersect(filtered); } diff --git a/core/trino-main/src/main/java/io/trino/cost/CachingCostProvider.java b/core/trino-main/src/main/java/io/trino/cost/CachingCostProvider.java index dc2545405d72..7abcc9342f31 100644 --- a/core/trino-main/src/main/java/io/trino/cost/CachingCostProvider.java +++ b/core/trino-main/src/main/java/io/trino/cost/CachingCostProvider.java @@ -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); diff --git a/core/trino-main/src/main/java/io/trino/cost/CachingStatsProvider.java b/core/trino-main/src/main/java/io/trino/cost/CachingStatsProvider.java index 7fedcc549a5b..57d3ab4a53fa 100644 --- a/core/trino-main/src/main/java/io/trino/cost/CachingStatsProvider.java +++ b/core/trino-main/src/main/java/io/trino/cost/CachingStatsProvider.java @@ -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); diff --git a/core/trino-main/src/main/java/io/trino/cost/ComposableStatsCalculator.java b/core/trino-main/src/main/java/io/trino/cost/ComposableStatsCalculator.java index 202e6a9def98..ce01afb134e8 100644 --- a/core/trino-main/src/main/java/io/trino/cost/ComposableStatsCalculator.java +++ b/core/trino-main/src/main/java/io/trino/cost/ComposableStatsCalculator.java @@ -40,8 +40,10 @@ public ComposableStatsCalculator(List> 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( diff --git a/core/trino-main/src/main/java/io/trino/cost/FilterProjectAggregationStatsRule.java b/core/trino-main/src/main/java/io/trino/cost/FilterProjectAggregationStatsRule.java index b943e1b45d40..4bd823f9ab2c 100644 --- a/core/trino-main/src/main/java/io/trino/cost/FilterProjectAggregationStatsRule.java +++ b/core/trino-main/src/main/java/io/trino/cost/FilterProjectAggregationStatsRule.java @@ -67,13 +67,13 @@ protected Optional 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(); diff --git a/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsEstimate.java b/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsEstimate.java index 3f676d57cb6c..874414ef7b09 100644 --- a/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsEstimate.java +++ b/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsEstimate.java @@ -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); diff --git a/core/trino-main/src/main/java/io/trino/cost/RemoteSourceStatsRule.java b/core/trino-main/src/main/java/io/trino/cost/RemoteSourceStatsRule.java index 69c4346dd1a0..85682d576a9c 100644 --- a/core/trino-main/src/main/java/io/trino/cost/RemoteSourceStatsRule.java +++ b/core/trino-main/src/main/java/io/trino/cost/RemoteSourceStatsRule.java @@ -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; diff --git a/core/trino-main/src/main/java/io/trino/cost/ScalarStatsCalculator.java b/core/trino-main/src/main/java/io/trino/cost/ScalarStatsCalculator.java index 50996634fefc..d7d5527cd069 100644 --- a/core/trino-main/src/main/java/io/trino/cost/ScalarStatsCalculator.java +++ b/core/trino-main/src/main/java/io/trino/cost/ScalarStatsCalculator.java @@ -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; diff --git a/core/trino-main/src/main/java/io/trino/cost/SimpleFilterProjectSemiJoinStatsRule.java b/core/trino-main/src/main/java/io/trino/cost/SimpleFilterProjectSemiJoinStatsRule.java index e99099725e3d..6b1089c1255b 100644 --- a/core/trino-main/src/main/java/io/trino/cost/SimpleFilterProjectSemiJoinStatsRule.java +++ b/core/trino-main/src/main/java/io/trino/cost/SimpleFilterProjectSemiJoinStatsRule.java @@ -71,13 +71,13 @@ protected Optional 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(); @@ -141,8 +141,7 @@ private Optional 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 diff --git a/core/trino-main/src/main/java/io/trino/cost/StatsNormalizer.java b/core/trino-main/src/main/java/io/trino/cost/StatsNormalizer.java index 6b4aa8c6c93c..98933b3fcd97 100644 --- a/core/trino-main/src/main/java/io/trino/cost/StatsNormalizer.java +++ b/core/trino-main/src/main/java/io/trino/cost/StatsNormalizer.java @@ -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); } diff --git a/core/trino-main/src/main/java/io/trino/execution/StateMachine.java b/core/trino-main/src/main/java/io/trino/execution/StateMachine.java index 66051d52d00b..023b885c62a1 100644 --- a/core/trino-main/src/main/java/io/trino/execution/StateMachine.java +++ b/core/trino-main/src/main/java/io/trino/execution/StateMachine.java @@ -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; diff --git a/core/trino-main/src/main/java/io/trino/json/PathEvaluationUtil.java b/core/trino-main/src/main/java/io/trino/json/PathEvaluationUtil.java index 3c89b879b862..2eb96980e426 100644 --- a/core/trino-main/src/main/java/io/trino/json/PathEvaluationUtil.java +++ b/core/trino-main/src/main/java/io/trino/json/PathEvaluationUtil.java @@ -30,8 +30,8 @@ public static List unwrapArrays(List 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); }) diff --git a/core/trino-main/src/main/java/io/trino/json/PathEvaluationVisitor.java b/core/trino-main/src/main/java/io/trino/json/PathEvaluationVisitor.java index b985fbdf9b63..e17b1448033c 100644 --- a/core/trino-main/src/main/java/io/trino/json/PathEvaluationVisitor.java +++ b/core/trino-main/src/main/java/io/trino/json/PathEvaluationVisitor.java @@ -156,9 +156,9 @@ protected List visitIrAbsMethod(IrAbsMethod node, PathEvaluationContext ImmutableList.Builder 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; @@ -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; @@ -285,9 +285,9 @@ protected List 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; @@ -295,9 +295,9 @@ protected List visitIrArithmeticBinary(IrArithmeticBinary node, PathEval 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; @@ -351,9 +351,9 @@ protected List visitIrArithmeticUnary(IrArithmeticUnary node, PathEvalua ImmutableList.Builder 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; @@ -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; @@ -448,9 +448,9 @@ protected List visitIrArrayAccessor(IrArrayAccessor node, PathEvaluation ImmutableList.Builder outputSequence = ImmutableList.builder(); for (Object object : sequence) { List 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); @@ -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); } @@ -581,9 +581,9 @@ protected List visitIrCeilingMethod(IrCeilingMethod node, PathEvaluation ImmutableList.Builder 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; @@ -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()) { @@ -843,11 +843,11 @@ protected List visitIrKeyValueMethod(IrKeyValueMethod node, PathEvaluati ImmutableList.Builder 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. diff --git a/core/trino-main/src/main/java/io/trino/json/PathPredicateEvaluationVisitor.java b/core/trino-main/src/main/java/io/trino/json/PathPredicateEvaluationVisitor.java index 3c781cc93250..f1a6ab1e26d4 100644 --- a/core/trino-main/src/main/java/io/trino/json/PathPredicateEvaluationVisitor.java +++ b/core/trino-main/src/main/java/io/trino/json/PathPredicateEvaluationVisitor.java @@ -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 { @@ -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 { @@ -443,8 +443,8 @@ private static List getScalars(List sequence) { ImmutableList.Builder 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; @@ -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(); } diff --git a/core/trino-main/src/main/java/io/trino/json/ir/SqlJsonLiteralConverter.java b/core/trino-main/src/main/java/io/trino/json/ir/SqlJsonLiteralConverter.java index af56da7c03b6..d665ca744ada 100644 --- a/core/trino-main/src/main/java/io/trino/json/ir/SqlJsonLiteralConverter.java +++ b/core/trino-main/src/main/java/io/trino/json/ir/SqlJsonLiteralConverter.java @@ -126,8 +126,8 @@ public static Optional 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())); @@ -144,15 +144,15 @@ public static Optional 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())); diff --git a/core/trino-main/src/main/java/io/trino/metadata/Catalog.java b/core/trino-main/src/main/java/io/trino/metadata/Catalog.java index 2df116cf687e..c4214df979fa 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/Catalog.java +++ b/core/trino-main/src/main/java/io/trino/metadata/Catalog.java @@ -128,8 +128,8 @@ private static CatalogTransaction beginTransaction( { Connector connector = connectorServices.getConnector(); ConnectorTransactionHandle transactionHandle; - if (connector instanceof InternalConnector) { - transactionHandle = ((InternalConnector) connector).beginTransaction(transactionId, isolationLevel, readOnly); + if (connector instanceof InternalConnector internalConnector) { + transactionHandle = internalConnector.beginTransaction(transactionId, isolationLevel, readOnly); } else { transactionHandle = connector.beginTransaction(isolationLevel, readOnly, autoCommitContext); diff --git a/core/trino-main/src/main/java/io/trino/metadata/SessionPropertyManager.java b/core/trino-main/src/main/java/io/trino/metadata/SessionPropertyManager.java index d54e8c9dddf9..8c818d418a3a 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/SessionPropertyManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/SessionPropertyManager.java @@ -298,13 +298,13 @@ private static JsonCodec getJsonCodecForType(Type type) if (DoubleType.DOUBLE.equals(type)) { return (JsonCodec) JSON_CODEC_FACTORY.jsonCodec(Double.class); } - if (type instanceof ArrayType) { - Type elementType = ((ArrayType) type).getElementType(); + if (type instanceof ArrayType arrayType) { + Type elementType = arrayType.getElementType(); return (JsonCodec) JSON_CODEC_FACTORY.listJsonCodec(getJsonCodecForType(elementType)); } - if (type instanceof MapType) { - Type keyType = ((MapType) type).getKeyType(); - Type valueType = ((MapType) type).getValueType(); + if (type instanceof MapType mapType) { + Type keyType = mapType.getKeyType(); + Type valueType = mapType.getValueType(); return (JsonCodec) JSON_CODEC_FACTORY.mapJsonCodec(getMapKeyType(keyType), getJsonCodecForType(valueType)); } throw new TrinoException(INVALID_SESSION_PROPERTY, format("Session property type %s is not supported", type)); diff --git a/core/trino-main/src/main/java/io/trino/operator/Driver.java b/core/trino-main/src/main/java/io/trino/operator/Driver.java index 0033344d7fe5..2fd41377d685 100644 --- a/core/trino-main/src/main/java/io/trino/operator/Driver.java +++ b/core/trino-main/src/main/java/io/trino/operator/Driver.java @@ -131,9 +131,9 @@ private Driver(DriverContext driverContext, List operators) Optional sourceOperator = Optional.empty(); for (Operator operator : operators) { - if (operator instanceof SourceOperator) { + if (operator instanceof SourceOperator value) { checkArgument(sourceOperator.isEmpty(), "There must be at most one SourceOperator"); - sourceOperator = Optional.of((SourceOperator) operator); + sourceOperator = Optional.of(value); } } this.sourceOperator = sourceOperator; diff --git a/core/trino-main/src/main/java/io/trino/sql/gen/InCodeGenerator.java b/core/trino-main/src/main/java/io/trino/sql/gen/InCodeGenerator.java index f5bf5f2f492e..33880eca930e 100644 --- a/core/trino-main/src/main/java/io/trino/sql/gen/InCodeGenerator.java +++ b/core/trino-main/src/main/java/io/trino/sql/gen/InCodeGenerator.java @@ -101,10 +101,10 @@ static SwitchGenerationCase checkSwitchGenerationCase(Type type, List translate(ConnectorExpression expression) { - if (expression instanceof Variable) { - String name = ((Variable) expression).getName(); + if (expression instanceof Variable variable) { + String name = variable.getName(); return Optional.of(variableMappings.get(name).toSymbolReference()); } @@ -210,8 +210,8 @@ public Optional translate(ConnectorExpression expression) .map(base -> new Subscript(dereference.getType(), base, new Constant(INTEGER, (long) (dereference.getField() + 1)))); } - if (expression instanceof io.trino.spi.expression.Call) { - return translateCall((io.trino.spi.expression.Call) expression); + if (expression instanceof io.trino.spi.expression.Call call) { + return translateCall(call); } return Optional.empty(); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/EffectivePredicateExtractor.java b/core/trino-main/src/main/java/io/trino/sql/planner/EffectivePredicateExtractor.java index 06ca2fa3e5b3..634ad05bc940 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/EffectivePredicateExtractor.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/EffectivePredicateExtractor.java @@ -344,10 +344,10 @@ public Expression visitValues(ValuesNode node, Void context) } List> valuesBuilders = builders.build(); - for (Expression row : node.getRows().get()) { - if (row instanceof Row) { + for (Expression expression : node.getRows().get()) { + if (expression instanceof Row row) { for (int i = 0; i < node.getOutputSymbols().size(); i++) { - Expression value = ((Row) row).items().get(i); + Expression value = row.items().get(i); if (!DeterminismEvaluator.isDeterministic(value)) { nonDeterministic[i] = true; } @@ -379,10 +379,10 @@ public Expression visitValues(ValuesNode node, Void context) } } else { - if (!DeterminismEvaluator.isDeterministic(row)) { + if (!DeterminismEvaluator.isDeterministic(expression)) { return TRUE; } - IrExpressionInterpreter interpreter = new IrExpressionInterpreter(row, plannerContext, session); + IrExpressionInterpreter interpreter = new IrExpressionInterpreter(expression, plannerContext, session); Object evaluated = interpreter.optimize(NoOpSymbolResolver.INSTANCE); if (evaluated instanceof Expression) { return TRUE; diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/DereferencePushdown.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/DereferencePushdown.java index 9e2bea55db37..4aeec100e0fe 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/DereferencePushdown.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/DereferencePushdown.java @@ -63,8 +63,8 @@ public static boolean exclusiveDereferences(Set projections) { return projections.stream() .allMatch(expression -> expression instanceof Reference || - (expression instanceof Subscript && - isRowSubscriptChain((Subscript) expression) && + (expression instanceof Subscript subscript && + isRowSubscriptChain(subscript) && !prefixExists(expression, projections))); } @@ -116,14 +116,14 @@ private static boolean isRowSubscriptChain(Subscript expression) } return (expression.base() instanceof Reference) || - ((expression.base() instanceof Subscript) && isRowSubscriptChain((Subscript) expression.base())); + ((expression.base() instanceof Subscript subscript) && isRowSubscriptChain(subscript)); } private static boolean prefixExists(Expression expression, Set expressions) { Expression current = expression; - while (current instanceof Subscript) { - current = ((Subscript) current).base(); + while (current instanceof Subscript subscript) { + current = subscript.base(); if (expressions.contains(current)) { return true; } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/InlineProjections.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/InlineProjections.java index 6b4daa1e99d8..d04a5575d0c1 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/InlineProjections.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/InlineProjections.java @@ -196,6 +196,6 @@ private static Set extractInliningTargets(ProjectNode parent, ProjectNod private static boolean isSymbolReference(Symbol symbol, Expression expression) { - return expression instanceof Reference && ((Reference) expression).name().equals(symbol.getName()); + return expression instanceof Reference reference && reference.name().equals(symbol.getName()); } }