Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@

import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.UUID;

import com.google.common.collect.ImmutableMap;
import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
import org.apache.ignite.internal.processors.query.calcite.message.InboxCloseMessage;
Expand Down Expand Up @@ -257,7 +257,7 @@ private ExecutionContext<?> baseInboxContext(String nodeId, UUID qryId, long fra
null,
null),
null,
ImmutableMap.of());
Map.of());
}

/** {@inheritDoc} */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;

import com.google.common.collect.ImmutableList;
import org.apache.calcite.plan.Contexts;
import org.apache.calcite.plan.ConventionTraitDef;
import org.apache.calcite.plan.RelOptUtil;
Expand Down Expand Up @@ -307,7 +306,7 @@ private List<QueryPlan> prepareQuery(PlanningContext ctx) {

/** */
private List<QueryPlan> prepareFragment(PlanningContext ctx) {
return ImmutableList.of(new FragmentPlan(fromJson(ctx, ctx.query())));
return List.of(new FragmentPlan(fromJson(ctx, ctx.query())));
}

/** */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@

import java.util.ArrayList;
import java.util.List;
import java.util.Set;

import com.google.common.collect.ImmutableSet;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelCollations;
import org.apache.calcite.rel.RelNode;
Expand Down Expand Up @@ -95,7 +95,7 @@ public static IgniteRel optimize(SqlNode sqlNode, IgnitePlanner planner, IgniteL
igniteRel = new IgniteProject(igniteRel.getCluster(), desired, igniteRel, projects, root.validatedRowType);
}

if (sqlNode.isA(ImmutableSet.of(SqlKind.INSERT, SqlKind.UPDATE, SqlKind.MERGE)))
if (sqlNode.isA(Set.of(SqlKind.INSERT, SqlKind.UPDATE, SqlKind.MERGE)))
igniteRel = new FixDependentModifyNodeShuttle().visit(igniteRel);

return igniteRel;
Expand Down Expand Up @@ -207,7 +207,7 @@ private IgniteRel processScan(TableScan scan) {
if (scan instanceof IgniteTableScan)
return (IgniteRel)scan;

ImmutableSet<Integer> indexedCols = ImmutableSet.copyOf(
Set<Integer> indexedCols = Set.copyOf(
tbl.getIndex(((AbstractIndexScan)scan).indexName()).collation().getKeys());

spoolNeeded = modifyNode.getUpdateColumnList().stream()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import java.util.function.Predicate;
import java.util.function.Supplier;

import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.RelCollation;
import org.apache.calcite.rel.RelFieldCollation;
import org.apache.calcite.rel.core.AggregateCall;
Expand Down Expand Up @@ -111,7 +110,7 @@ Supplier<List<AccumulatorWrapper<Row>>> accumulatorsFactory(
* @return Scalar.
*/
default Scalar scalar(RexNode node, RelDataType type) {
return scalar(ImmutableList.of(node), type);
return scalar(List.of(node), type);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,9 +29,6 @@
import java.util.function.Supplier;
import java.util.stream.Collectors;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Primitives;
import org.apache.calcite.DataContext;
import org.apache.calcite.adapter.enumerable.EnumUtils;
import org.apache.calcite.linq4j.function.Function1;
Expand Down Expand Up @@ -65,6 +62,7 @@
import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
import org.apache.ignite.internal.processors.query.calcite.util.Commons;
import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
import org.apache.ignite.internal.processors.query.calcite.util.Primitives;

import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;

Expand Down Expand Up @@ -119,7 +117,8 @@ public ExpressionFactoryImpl(ExecutionContext<Row> ctx, IgniteTypeFactory typeFa
return null;
else if (collation.getFieldCollations().size() == 1)
return comparator(collation.getFieldCollations().get(0));
return Ordering.compound(collation.getFieldCollations()

return Commons.compoundComparator(collation.getFieldCollations()
.stream()
.map(this::comparator)
.collect(Collectors.toList()));
Expand All @@ -136,7 +135,7 @@ else if (collation.getFieldCollations().size() == 1)
for (int i = 0; i < left.size(); i++)
comparators.add(comparator(left.get(i), right.get(i)));

return Ordering.compound(comparators);
return Commons.compoundComparator(comparators);
}

/** */
Expand Down Expand Up @@ -291,7 +290,7 @@ private Scalar compile(Iterable<RexNode> nodes, RelDataType type) {

MethodDeclaration decl = Expressions.methodDecl(
Modifier.PUBLIC, void.class, IgniteMethod.SCALAR_EXECUTE.method().getName(),
ImmutableList.of(ctx_, in_, out_), builder.toBlock());
List.of(ctx_, in_, out_), builder.toBlock());

return Commons.compile(Scalar.class, Expressions.toString(List.of(decl), "\n", false));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import java.util.function.Supplier;
import java.util.stream.Collectors;

import com.google.common.collect.ImmutableList;
import org.apache.calcite.adapter.enumerable.EnumUtils;
import org.apache.calcite.adapter.enumerable.NullPolicy;
import org.apache.calcite.adapter.java.JavaTypeFactory;
Expand Down Expand Up @@ -1107,7 +1106,7 @@ private static class BinaryImplementor extends AbstractRexCallImplementor {
* Types that can be arguments to comparison operators such as {@code <}.
*/
private static final List<Primitive> COMP_OP_TYPES =
ImmutableList.of(
List.of(
Primitive.BYTE,
Primitive.CHAR,
Primitive.SHORT,
Expand All @@ -1118,15 +1117,15 @@ private static class BinaryImplementor extends AbstractRexCallImplementor {

/** */
private static final List<SqlBinaryOperator> COMPARISON_OPERATORS =
ImmutableList.of(
List.of(
SqlStdOperatorTable.LESS_THAN,
SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
SqlStdOperatorTable.GREATER_THAN,
SqlStdOperatorTable.GREATER_THAN_OR_EQUAL);

/** */
private static final List<SqlBinaryOperator> EQUALS_OPERATORS =
ImmutableList.of(
List.of(
SqlStdOperatorTable.EQUALS,
SqlStdOperatorTable.NOT_EQUALS);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import java.util.Map;
import java.util.Objects;

import com.google.common.collect.ImmutableList;
import org.apache.calcite.adapter.enumerable.PhysType;
import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.avatica.util.ByteString;
Expand Down Expand Up @@ -1181,7 +1180,7 @@ private Result toInnerStorageType(final Result result, final Type storageType) {
builder.makeLiteral(fieldName, typeFactory.createType(String.class), true);
RexCall accessCall = (RexCall)builder.makeCall(
fieldAccess.getType(), SqlStdOperatorTable.STRUCT_ACCESS,
ImmutableList.of(target, rxIndex, rxName));
List.of(target, rxIndex, rxName));
final Result result2 = accessCall.accept(this);
rexWithStorageTypeResultMap.put(key, result2);
return result2;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,9 @@

import java.lang.reflect.Modifier;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.function.Function;
import java.util.function.Supplier;

import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableList;
import com.google.common.primitives.Primitives;
import org.apache.calcite.DataContext;
import org.apache.calcite.adapter.enumerable.EnumUtils;
import org.apache.calcite.adapter.enumerable.JavaRowFormat;
Expand All @@ -52,7 +46,8 @@
import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
import org.apache.ignite.internal.processors.query.calcite.util.Commons;
import org.apache.ignite.lang.IgniteInternalException;
import org.apache.ignite.internal.processors.query.calcite.util.LocalCache;
import org.apache.ignite.internal.processors.query.calcite.util.Primitives;
import org.jetbrains.annotations.NotNull;

import static org.apache.ignite.internal.processors.query.calcite.util.TypeUtils.createRowType;
Expand All @@ -61,8 +56,8 @@
/** */
public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrapper<Row>>> {
/** */
private static final LoadingCache<Pair<RelDataType, RelDataType>, Function<Object, Object>> CACHE =
CacheBuilder.newBuilder().build(CacheLoader.from(AccumulatorsFactory::cast0));
private static final LocalCache<Pair<RelDataType, RelDataType>, Function<Object, Object>> CACHE =
new LocalCache<>(AccumulatorsFactory::cast0);

/** */
public static interface CastFunction extends Function<Object, Object> {
Expand All @@ -79,12 +74,7 @@ private static Function<Object, Object> cast(RelDataType from, RelDataType to) {

/** */
private static Function<Object, Object> cast(Pair<RelDataType, RelDataType> types) {
try {
return CACHE.get(types);
}
catch (ExecutionException e) {
throw new IgniteInternalException(e);
}
return CACHE.get(types);
}

/** */
Expand Down Expand Up @@ -114,7 +104,7 @@ private static Function<Object, Object> compileCast(IgniteTypeFactory typeFactor

RexToLixTranslator.InputGetter getter =
new RexToLixTranslator.InputGetterImpl(
ImmutableList.of(
List.of(
Pair.of(EnumUtils.convert(in_, Object.class, typeFactory.getJavaClass(from)),
PhysTypeImpl.of(typeFactory, rowType,
JavaRowFormat.SCALAR, false))));
Expand All @@ -130,7 +120,7 @@ private static Function<Object, Object> compileCast(IgniteTypeFactory typeFactor
list.add(projects.get(0));

MethodDeclaration decl = Expressions.methodDecl(
Modifier.PUBLIC, Object.class, "apply", ImmutableList.of(in_), list.toBlock());
Modifier.PUBLIC, Object.class, "apply", List.of(in_), list.toBlock());

return Commons.compile(CastFunction.class, Expressions.toString(List.of(decl), "\n", false));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.lang.reflect.Modifier;
import java.math.BigDecimal;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
Expand All @@ -29,11 +30,7 @@
import java.util.function.Function;
import java.util.stream.Collectors;

import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.calcite.avatica.AvaticaUtils;
import org.apache.calcite.avatica.util.TimeUnit;
import org.apache.calcite.avatica.util.TimeUnitRange;
Expand Down Expand Up @@ -103,6 +100,7 @@
import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
import org.apache.ignite.internal.processors.query.calcite.util.Commons;
import org.apache.ignite.internal.processors.query.calcite.util.LocalCache;
import org.apache.ignite.internal.util.IgniteUtils;
import org.apache.ignite.lang.IgniteException;

Expand All @@ -126,8 +124,7 @@ public static interface RelFactory extends Function<RelInput, RelNode> {
}

/** */
private static final LoadingCache<String, RelFactory> FACTORIES_CACHE = CacheBuilder.newBuilder()
.build(CacheLoader.from(RelJson::relFactory));
private static final LocalCache<String, RelFactory> FACTORIES_CACHE = new LocalCache<>(RelJson::relFactory);

/** */
private static RelFactory relFactory(String typeName) {
Expand Down Expand Up @@ -164,15 +161,14 @@ private static RelFactory relFactory(String typeName) {
}

/** */
private static final ImmutableMap<String, Enum<?>> ENUM_BY_NAME;
private static final Map<String, Enum<?>> ENUM_BY_NAME;

/** */
static {
// Build a mapping from enum constants (e.g. LEADING) to the enum
// that contains them (e.g. SqlTrimFunction.Flag). If there two
// enum constants have the same name, the builder will throw.
final ImmutableMap.Builder<String, Enum<?>> enumByName =
ImmutableMap.builder();
final Map<String, Enum<?>> enumByName = new HashMap<>();

register(enumByName, JoinConditionType.class);
register(enumByName, JoinType.class);
Expand All @@ -193,14 +189,14 @@ private static RelFactory relFactory(String typeName) {
register(enumByName, SqlTrimFunction.Flag.class);
register(enumByName, TimeUnitRange.class);

ENUM_BY_NAME = enumByName.build();
ENUM_BY_NAME = Map.copyOf(enumByName);
}

/** */
private static void register(ImmutableMap.Builder<String, Enum<?>> builder, Class<? extends Enum> aClass) {
private static void register(Map<String, Enum<?>> map, Class<? extends Enum> aClass) {
String preffix = aClass.getSimpleName() + "#";
for (Enum enumConstant : aClass.getEnumConstants())
builder.put(preffix + enumConstant.name(), enumConstant);
map.put(preffix + enumConstant.name(), enumConstant);
}

/** */
Expand All @@ -218,7 +214,7 @@ private static Class<?> classForName(String typeName, boolean skipNotFound) {

/** */
private static final List<String> PACKAGES =
ImmutableList.of(
List.of(
"org.apache.ignite.internal.processors.query.calcite.rel.",
"org.apache.ignite.internal.processors.query.calcite.rel.agg.",
"org.apache.ignite.internal.processors.query.calcite.rel.set.",
Expand All @@ -235,7 +231,7 @@ private static Class<?> classForName(String typeName, boolean skipNotFound) {

/** */
Function<RelInput, RelNode> factory(String type) {
return FACTORIES_CACHE.getUnchecked(type);
return FACTORIES_CACHE.get(type);
}

/** */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,7 @@ private RelInputImpl(Map<String, Object> jsonRel) {
@Override public List<RelNode> getInputs() {
List<String> jsonInputs = getStringList("inputs");
if (jsonInputs == null)
return ImmutableList.of(lastRel);
return List.of(lastRel);
List<RelNode> inputs = new ArrayList<>();
for (String jsonInput : jsonInputs)
inputs.add(lookupInput(jsonInput));
Expand All @@ -176,13 +176,16 @@ private RelInputImpl(Map<String, Object> jsonRel) {
/** {@inheritDoc} */
@Override public List<ImmutableBitSet> getBitSetList(String tag) {
List<List<Integer>> list = getIntegerListList(tag);

if (list == null)
return null;
ImmutableList.Builder<ImmutableBitSet> builder =
ImmutableList.builder();

List<ImmutableBitSet> bitSets = new ArrayList<>();

for (List<Integer> integers : list)
builder.add(ImmutableBitSet.of(integers));
return builder.build();
bitSets.add(ImmutableBitSet.of(integers));

return List.copyOf(bitSets);
}

/** {@inheritDoc} */
Expand Down Expand Up @@ -289,8 +292,10 @@ private RelInputImpl(Map<String, Object> jsonRel) {
List<List> jsonTuples = (List)get(tag);
ImmutableList.Builder<ImmutableList<RexLiteral>> builder =
ImmutableList.builder();

for (List jsonTuple : jsonTuples)
builder.add(getTuple(jsonTuple));

return builder.build();
}

Expand All @@ -307,8 +312,10 @@ private RelNode lookupInput(String jsonInput) {
private ImmutableList<RexLiteral> getTuple(List jsonTuple) {
ImmutableList.Builder<RexLiteral> builder =
ImmutableList.builder();

for (Object jsonValue : jsonTuple)
builder.add((RexLiteral)relJson.toRex(this, jsonValue));

return builder.build();
}

Expand Down
Loading