diff --git a/core/trino-main/pom.xml b/core/trino-main/pom.xml
index 28a2b4b1d762..5d11dc02e735 100644
--- a/core/trino-main/pom.xml
+++ b/core/trino-main/pom.xml
@@ -439,6 +439,12 @@
test
+
+ com.squareup
+ javapoet
+ test
+
+
org.junit.jupiter
junit-jupiter-api
diff --git a/core/trino-main/src/main/java/io/trino/FeaturesConfig.java b/core/trino-main/src/main/java/io/trino/FeaturesConfig.java
index b8bdf0aabd96..d63fb5e87642 100644
--- a/core/trino-main/src/main/java/io/trino/FeaturesConfig.java
+++ b/core/trino-main/src/main/java/io/trino/FeaturesConfig.java
@@ -101,6 +101,7 @@ public class FeaturesConfig
private boolean legacyCatalogRoles;
private boolean incrementalHashArrayLoadFactorEnabled = true;
private boolean allowSetViewAuthorization;
+ private boolean useEnhancedGroupBy = true;
private boolean hideInaccessibleColumns;
@@ -216,6 +217,17 @@ public FeaturesConfig setWriterMinSize(DataSize writerMinSize)
return this;
}
+ public boolean isUseEnhancedGroupBy()
+ {
+ return useEnhancedGroupBy;
+ }
+
+ @Config("optimizer.use-enhanced-group-by")
+ public void setUseEnhancedGroupBy(boolean useEnhancedGroupBy)
+ {
+ this.useEnhancedGroupBy = useEnhancedGroupBy;
+ }
+
@Min(2)
public int getRe2JDfaStatesLimit()
{
diff --git a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java
index 9d7a1e957e0d..617a65eb7d12 100644
--- a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java
+++ b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java
@@ -165,6 +165,8 @@ public final class SystemSessionProperties
public static final String ADAPTIVE_PARTIAL_AGGREGATION_ENABLED = "adaptive_partial_aggregation_enabled";
public static final String ADAPTIVE_PARTIAL_AGGREGATION_MIN_ROWS = "adaptive_partial_aggregation_min_rows";
public static final String ADAPTIVE_PARTIAL_AGGREGATION_UNIQUE_ROWS_RATIO_THRESHOLD = "adaptive_partial_aggregation_unique_rows_ratio_threshold";
+ public static final String TASK_MAX_PARTIAL_AGGREGATION_MEMORY = "task_max_partial_aggregation_memory";
+ public static final String USE_ENHANCED_GROUP_BY = "use_enhanced_group_by";
private final List> sessionProperties;
@@ -791,6 +793,16 @@ public SystemSessionProperties(
ADAPTIVE_PARTIAL_AGGREGATION_UNIQUE_ROWS_RATIO_THRESHOLD,
"Ratio between aggregation output and input rows above which partial aggregation might be adaptively turned off",
optimizerConfig.getAdaptivePartialAggregationUniqueRowsRatioThreshold(),
+ false),
+ dataSizeProperty(
+ TASK_MAX_PARTIAL_AGGREGATION_MEMORY,
+ "Maximum size of partial aggregation results for distributed aggregations.",
+ taskManagerConfig.getMaxPartialAggregationMemoryUsage(),
+ false),
+ booleanProperty(
+ USE_ENHANCED_GROUP_BY,
+ "Enable optimization for aggregations",
+ featuresConfig.isUseEnhancedGroupBy(),
false));
}
@@ -1428,4 +1440,14 @@ public static double getAdaptivePartialAggregationUniqueRowsRatioThreshold(Sessi
{
return session.getSystemProperty(ADAPTIVE_PARTIAL_AGGREGATION_UNIQUE_ROWS_RATIO_THRESHOLD, Double.class);
}
+
+ public static DataSize getMaxPartialAggregationMemoryUsage(Session session)
+ {
+ return session.getSystemProperty(TASK_MAX_PARTIAL_AGGREGATION_MEMORY, DataSize.class);
+ }
+
+ public static boolean isUseEnhancedGroupByEnabled(Session session)
+ {
+ return session.getSystemProperty(USE_ENHANCED_GROUP_BY, Boolean.class);
+ }
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/BigintGroupByHash.java b/core/trino-main/src/main/java/io/trino/operator/BigintGroupByHash.java
index af91384cb602..cda912749b29 100644
--- a/core/trino-main/src/main/java/io/trino/operator/BigintGroupByHash.java
+++ b/core/trino-main/src/main/java/io/trino/operator/BigintGroupByHash.java
@@ -191,7 +191,7 @@ public Work getGroupIds(Page page)
}
@Override
- public boolean contains(int position, Page page, int[] hashChannels)
+ public boolean contains(int position, Page page)
{
Block block = page.getBlock(hashChannel);
if (block.isNull(position)) {
diff --git a/core/trino-main/src/main/java/io/trino/operator/ChannelSet.java b/core/trino-main/src/main/java/io/trino/operator/ChannelSet.java
index 4b3fa88da142..547479212e2a 100644
--- a/core/trino-main/src/main/java/io/trino/operator/ChannelSet.java
+++ b/core/trino-main/src/main/java/io/trino/operator/ChannelSet.java
@@ -17,14 +17,14 @@
import com.google.common.collect.ImmutableList;
import io.trino.memory.context.LocalMemoryContext;
import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+import io.trino.spi.type.BigintType;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
-import io.trino.type.BlockTypeOperators;
import java.util.List;
import java.util.Optional;
-import static io.trino.operator.GroupByHash.createGroupByHash;
+import static io.trino.type.TypeUtils.NULL_HASH_CODE;
import static io.trino.type.UnknownType.UNKNOWN;
import static java.util.Objects.requireNonNull;
@@ -32,13 +32,11 @@ public class ChannelSet
{
private final GroupByHash hash;
private final boolean containsNull;
- private final int[] hashChannels;
- public ChannelSet(GroupByHash hash, boolean containsNull, int[] hashChannels)
+ public ChannelSet(GroupByHash hash, boolean containsNull)
{
this.hash = hash;
this.containsNull = containsNull;
- this.hashChannels = hashChannels;
}
public Type getType()
@@ -68,12 +66,12 @@ public boolean containsNull()
public boolean contains(int position, Page page)
{
- return hash.contains(position, page, hashChannels);
+ return hash.contains(position, page);
}
public boolean contains(int position, Page page, long rawHash)
{
- return hash.contains(position, page, hashChannels, rawHash);
+ return hash.contains(position, page, rawHash);
}
public static class ChannelSetBuilder
@@ -85,26 +83,38 @@ public static class ChannelSetBuilder
private final OperatorContext operatorContext;
private final LocalMemoryContext localMemoryContext;
- public ChannelSetBuilder(Type type, Optional hashChannel, int expectedPositions, OperatorContext operatorContext, JoinCompiler joinCompiler, BlockTypeOperators blockTypeOperators)
+ public ChannelSetBuilder(Type type, boolean hashPresent, int expectedPositions, OperatorContext operatorContext, GroupByHashFactory groupByHashFactory)
{
+ // Set builder has a single channel which goes in channel 0, if hash is present, add a hashBlock to channel 1
+ Optional hashChannel = hashPresent ? Optional.of(1) : Optional.empty();
List types = ImmutableList.of(type);
- this.hash = createGroupByHash(
+ this.hash = groupByHashFactory.createGroupByHash(
operatorContext.getSession(),
types,
HASH_CHANNELS,
hashChannel,
expectedPositions,
- joinCompiler,
- blockTypeOperators,
this::updateMemoryReservation);
- this.nullBlockPage = new Page(type.createBlockBuilder(null, 1, UNKNOWN.getFixedSize()).appendNull().build());
+ this.nullBlockPage = createNullPage(type, hashPresent);
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
this.localMemoryContext = operatorContext.localUserMemoryContext();
}
+ private static Page createNullPage(Type type, boolean hashPresent)
+ {
+ Block nullBlock = type.createBlockBuilder(null, 1, UNKNOWN.getFixedSize()).appendNull().build();
+ if (hashPresent) {
+ Block nullHashCode = BigintType.BIGINT.createBlockBuilder(null, 1).writeLong(NULL_HASH_CODE).build();
+ return new Page(nullBlock, nullHashCode);
+ }
+ else {
+ return new Page(nullBlock);
+ }
+ }
+
public ChannelSet build()
{
- return new ChannelSet(hash, hash.contains(0, nullBlockPage, HASH_CHANNELS), HASH_CHANNELS);
+ return new ChannelSet(hash, hash.contains(0, nullBlockPage));
}
public long getEstimatedSize()
diff --git a/core/trino-main/src/main/java/io/trino/operator/DistinctLimitOperator.java b/core/trino-main/src/main/java/io/trino/operator/DistinctLimitOperator.java
index 6fda981f6c51..e87b224bd739 100644
--- a/core/trino-main/src/main/java/io/trino/operator/DistinctLimitOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/DistinctLimitOperator.java
@@ -19,9 +19,7 @@
import io.trino.memory.context.LocalMemoryContext;
import io.trino.spi.Page;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.sql.planner.plan.PlanNodeId;
-import io.trino.type.BlockTypeOperators;
import java.util.Arrays;
import java.util.List;
@@ -32,7 +30,6 @@
import static com.google.common.base.Verify.verify;
import static com.google.common.base.Verify.verifyNotNull;
import static com.google.common.collect.ImmutableList.toImmutableList;
-import static io.trino.operator.GroupByHash.createGroupByHash;
import static java.lang.Math.min;
import static java.lang.Math.toIntExact;
import static java.util.Objects.requireNonNull;
@@ -50,8 +47,7 @@ public static class DistinctLimitOperatorFactory
private final long limit;
private final Optional hashChannel;
private boolean closed;
- private final JoinCompiler joinCompiler;
- private final BlockTypeOperators blockTypeOperators;
+ private final GroupByHashFactory groupByHashFactory;
public DistinctLimitOperatorFactory(
int operatorId,
@@ -60,8 +56,7 @@ public DistinctLimitOperatorFactory(
List distinctChannels,
long limit,
Optional hashChannel,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
this.operatorId = operatorId;
this.planNodeId = requireNonNull(planNodeId, "planNodeId is null");
@@ -71,8 +66,7 @@ public DistinctLimitOperatorFactory(
checkArgument(limit >= 0, "limit must be at least zero");
this.limit = limit;
this.hashChannel = requireNonNull(hashChannel, "hashChannel is null");
- this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
- this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
+ this.groupByHashFactory = requireNonNull(groupByHashFactory, "groupByHashFactory is null");
}
@Override
@@ -83,7 +77,7 @@ public Operator createOperator(DriverContext driverContext)
List distinctTypes = distinctChannels.stream()
.map(sourceTypes::get)
.collect(toImmutableList());
- return new DistinctLimitOperator(operatorContext, distinctChannels, distinctTypes, limit, hashChannel, joinCompiler, blockTypeOperators);
+ return new DistinctLimitOperator(operatorContext, distinctChannels, distinctTypes, limit, hashChannel, groupByHashFactory);
}
@Override
@@ -95,7 +89,7 @@ public void noMoreOperators()
@Override
public OperatorFactory duplicate()
{
- return new DistinctLimitOperatorFactory(operatorId, planNodeId, sourceTypes, distinctChannels, limit, hashChannel, joinCompiler, blockTypeOperators);
+ return new DistinctLimitOperatorFactory(operatorId, planNodeId, sourceTypes, distinctChannels, limit, hashChannel, groupByHashFactory);
}
}
@@ -115,7 +109,7 @@ public OperatorFactory duplicate()
private GroupByIdBlock groupByIds;
private Work unfinishedWork;
- public DistinctLimitOperator(OperatorContext operatorContext, List distinctChannels, List distinctTypes, long limit, Optional hashChannel, JoinCompiler joinCompiler, BlockTypeOperators blockTypeOperators)
+ public DistinctLimitOperator(OperatorContext operatorContext, List distinctChannels, List distinctTypes, long limit, Optional hashChannel, GroupByHashFactory groupByHashFactory)
{
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
this.localUserMemoryContext = operatorContext.localUserMemoryContext();
@@ -131,14 +125,12 @@ public DistinctLimitOperator(OperatorContext operatorContext, List dist
outputChannels = distinctChannelInts.clone(); // defensive copy since this is passed into createGroupByHash
}
- this.groupByHash = createGroupByHash(
+ this.groupByHash = groupByHashFactory.createGroupByHash(
operatorContext.getSession(),
distinctTypes,
distinctChannelInts,
hashChannel,
toIntExact(Math.min(limit, 10_000)),
- joinCompiler,
- blockTypeOperators,
this::updateMemoryReservation);
remainingLimit = limit;
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/GroupByHash.java b/core/trino-main/src/main/java/io/trino/operator/GroupByHash.java
index 5ca6fdcfcb93..9455feaef0e8 100644
--- a/core/trino-main/src/main/java/io/trino/operator/GroupByHash.java
+++ b/core/trino-main/src/main/java/io/trino/operator/GroupByHash.java
@@ -14,50 +14,19 @@
package io.trino.operator;
import com.google.common.annotations.VisibleForTesting;
-import io.trino.Session;
+import io.trino.operator.aggregation.GroupedAggregator;
import io.trino.spi.Page;
import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
-import io.trino.type.BlockTypeOperators;
+import it.unimi.dsi.fastutil.ints.IntIterator;
import java.util.List;
-import java.util.Optional;
-import static io.trino.SystemSessionProperties.isDictionaryAggregationEnabled;
-import static io.trino.spi.type.BigintType.BIGINT;
+import static io.trino.operator.WorkProcessor.ProcessState;
public interface GroupByHash
{
- static GroupByHash createGroupByHash(
- Session session,
- List extends Type> hashTypes,
- int[] hashChannels,
- Optional inputHashChannel,
- int expectedSize,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators,
- UpdateMemory updateMemory)
- {
- return createGroupByHash(hashTypes, hashChannels, inputHashChannel, expectedSize, isDictionaryAggregationEnabled(session), joinCompiler, blockTypeOperators, updateMemory);
- }
-
- static GroupByHash createGroupByHash(
- List extends Type> hashTypes,
- int[] hashChannels,
- Optional inputHashChannel,
- int expectedSize,
- boolean processDictionary,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators,
- UpdateMemory updateMemory)
- {
- if (hashTypes.size() == 1 && hashTypes.get(0).equals(BIGINT) && hashChannels.length == 1) {
- return new BigintGroupByHash(hashChannels[0], inputHashChannel.isPresent(), expectedSize, updateMemory);
- }
- return new MultiChannelGroupByHash(hashTypes, hashChannels, inputHashChannel, expectedSize, processDictionary, joinCompiler, blockTypeOperators, updateMemory);
- }
-
long getEstimatedSize();
long getHashCollisions();
@@ -74,15 +43,42 @@ static GroupByHash createGroupByHash(
Work getGroupIds(Page page);
- boolean contains(int position, Page page, int[] hashChannels);
+ boolean contains(int position, Page page);
- default boolean contains(int position, Page page, int[] hashChannels, long rawHash)
+ default boolean contains(int position, Page page, long rawHash)
{
- return contains(position, page, hashChannels);
+ return contains(position, page);
}
long getRawHash(int groupyId);
@VisibleForTesting
int getCapacity();
+
+ default WorkProcessor buildResult(IntIterator groupIds, PageBuilder pageBuilder, List groupedAggregators)
+ {
+ return WorkProcessor.create(() -> {
+ if (!groupIds.hasNext()) {
+ return ProcessState.finished();
+ }
+
+ pageBuilder.reset();
+
+ List types = getTypes();
+ while (!pageBuilder.isFull() && groupIds.hasNext()) {
+ int groupId = groupIds.nextInt();
+
+ appendValuesTo(groupId, pageBuilder);
+
+ pageBuilder.declarePosition();
+ for (int i = 0; i < groupedAggregators.size(); i++) {
+ GroupedAggregator groupedAggregator = groupedAggregators.get(i);
+ BlockBuilder output = pageBuilder.getBlockBuilder(types.size() + i);
+ groupedAggregator.evaluate(groupId, output);
+ }
+ }
+
+ return ProcessState.ofResult(pageBuilder.build());
+ });
+ }
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/GroupByHashFactory.java b/core/trino-main/src/main/java/io/trino/operator/GroupByHashFactory.java
new file mode 100644
index 000000000000..f390e09e88e6
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/GroupByHashFactory.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator;
+
+import io.trino.Session;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableDataGroupByHash;
+import io.trino.operator.hash.IsolatedHashTableFactory;
+import io.trino.spi.type.Type;
+import io.trino.sql.gen.JoinCompiler;
+import io.trino.type.BlockTypeOperators;
+
+import javax.inject.Inject;
+
+import java.util.List;
+import java.util.Optional;
+
+import static io.trino.SystemSessionProperties.isDictionaryAggregationEnabled;
+import static io.trino.SystemSessionProperties.isUseEnhancedGroupByEnabled;
+import static io.trino.operator.hash.IsolatedHashTableFactory.MAX_SUPPORTED_CHANNELS;
+import static io.trino.spi.type.BigintType.BIGINT;
+
+public class GroupByHashFactory
+{
+ private final JoinCompiler joinCompiler;
+ private final BlockTypeOperators blockTypeOperators;
+ private final IsolatedHashTableFactory isolatedHashTableFactory;
+
+ @Inject
+ public GroupByHashFactory(JoinCompiler joinCompiler, BlockTypeOperators blockTypeOperators, IsolatedHashTableFactory isolatedHashTableFactory)
+ {
+ this.joinCompiler = joinCompiler;
+ this.blockTypeOperators = blockTypeOperators;
+ this.isolatedHashTableFactory = isolatedHashTableFactory;
+ }
+
+ public GroupByHash createGroupByHash(
+ Session session,
+ List extends Type> hashTypes,
+ int[] hashChannels,
+ Optional inputHashChannel,
+ int expectedSize,
+ UpdateMemory updateMemory)
+ {
+ return createGroupByHash(
+ hashTypes,
+ hashChannels,
+ inputHashChannel,
+ expectedSize,
+ isDictionaryAggregationEnabled(session),
+ isUseEnhancedGroupByEnabled(session),
+ updateMemory);
+ }
+
+ public GroupByHash createGroupByHash(
+ List extends Type> hashTypes,
+ int[] hashChannels,
+ Optional inputHashChannel,
+ int expectedSize,
+ boolean processDictionary,
+ UpdateMemory updateMemory)
+ {
+ return createGroupByHash(hashTypes, hashChannels, inputHashChannel, expectedSize, processDictionary, false, updateMemory);
+ }
+
+ public GroupByHash createGroupByHash(
+ List extends Type> hashTypes,
+ int[] hashChannels,
+ Optional inputHashChannel,
+ int expectedSize,
+ boolean processDictionary,
+ boolean useEnhancedGroupBy,
+ UpdateMemory updateMemory)
+ {
+ if (hashTypes.size() == 1 && hashTypes.get(0).equals(BIGINT) && hashChannels.length == 1) {
+ return new BigintGroupByHash(hashChannels[0], inputHashChannel.isPresent(), expectedSize, updateMemory);
+ }
+ if (useEnhancedGroupBy &&
+ hashTypes.size() <= MAX_SUPPORTED_CHANNELS &&
+ hashTypes.stream()
+ .map(ColumnValueExtractor::columnValueExtractor)
+ .allMatch(extractor -> extractor.map(ColumnValueExtractor::isFixedSize).orElse(false))) {
+ return new HashTableDataGroupByHash(
+ isolatedHashTableFactory,
+ hashTypes,
+ hashChannels,
+ inputHashChannel,
+ expectedSize,
+ processDictionary,
+ updateMemory,
+ blockTypeOperators);
+ }
+
+ return new MultiChannelGroupByHash(hashTypes, hashChannels, inputHashChannel, expectedSize, processDictionary, joinCompiler, blockTypeOperators, updateMemory);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/GroupByHashPageIndexer.java b/core/trino-main/src/main/java/io/trino/operator/GroupByHashPageIndexer.java
index 62a74ce95fde..7d254795bffd 100644
--- a/core/trino-main/src/main/java/io/trino/operator/GroupByHashPageIndexer.java
+++ b/core/trino-main/src/main/java/io/trino/operator/GroupByHashPageIndexer.java
@@ -16,8 +16,6 @@
import io.trino.spi.Page;
import io.trino.spi.PageIndexer;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
-import io.trino.type.BlockTypeOperators;
import java.util.List;
import java.util.Optional;
@@ -33,16 +31,14 @@ public class GroupByHashPageIndexer
{
private final GroupByHash hash;
- public GroupByHashPageIndexer(List extends Type> hashTypes, JoinCompiler joinCompiler, BlockTypeOperators blockTypeOperators)
+ public GroupByHashPageIndexer(List extends Type> hashTypes, GroupByHashFactory groupByHashFactory)
{
- this(GroupByHash.createGroupByHash(
+ this(groupByHashFactory.createGroupByHash(
hashTypes,
IntStream.range(0, hashTypes.size()).toArray(),
Optional.empty(),
20,
false,
- joinCompiler,
- blockTypeOperators,
NOOP));
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/GroupByHashPageIndexerFactory.java b/core/trino-main/src/main/java/io/trino/operator/GroupByHashPageIndexerFactory.java
index 45d9e9c04713..cdc0d48a79f6 100644
--- a/core/trino-main/src/main/java/io/trino/operator/GroupByHashPageIndexerFactory.java
+++ b/core/trino-main/src/main/java/io/trino/operator/GroupByHashPageIndexerFactory.java
@@ -17,8 +17,6 @@
import io.trino.spi.PageIndexer;
import io.trino.spi.PageIndexerFactory;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
-import io.trino.type.BlockTypeOperators;
import javax.inject.Inject;
@@ -29,14 +27,12 @@
public class GroupByHashPageIndexerFactory
implements PageIndexerFactory
{
- private final JoinCompiler joinCompiler;
- private final BlockTypeOperators blockTypeOperators;
+ private final GroupByHashFactory groupByHashFactory;
@Inject
- public GroupByHashPageIndexerFactory(JoinCompiler joinCompiler, BlockTypeOperators blockTypeOperators)
+ public GroupByHashPageIndexerFactory(GroupByHashFactory groupByHashFactory)
{
- this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
- this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
+ this.groupByHashFactory = requireNonNull(groupByHashFactory, "groupByHashFactory is null");
}
@Override
@@ -45,7 +41,7 @@ public PageIndexer createPageIndexer(List extends Type> types)
if (types.isEmpty()) {
return new NoHashPageIndexer();
}
- return new GroupByHashPageIndexer(types, joinCompiler, blockTypeOperators);
+ return new GroupByHashPageIndexer(types, groupByHashFactory);
}
private static class NoHashPageIndexer
diff --git a/core/trino-main/src/main/java/io/trino/operator/HashAggregationOperator.java b/core/trino-main/src/main/java/io/trino/operator/HashAggregationOperator.java
index a500a160e102..73731c00d1f0 100644
--- a/core/trino-main/src/main/java/io/trino/operator/HashAggregationOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/HashAggregationOperator.java
@@ -27,21 +27,25 @@
import io.trino.operator.scalar.CombineHashFunction;
import io.trino.spi.Page;
import io.trino.spi.PageBuilder;
+import io.trino.spi.block.Block;
import io.trino.spi.type.BigintType;
import io.trino.spi.type.Type;
import io.trino.spiller.SpillerFactory;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.sql.planner.plan.AggregationNode.Step;
import io.trino.sql.planner.plan.PlanNodeId;
import io.trino.type.BlockTypeOperators;
import java.util.List;
import java.util.Optional;
+import java.util.OptionalInt;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static io.airlift.units.DataSize.Unit.MEGABYTE;
+import static io.trino.operator.aggregation.builder.InMemoryHashAggregationBuilder.nullRle;
import static io.trino.operator.aggregation.builder.InMemoryHashAggregationBuilder.toTypes;
+import static io.trino.spi.type.BooleanType.BOOLEAN;
import static io.trino.sql.planner.optimizations.HashGenerationOptimizer.INITIAL_HASH_VALUE;
import static io.trino.type.TypeUtils.NULL_HASH_CODE;
import static java.util.Objects.requireNonNull;
@@ -62,6 +66,9 @@ public static class HashAggregationOperatorFactory
private final Step step;
private final boolean produceDefaultOutput;
private final List aggregatorFactories;
+ private final List aggregationInputTypes;
+ private final List aggregationInputChannels;
+ private final List maskBlockChannels;
private final Optional hashChannel;
private final Optional groupIdChannel;
@@ -71,7 +78,7 @@ public static class HashAggregationOperatorFactory
private final DataSize memoryLimitForMerge;
private final DataSize memoryLimitForMergeWithMemory;
private final SpillerFactory spillerFactory;
- private final JoinCompiler joinCompiler;
+ private final GroupByHashFactory groupByHashFactory;
private final BlockTypeOperators blockTypeOperators;
private final Optional partialAggregationController;
@@ -83,6 +90,8 @@ public HashAggregationOperatorFactory(
PlanNodeId planNodeId,
List extends Type> groupByTypes,
List groupByChannels,
+ List extends Type> aggregationInputTypes,
+ List aggregationInputChannels,
List globalAggregationGroupIds,
Step step,
List aggregatorFactories,
@@ -90,7 +99,7 @@ public HashAggregationOperatorFactory(
Optional groupIdChannel,
int expectedGroups,
Optional maxPartialMemory,
- JoinCompiler joinCompiler,
+ GroupByHashFactory groupByHashFactory,
BlockTypeOperators blockTypeOperators,
Optional partialAggregationController)
{
@@ -98,6 +107,8 @@ public HashAggregationOperatorFactory(
planNodeId,
groupByTypes,
groupByChannels,
+ aggregationInputTypes,
+ aggregationInputChannels,
globalAggregationGroupIds,
step,
false,
@@ -112,7 +123,7 @@ public HashAggregationOperatorFactory(
(types, spillContext, memoryContext) -> {
throw new UnsupportedOperationException();
},
- joinCompiler,
+ groupByHashFactory,
blockTypeOperators,
partialAggregationController);
}
@@ -122,6 +133,8 @@ public HashAggregationOperatorFactory(
PlanNodeId planNodeId,
List extends Type> groupByTypes,
List groupByChannels,
+ List extends Type> aggregationInputTypes,
+ List aggregationInputChannels,
List globalAggregationGroupIds,
Step step,
boolean produceDefaultOutput,
@@ -133,7 +146,7 @@ public HashAggregationOperatorFactory(
boolean spillEnabled,
DataSize unspillMemoryLimit,
SpillerFactory spillerFactory,
- JoinCompiler joinCompiler,
+ GroupByHashFactory groupByHashFactory,
BlockTypeOperators blockTypeOperators,
Optional partialAggregationController)
{
@@ -141,6 +154,8 @@ public HashAggregationOperatorFactory(
planNodeId,
groupByTypes,
groupByChannels,
+ aggregationInputTypes,
+ aggregationInputChannels,
globalAggregationGroupIds,
step,
produceDefaultOutput,
@@ -153,7 +168,7 @@ public HashAggregationOperatorFactory(
unspillMemoryLimit,
DataSize.succinctBytes((long) (unspillMemoryLimit.toBytes() * MERGE_WITH_MEMORY_RATIO)),
spillerFactory,
- joinCompiler,
+ groupByHashFactory,
blockTypeOperators,
partialAggregationController);
}
@@ -164,6 +179,8 @@ public HashAggregationOperatorFactory(
PlanNodeId planNodeId,
List extends Type> groupByTypes,
List groupByChannels,
+ List extends Type> aggregationInputTypes,
+ List aggregationInputChannels,
List globalAggregationGroupIds,
Step step,
boolean produceDefaultOutput,
@@ -176,7 +193,7 @@ public HashAggregationOperatorFactory(
DataSize memoryLimitForMerge,
DataSize memoryLimitForMergeWithMemory,
SpillerFactory spillerFactory,
- JoinCompiler joinCompiler,
+ GroupByHashFactory groupByHashFactory,
BlockTypeOperators blockTypeOperators,
Optional partialAggregationController)
{
@@ -186,6 +203,13 @@ public HashAggregationOperatorFactory(
this.groupIdChannel = requireNonNull(groupIdChannel, "groupIdChannel is null");
this.groupByTypes = ImmutableList.copyOf(groupByTypes);
this.groupByChannels = ImmutableList.copyOf(groupByChannels);
+ this.aggregationInputTypes = ImmutableList.copyOf(aggregationInputTypes);
+ this.aggregationInputChannels = ImmutableList.copyOf(aggregationInputChannels);
+ this.maskBlockChannels = ImmutableList.copyOf(aggregatorFactories.stream()
+ .map(AggregatorFactory::getMaskChannel)
+ .flatMapToInt(OptionalInt::stream)
+ .boxed()
+ .collect(toImmutableSet()));
this.globalAggregationGroupIds = ImmutableList.copyOf(globalAggregationGroupIds);
this.step = step;
this.produceDefaultOutput = produceDefaultOutput;
@@ -196,7 +220,7 @@ public HashAggregationOperatorFactory(
this.memoryLimitForMerge = requireNonNull(memoryLimitForMerge, "memoryLimitForMerge is null");
this.memoryLimitForMergeWithMemory = requireNonNull(memoryLimitForMergeWithMemory, "memoryLimitForMergeWithMemory is null");
this.spillerFactory = requireNonNull(spillerFactory, "spillerFactory is null");
- this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
+ this.groupByHashFactory = requireNonNull(groupByHashFactory, "groupByHashFactory is null");
this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
this.partialAggregationController = requireNonNull(partialAggregationController, "partialAggregationController is null");
}
@@ -211,6 +235,9 @@ public Operator createOperator(DriverContext driverContext)
operatorContext,
groupByTypes,
groupByChannels,
+ aggregationInputTypes,
+ aggregationInputChannels,
+ maskBlockChannels,
globalAggregationGroupIds,
step,
produceDefaultOutput,
@@ -223,7 +250,7 @@ public Operator createOperator(DriverContext driverContext)
memoryLimitForMerge,
memoryLimitForMergeWithMemory,
spillerFactory,
- joinCompiler,
+ groupByHashFactory,
blockTypeOperators,
partialAggregationController);
return hashAggregationOperator;
@@ -243,6 +270,8 @@ public OperatorFactory duplicate()
planNodeId,
groupByTypes,
groupByChannels,
+ aggregationInputTypes,
+ aggregationInputChannels,
globalAggregationGroupIds,
step,
produceDefaultOutput,
@@ -255,7 +284,7 @@ public OperatorFactory duplicate()
memoryLimitForMerge,
memoryLimitForMergeWithMemory,
spillerFactory,
- joinCompiler,
+ groupByHashFactory,
blockTypeOperators,
partialAggregationController.map(PartialAggregationController::duplicate));
}
@@ -265,6 +294,9 @@ public OperatorFactory duplicate()
private final Optional partialAggregationController;
private final List groupByTypes;
private final List groupByChannels;
+ private final List aggregationInputTypes;
+ private final List aggregationInputChannels;
+ private final List maskBlockChannels;
private final List globalAggregationGroupIds;
private final Step step;
private final boolean produceDefaultOutput;
@@ -277,7 +309,7 @@ public OperatorFactory duplicate()
private final DataSize memoryLimitForMerge;
private final DataSize memoryLimitForMergeWithMemory;
private final SpillerFactory spillerFactory;
- private final JoinCompiler joinCompiler;
+ private final GroupByHashFactory groupByHashFactory;
private final BlockTypeOperators blockTypeOperators;
private final List types;
@@ -299,6 +331,9 @@ private HashAggregationOperator(
OperatorContext operatorContext,
List groupByTypes,
List groupByChannels,
+ List aggregationInputTypes,
+ List aggregationInputChannels,
+ List maskBlockChannels,
List globalAggregationGroupIds,
Step step,
boolean produceDefaultOutput,
@@ -311,7 +346,7 @@ private HashAggregationOperator(
DataSize memoryLimitForMerge,
DataSize memoryLimitForMergeWithMemory,
SpillerFactory spillerFactory,
- JoinCompiler joinCompiler,
+ GroupByHashFactory groupByHashFactory,
BlockTypeOperators blockTypeOperators,
Optional partialAggregationController)
{
@@ -324,6 +359,9 @@ private HashAggregationOperator(
this.groupByTypes = ImmutableList.copyOf(groupByTypes);
this.groupByChannels = ImmutableList.copyOf(groupByChannels);
+ this.aggregationInputTypes = ImmutableList.copyOf(aggregationInputTypes);
+ this.aggregationInputChannels = ImmutableList.copyOf(aggregationInputChannels);
+ this.maskBlockChannels = ImmutableList.copyOf(maskBlockChannels);
this.globalAggregationGroupIds = ImmutableList.copyOf(globalAggregationGroupIds);
this.aggregatorFactories = ImmutableList.copyOf(aggregatorFactories);
this.hashChannel = requireNonNull(hashChannel, "hashChannel is null");
@@ -337,7 +375,7 @@ private HashAggregationOperator(
this.memoryLimitForMerge = requireNonNull(memoryLimitForMerge, "memoryLimitForMerge is null");
this.memoryLimitForMergeWithMemory = requireNonNull(memoryLimitForMergeWithMemory, "memoryLimitForMergeWithMemory is null");
this.spillerFactory = requireNonNull(spillerFactory, "spillerFactory is null");
- this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
+ this.groupByHashFactory = requireNonNull(groupByHashFactory, "groupByHashFactory is null");
this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
this.hashCollisionsCounter = new HashCollisionsCounter(operatorContext);
operatorContext.setInfoSupplier(hashCollisionsCounter);
@@ -390,7 +428,7 @@ public void addInput(Page page)
.map(PartialAggregationController::isPartialAggregationDisabled)
.orElse(false);
if (step.isOutputPartial() && partialAggregationDisabled) {
- aggregationBuilder = new SkipAggregationBuilder(groupByChannels, hashChannel, aggregatorFactories, memoryContext);
+ aggregationBuilder = new SkipAggregationBuilder(groupByChannels, hashChannel, aggregatorFactories, memoryContext, aggregationInputChannels, maskBlockChannels);
}
else if (step.isOutputPartial() || !spillEnabled || !isSpillable()) {
// TODO: We ignore spillEnabled here if any aggregate has ORDER BY clause or DISTINCT because they are not yet implemented for spilling.
@@ -400,11 +438,12 @@ else if (step.isOutputPartial() || !spillEnabled || !isSpillable()) {
expectedGroups,
groupByTypes,
groupByChannels,
+ aggregationInputTypes,
+ maskBlockChannels.size(),
hashChannel,
operatorContext,
maxPartialMemory,
- joinCompiler,
- blockTypeOperators,
+ groupByHashFactory,
() -> {
memoryContext.setBytes(((InMemoryHashAggregationBuilder) aggregationBuilder).getSizeInMemory());
if (step.isOutputPartial() && maxPartialMemory.isPresent()) {
@@ -421,12 +460,14 @@ else if (step.isOutputPartial() || !spillEnabled || !isSpillable()) {
expectedGroups,
groupByTypes,
groupByChannels,
+ aggregationInputTypes,
+ maskBlockChannels.size(),
hashChannel,
operatorContext,
memoryLimitForMerge,
memoryLimitForMergeWithMemory,
spillerFactory,
- joinCompiler,
+ groupByHashFactory,
blockTypeOperators);
}
@@ -584,7 +625,28 @@ private Page getGlobalAggregationOutput()
if (output.isEmpty()) {
return null;
}
- return output.build();
+
+ Page page = output.build();
+ if (step.isOutputPartial()) {
+ // only from partial step output raw input columns
+ Block[] finalPage = new Block[page.getChannelCount() + aggregationInputTypes.size() + 1];
+ for (int i = 0; i < page.getChannelCount(); i++) {
+ finalPage[i] = page.getBlock(i);
+ }
+ int positionCount = page.getPositionCount();
+ int maskChannelCount = maskBlockChannels.size();
+ for (int i = 0; i < maskChannelCount; i++) {
+ finalPage[page.getChannelCount() + i] = nullRle(BOOLEAN, positionCount);
+ }
+ for (int i = 0; i < aggregationInputTypes.size(); i++) {
+ finalPage[page.getChannelCount() + maskChannelCount + i] = nullRle(aggregationInputTypes.get(i), positionCount);
+ }
+
+ finalPage[finalPage.length - 1] = nullRle(BOOLEAN, positionCount);
+ page = Page.wrapBlocksWithoutCopy(positionCount, finalPage);
+ }
+
+ return page;
}
private static long calculateDefaultOutputHash(List groupByChannels, int groupIdChannel, int groupId)
diff --git a/core/trino-main/src/main/java/io/trino/operator/MarkDistinctHash.java b/core/trino-main/src/main/java/io/trino/operator/MarkDistinctHash.java
index 859d36ff2b8e..117eb6590cc2 100644
--- a/core/trino-main/src/main/java/io/trino/operator/MarkDistinctHash.java
+++ b/core/trino-main/src/main/java/io/trino/operator/MarkDistinctHash.java
@@ -20,28 +20,25 @@
import io.trino.spi.block.RunLengthEncodedBlock;
import io.trino.spi.type.BooleanType;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
-import io.trino.type.BlockTypeOperators;
import java.util.List;
import java.util.Optional;
import static com.google.common.base.Preconditions.checkState;
-import static io.trino.operator.GroupByHash.createGroupByHash;
public class MarkDistinctHash
{
private final GroupByHash groupByHash;
private long nextDistinctId;
- public MarkDistinctHash(Session session, List types, int[] channels, Optional hashChannel, JoinCompiler joinCompiler, BlockTypeOperators blockTypeOperators, UpdateMemory updateMemory)
+ public MarkDistinctHash(Session session, List types, int[] channels, Optional hashChannel, GroupByHashFactory groupByHashFactory, UpdateMemory updateMemory)
{
- this(session, types, channels, hashChannel, 10_000, joinCompiler, blockTypeOperators, updateMemory);
+ this(session, types, channels, hashChannel, 10_000, groupByHashFactory, updateMemory);
}
- public MarkDistinctHash(Session session, List types, int[] channels, Optional hashChannel, int expectedDistinctValues, JoinCompiler joinCompiler, BlockTypeOperators blockTypeOperators, UpdateMemory updateMemory)
+ public MarkDistinctHash(Session session, List types, int[] channels, Optional hashChannel, int expectedDistinctValues, GroupByHashFactory groupByHashFactory, UpdateMemory updateMemory)
{
- this.groupByHash = createGroupByHash(session, types, channels, hashChannel, expectedDistinctValues, joinCompiler, blockTypeOperators, updateMemory);
+ this.groupByHash = groupByHashFactory.createGroupByHash(session, types, channels, hashChannel, expectedDistinctValues, updateMemory);
}
public long getEstimatedSize()
diff --git a/core/trino-main/src/main/java/io/trino/operator/MarkDistinctOperator.java b/core/trino-main/src/main/java/io/trino/operator/MarkDistinctOperator.java
index e000d6e44faf..a8e1cedb72b6 100644
--- a/core/trino-main/src/main/java/io/trino/operator/MarkDistinctOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/MarkDistinctOperator.java
@@ -20,9 +20,7 @@
import io.trino.spi.Page;
import io.trino.spi.block.Block;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.sql.planner.plan.PlanNodeId;
-import io.trino.type.BlockTypeOperators;
import java.util.Collection;
import java.util.List;
@@ -44,8 +42,7 @@ public static class MarkDistinctOperatorFactory
private final Optional hashChannel;
private final List markDistinctChannels;
private final List types;
- private final JoinCompiler joinCompiler;
- private final BlockTypeOperators blockTypeOperators;
+ private final GroupByHashFactory groupByHashFactory;
private boolean closed;
public MarkDistinctOperatorFactory(
@@ -54,16 +51,14 @@ public MarkDistinctOperatorFactory(
List extends Type> sourceTypes,
Collection markDistinctChannels,
Optional hashChannel,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
this.operatorId = operatorId;
this.planNodeId = requireNonNull(planNodeId, "planNodeId is null");
this.markDistinctChannels = ImmutableList.copyOf(requireNonNull(markDistinctChannels, "markDistinctChannels is null"));
checkArgument(!markDistinctChannels.isEmpty(), "markDistinctChannels is empty");
this.hashChannel = requireNonNull(hashChannel, "hashChannel is null");
- this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
- this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
+ this.groupByHashFactory = requireNonNull(groupByHashFactory, "groupByHashFactory is null");
this.types = ImmutableList.builder()
.addAll(sourceTypes)
.add(BOOLEAN)
@@ -75,7 +70,7 @@ public Operator createOperator(DriverContext driverContext)
{
checkState(!closed, "Factory is already closed");
OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, MarkDistinctOperator.class.getSimpleName());
- return new MarkDistinctOperator(operatorContext, types, markDistinctChannels, hashChannel, joinCompiler, blockTypeOperators);
+ return new MarkDistinctOperator(operatorContext, types, markDistinctChannels, hashChannel, groupByHashFactory);
}
@Override
@@ -87,7 +82,7 @@ public void noMoreOperators()
@Override
public OperatorFactory duplicate()
{
- return new MarkDistinctOperatorFactory(operatorId, planNodeId, types.subList(0, types.size() - 1), markDistinctChannels, hashChannel, joinCompiler, blockTypeOperators);
+ return new MarkDistinctOperatorFactory(operatorId, planNodeId, types.subList(0, types.size() - 1), markDistinctChannels, hashChannel, groupByHashFactory);
}
}
@@ -101,7 +96,7 @@ public OperatorFactory duplicate()
// for yield when memory is not available
private Work unfinishedWork;
- public MarkDistinctOperator(OperatorContext operatorContext, List types, List markDistinctChannels, Optional hashChannel, JoinCompiler joinCompiler, BlockTypeOperators blockTypeOperators)
+ public MarkDistinctOperator(OperatorContext operatorContext, List types, List markDistinctChannels, Optional hashChannel, GroupByHashFactory groupByHashFactory)
{
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
@@ -112,7 +107,7 @@ public MarkDistinctOperator(OperatorContext operatorContext, List types, L
for (int channel : markDistinctChannels) {
distinctTypes.add(types.get(channel));
}
- this.markDistinctHash = new MarkDistinctHash(operatorContext.getSession(), distinctTypes.build(), Ints.toArray(markDistinctChannels), hashChannel, joinCompiler, blockTypeOperators, this::updateMemoryReservation);
+ this.markDistinctHash = new MarkDistinctHash(operatorContext.getSession(), distinctTypes.build(), Ints.toArray(markDistinctChannels), hashChannel, groupByHashFactory, this::updateMemoryReservation);
this.localUserMemoryContext = operatorContext.localUserMemoryContext();
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/MultiChannelGroupByHash.java b/core/trino-main/src/main/java/io/trino/operator/MultiChannelGroupByHash.java
index ad7ec9063cc4..e0f5a94aeb84 100644
--- a/core/trino-main/src/main/java/io/trino/operator/MultiChannelGroupByHash.java
+++ b/core/trino-main/src/main/java/io/trino/operator/MultiChannelGroupByHash.java
@@ -252,20 +252,20 @@ public Work getGroupIds(Page page)
}
@Override
- public boolean contains(int position, Page page, int[] hashChannels)
+ public boolean contains(int position, Page page)
{
long rawHash = hashStrategy.hashRow(position, page);
- return contains(position, page, hashChannels, rawHash);
+ return contains(position, page, rawHash);
}
@Override
- public boolean contains(int position, Page page, int[] hashChannels, long rawHash)
+ public boolean contains(int position, Page page, long rawHash)
{
int hashPosition = (int) getHashPosition(rawHash, mask);
// look for a slot containing this key
while (groupAddressByHash[hashPosition] != -1) {
- if (positionNotDistinctFromCurrentRow(groupAddressByHash[hashPosition], hashPosition, position, page, (byte) rawHash, hashChannels)) {
+ if (positionNotDistinctFromCurrentRow(groupAddressByHash[hashPosition], hashPosition, position, page, (byte) rawHash)) {
// found an existing slot for this key
return true;
}
@@ -457,6 +457,14 @@ private boolean positionNotDistinctFromCurrentRow(long address, int hashPosition
return hashStrategy.positionNotDistinctFromRow(decodeSliceIndex(address), decodePosition(address), position, page, hashChannels);
}
+ private boolean positionNotDistinctFromCurrentRow(long address, int hashPosition, int position, Page page, byte rawHash)
+ {
+ if (rawHashByHashPosition[hashPosition] != rawHash) {
+ return false;
+ }
+ return hashStrategy.positionNotDistinctFromRow(decodeSliceIndex(address), decodePosition(address), position, page);
+ }
+
private static long getHashPosition(long rawHash, int mask)
{
return murmurHash3(rawHash) & mask;
@@ -558,7 +566,7 @@ private int registerGroupId(HashGenerator hashGenerator, Page page, int position
return groupId;
}
- private static final class DictionaryLookBack
+ public static final class DictionaryLookBack
{
private final Block dictionary;
private final int[] processed;
diff --git a/core/trino-main/src/main/java/io/trino/operator/NoChannelGroupByHash.java b/core/trino-main/src/main/java/io/trino/operator/NoChannelGroupByHash.java
index fc8d848fb6ff..2597d90aa074 100644
--- a/core/trino-main/src/main/java/io/trino/operator/NoChannelGroupByHash.java
+++ b/core/trino-main/src/main/java/io/trino/operator/NoChannelGroupByHash.java
@@ -83,7 +83,7 @@ public Work getGroupIds(Page page)
}
@Override
- public boolean contains(int position, Page page, int[] hashChannels)
+ public boolean contains(int position, Page page)
{
throw new UnsupportedOperationException("NoChannelGroupByHash does not support getHashCollisions");
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/PrecomputedHashGenerator.java b/core/trino-main/src/main/java/io/trino/operator/PrecomputedHashGenerator.java
index 95c5ec91f2e1..b88802291832 100644
--- a/core/trino-main/src/main/java/io/trino/operator/PrecomputedHashGenerator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/PrecomputedHashGenerator.java
@@ -14,7 +14,7 @@
package io.trino.operator;
import io.trino.spi.Page;
-import io.trino.spi.type.BigintType;
+import io.trino.spi.block.Block;
import static com.google.common.base.MoreObjects.toStringHelper;
@@ -31,7 +31,8 @@ public PrecomputedHashGenerator(int hashChannel)
@Override
public long hashPosition(int position, Page page)
{
- return BigintType.BIGINT.getLong(page.getBlock(hashChannel), position);
+ Block hashBlock = page.getBlock(hashChannel);
+ return hashBlock.getLong(position, 0);
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/operator/RowNumberOperator.java b/core/trino-main/src/main/java/io/trino/operator/RowNumberOperator.java
index 320bfd3447a3..a7e251a3d284 100644
--- a/core/trino-main/src/main/java/io/trino/operator/RowNumberOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/RowNumberOperator.java
@@ -23,9 +23,7 @@
import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.sql.planner.plan.PlanNodeId;
-import io.trino.type.BlockTypeOperators;
import java.util.List;
import java.util.Optional;
@@ -34,7 +32,6 @@
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Verify.verify;
import static com.google.common.base.Verify.verifyNotNull;
-import static io.trino.operator.GroupByHash.createGroupByHash;
import static io.trino.spi.type.BigintType.BIGINT;
import static java.util.Objects.requireNonNull;
@@ -54,8 +51,7 @@ public static class RowNumberOperatorFactory
private final Optional hashChannel;
private final int expectedPositions;
private boolean closed;
- private final JoinCompiler joinCompiler;
- private final BlockTypeOperators blockTypeOperators;
+ private final GroupByHashFactory groupByHashFactory;
public RowNumberOperatorFactory(
int operatorId,
@@ -67,8 +63,7 @@ public RowNumberOperatorFactory(
Optional maxRowsPerPartition,
Optional hashChannel,
int expectedPositions,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
this.operatorId = operatorId;
this.planNodeId = requireNonNull(planNodeId, "planNodeId is null");
@@ -81,8 +76,7 @@ public RowNumberOperatorFactory(
this.hashChannel = requireNonNull(hashChannel, "hashChannel is null");
checkArgument(expectedPositions > 0, "expectedPositions < 0");
this.expectedPositions = expectedPositions;
- this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
- this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
+ this.groupByHashFactory = requireNonNull(groupByHashFactory, "groupByHashFactory is null");
}
@Override
@@ -100,8 +94,7 @@ public Operator createOperator(DriverContext driverContext)
maxRowsPerPartition,
hashChannel,
expectedPositions,
- joinCompiler,
- blockTypeOperators);
+ groupByHashFactory);
}
@Override
@@ -113,7 +106,7 @@ public void noMoreOperators()
@Override
public OperatorFactory duplicate()
{
- return new RowNumberOperatorFactory(operatorId, planNodeId, sourceTypes, outputChannels, partitionChannels, partitionTypes, maxRowsPerPartition, hashChannel, expectedPositions, joinCompiler, blockTypeOperators);
+ return new RowNumberOperatorFactory(operatorId, planNodeId, sourceTypes, outputChannels, partitionChannels, partitionTypes, maxRowsPerPartition, hashChannel, expectedPositions, groupByHashFactory);
}
}
@@ -146,8 +139,7 @@ public RowNumberOperator(
Optional maxRowsPerPartition,
Optional hashChannel,
int expectedPositions,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
this.localUserMemoryContext = operatorContext.localUserMemoryContext();
@@ -168,7 +160,7 @@ public RowNumberOperator(
}
else {
int[] channels = Ints.toArray(partitionChannels);
- this.groupByHash = Optional.of(createGroupByHash(operatorContext.getSession(), partitionTypes, channels, hashChannel, expectedPositions, joinCompiler, blockTypeOperators, this::updateMemoryReservation));
+ this.groupByHash = Optional.of(groupByHashFactory.createGroupByHash(operatorContext.getSession(), partitionTypes, channels, hashChannel, expectedPositions, this::updateMemoryReservation));
}
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/SetBuilderOperator.java b/core/trino-main/src/main/java/io/trino/operator/SetBuilderOperator.java
index 3bdc6a23ddb2..f1981799d60c 100644
--- a/core/trino-main/src/main/java/io/trino/operator/SetBuilderOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/SetBuilderOperator.java
@@ -19,9 +19,7 @@
import io.trino.operator.ChannelSet.ChannelSetBuilder;
import io.trino.spi.Page;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.sql.planner.plan.PlanNodeId;
-import io.trino.type.BlockTypeOperators;
import javax.annotation.Nullable;
import javax.annotation.concurrent.ThreadSafe;
@@ -73,8 +71,7 @@ public static class SetBuilderOperatorFactory
private final int setChannel;
private final int expectedPositions;
private boolean closed;
- private final JoinCompiler joinCompiler;
- private final BlockTypeOperators blockTypeOperators;
+ private final GroupByHashFactory groupByHashFactory;
public SetBuilderOperatorFactory(
int operatorId,
@@ -83,8 +80,7 @@ public SetBuilderOperatorFactory(
int setChannel,
Optional hashChannel,
int expectedPositions,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
this.operatorId = operatorId;
this.planNodeId = requireNonNull(planNodeId, "planNodeId is null");
@@ -93,8 +89,7 @@ public SetBuilderOperatorFactory(
this.setChannel = setChannel;
this.hashChannel = requireNonNull(hashChannel, "hashChannel is null");
this.expectedPositions = expectedPositions;
- this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
- this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
+ this.groupByHashFactory = requireNonNull(groupByHashFactory, "groupByHashFactory is null");
}
public SetSupplier getSetProvider()
@@ -107,7 +102,7 @@ public Operator createOperator(DriverContext driverContext)
{
checkState(!closed, "Factory is already closed");
OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, SetBuilderOperator.class.getSimpleName());
- return new SetBuilderOperator(operatorContext, setProvider, setChannel, hashChannel, expectedPositions, joinCompiler, blockTypeOperators);
+ return new SetBuilderOperator(operatorContext, setProvider, setChannel, hashChannel, expectedPositions, groupByHashFactory);
}
@Override
@@ -119,7 +114,7 @@ public void noMoreOperators()
@Override
public OperatorFactory duplicate()
{
- return new SetBuilderOperatorFactory(operatorId, planNodeId, setProvider.getType(), setChannel, hashChannel, expectedPositions, joinCompiler, blockTypeOperators);
+ return new SetBuilderOperatorFactory(operatorId, planNodeId, setProvider.getType(), setChannel, hashChannel, expectedPositions, groupByHashFactory);
}
}
@@ -140,8 +135,7 @@ public SetBuilderOperator(
int setChannel,
Optional hashChannel,
int expectedPositions,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
this.setSupplier = requireNonNull(setSupplier, "setSupplier is null");
@@ -152,15 +146,12 @@ public SetBuilderOperator(
else {
this.sourceChannels = new int[] {setChannel};
}
- // Set builder is has a single channel which goes in channel 0, if hash is present, add a hachBlock to channel 1
- Optional channelSetHashChannel = hashChannel.isPresent() ? Optional.of(1) : Optional.empty();
this.channelSetBuilder = new ChannelSetBuilder(
setSupplier.getType(),
- channelSetHashChannel,
+ hashChannel.isPresent(),
expectedPositions,
requireNonNull(operatorContext, "operatorContext is null"),
- requireNonNull(joinCompiler, "joinCompiler is null"),
- requireNonNull(blockTypeOperators, "blockTypeOperators is null"));
+ requireNonNull(groupByHashFactory, "groupByHashFactory is null"));
}
@Override
diff --git a/core/trino-main/src/main/java/io/trino/operator/TopNRankingOperator.java b/core/trino-main/src/main/java/io/trino/operator/TopNRankingOperator.java
index b39b624cd4b4..91668fc3ba55 100644
--- a/core/trino-main/src/main/java/io/trino/operator/TopNRankingOperator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/TopNRankingOperator.java
@@ -25,7 +25,6 @@
import io.trino.spi.connector.SortOrder;
import io.trino.spi.type.Type;
import io.trino.spi.type.TypeOperators;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.sql.planner.plan.PlanNodeId;
import io.trino.sql.planner.plan.TopNRankingNode.RankingType;
import io.trino.type.BlockTypeOperators;
@@ -36,7 +35,6 @@
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
-import static io.trino.operator.GroupByHash.createGroupByHash;
import static java.util.Objects.requireNonNull;
public class TopNRankingOperator
@@ -63,7 +61,7 @@ public static class TopNRankingOperatorFactory
private final boolean generateRanking;
private boolean closed;
- private final JoinCompiler joinCompiler;
+ private final GroupByHashFactory groupByHashFactory;
private final TypeOperators typeOperators;
private final BlockTypeOperators blockTypeOperators;
private final Optional maxPartialMemory;
@@ -83,7 +81,7 @@ public TopNRankingOperatorFactory(
Optional hashChannel,
int expectedPositions,
Optional maxPartialMemory,
- JoinCompiler joinCompiler,
+ GroupByHashFactory groupByHashFactory,
TypeOperators typeOperators,
BlockTypeOperators blockTypeOperators)
{
@@ -103,7 +101,7 @@ public TopNRankingOperatorFactory(
checkArgument(expectedPositions > 0, "expectedPositions must be > 0");
this.generateRanking = !partial;
this.expectedPositions = expectedPositions;
- this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
+ this.groupByHashFactory = requireNonNull(groupByHashFactory, "groupByHashFactory is null");
this.typeOperators = requireNonNull(typeOperators, "typeOperators is null");
this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
this.maxPartialMemory = requireNonNull(maxPartialMemory, "maxPartialMemory is null");
@@ -128,7 +126,7 @@ public Operator createOperator(DriverContext driverContext)
hashChannel,
expectedPositions,
maxPartialMemory,
- joinCompiler,
+ groupByHashFactory,
typeOperators,
blockTypeOperators);
}
@@ -157,7 +155,7 @@ public OperatorFactory duplicate()
hashChannel,
expectedPositions,
maxPartialMemory,
- joinCompiler,
+ groupByHashFactory,
typeOperators,
blockTypeOperators);
}
@@ -190,7 +188,7 @@ public TopNRankingOperator(
Optional hashChannel,
int expectedPositions,
Optional maxPartialMemory,
- JoinCompiler joinCompiler,
+ GroupByHashFactory groupByHashFactory,
TypeOperators typeOperators,
BlockTypeOperators blockTypeOperators)
{
@@ -228,8 +226,7 @@ public TopNRankingOperator(
partitionTypes,
hashChannel,
operatorContext.getSession(),
- joinCompiler,
- blockTypeOperators,
+ groupByHashFactory,
this::updateMemoryReservation));
}
@@ -239,8 +236,7 @@ private static Supplier getGroupByHashSupplier(
List partitionTypes,
Optional hashChannel,
Session session,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators,
+ GroupByHashFactory groupByHashFactory,
UpdateMemory updateMemory)
{
if (partitionChannels.isEmpty()) {
@@ -249,14 +245,12 @@ private static Supplier getGroupByHashSupplier(
else {
checkArgument(expectedPositions > 0, "expectedPositions must be > 0");
int[] channels = Ints.toArray(partitionChannels);
- return () -> createGroupByHash(
+ return () -> groupByHashFactory.createGroupByHash(
session,
partitionTypes,
channels,
hashChannel,
expectedPositions,
- joinCompiler,
- blockTypeOperators,
updateMemory);
}
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/AggregatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/AggregatorFactory.java
index 968162f39347..9c1ea2032f1d 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/AggregatorFactory.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/AggregatorFactory.java
@@ -31,6 +31,8 @@ public class AggregatorFactory
private final Type intermediateType;
private final Type finalType;
private final List inputChannels;
+ private final int intermediateStateChannel;
+ private final OptionalInt useRawInputChannel;
private final OptionalInt maskChannel;
private final boolean spillable;
private final List> lambdaProviders;
@@ -41,6 +43,7 @@ public AggregatorFactory(
Type intermediateType,
Type finalType,
List inputChannels,
+ OptionalInt useRawInputChannel,
OptionalInt maskChannel,
boolean spillable,
List> lambdaProviders)
@@ -49,12 +52,21 @@ public AggregatorFactory(
this.step = requireNonNull(step, "step is null");
this.intermediateType = requireNonNull(intermediateType, "intermediateType is null");
this.finalType = requireNonNull(finalType, "finalType is null");
- this.inputChannels = ImmutableList.copyOf(requireNonNull(inputChannels, "inputChannels is null"));
+ requireNonNull(inputChannels, "inputChannels is null");
+ if (step.isInputRaw()) {
+ intermediateStateChannel = -1;
+ this.inputChannels = ImmutableList.copyOf(inputChannels);
+ }
+ else {
+ intermediateStateChannel = inputChannels.get(0);
+ this.inputChannels = ImmutableList.copyOf(inputChannels.subList(1, inputChannels.size()));
+ }
+ this.useRawInputChannel = requireNonNull(useRawInputChannel, "useRawInputChannel is null");
this.maskChannel = requireNonNull(maskChannel, "maskChannel is null");
this.spillable = spillable;
this.lambdaProviders = ImmutableList.copyOf(requireNonNull(lambdaProviders, "lambdaProviders is null"));
- checkArgument(step.isInputRaw() || inputChannels.size() == 1, "expected 1 input channel for intermediate aggregation");
+ checkArgument(step.isInputRaw() || intermediateStateChannel != -1, "expected intermediateStateChannel for intermediate aggregation but got %s ", intermediateStateChannel);
}
public Aggregator createAggregator()
@@ -66,7 +78,8 @@ public Aggregator createAggregator()
else {
accumulator = accumulatorFactory.createIntermediateAccumulator(lambdaProviders);
}
- return new Aggregator(accumulator, step, intermediateType, finalType, inputChannels, maskChannel);
+ List aggregatorInputChannels = intermediateStateChannel == -1 ? inputChannels : ImmutableList.of(intermediateStateChannel);
+ return new Aggregator(accumulator, step, intermediateType, finalType, aggregatorInputChannels, maskChannel);
}
public GroupedAggregator createGroupedAggregator()
@@ -78,7 +91,7 @@ public GroupedAggregator createGroupedAggregator()
else {
accumulator = accumulatorFactory.createGroupedIntermediateAccumulator(lambdaProviders);
}
- return new GroupedAggregator(accumulator, step, intermediateType, finalType, inputChannels, maskChannel);
+ return new GroupedAggregator(accumulator, step, intermediateType, finalType, inputChannels, intermediateStateChannel, useRawInputChannel, maskChannel);
}
public GroupedAggregator createUnspillGroupedAggregator(Step step, int inputChannel)
@@ -90,11 +103,16 @@ public GroupedAggregator createUnspillGroupedAggregator(Step step, int inputChan
else {
accumulator = accumulatorFactory.createGroupedIntermediateAccumulator(lambdaProviders);
}
- return new GroupedAggregator(accumulator, step, intermediateType, finalType, ImmutableList.of(inputChannel), maskChannel);
+ return new GroupedAggregator(accumulator, step, intermediateType, finalType, ImmutableList.of(inputChannel), inputChannel, OptionalInt.empty(), maskChannel);
}
public boolean isSpillable()
{
return spillable;
}
+
+ public OptionalInt getMaskChannel()
+ {
+ return maskChannel;
+ }
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/DistinctAccumulatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/DistinctAccumulatorFactory.java
index aad714ab3def..9d91f9c5b46a 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/DistinctAccumulatorFactory.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/DistinctAccumulatorFactory.java
@@ -15,6 +15,7 @@
import com.google.common.collect.ImmutableList;
import io.trino.Session;
+import io.trino.operator.GroupByHashFactory;
import io.trino.operator.GroupByIdBlock;
import io.trino.operator.MarkDistinctHash;
import io.trino.operator.UpdateMemory;
@@ -24,8 +25,6 @@
import io.trino.spi.block.BlockBuilder;
import io.trino.spi.block.RunLengthEncodedBlock;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
-import io.trino.type.BlockTypeOperators;
import java.util.List;
import java.util.Optional;
@@ -42,21 +41,18 @@ public class DistinctAccumulatorFactory
{
private final AccumulatorFactory delegate;
private final List argumentTypes;
- private final JoinCompiler joinCompiler;
- private final BlockTypeOperators blockTypeOperators;
+ private final GroupByHashFactory groupByHashFactory;
private final Session session;
public DistinctAccumulatorFactory(
AccumulatorFactory delegate,
List argumentTypes,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators,
+ GroupByHashFactory groupByHashFactory,
Session session)
{
this.delegate = requireNonNull(delegate, "delegate is null");
this.argumentTypes = ImmutableList.copyOf(requireNonNull(argumentTypes, "argumentTypes is null"));
- this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
- this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
+ this.groupByHashFactory = requireNonNull(groupByHashFactory, "groupByHashFactory is null");
this.session = requireNonNull(session, "session is null");
}
@@ -73,8 +69,7 @@ public Accumulator createAccumulator(List> lambdaProviders)
delegate.createAccumulator(lambdaProviders),
argumentTypes,
session,
- joinCompiler,
- blockTypeOperators);
+ groupByHashFactory);
}
@Override
@@ -90,8 +85,7 @@ public GroupedAccumulator createGroupedAccumulator(List> lambda
delegate.createGroupedAccumulator(lambdaProviders),
argumentTypes,
session,
- joinCompiler,
- blockTypeOperators);
+ groupByHashFactory);
}
@Override
@@ -110,8 +104,7 @@ private DistinctAccumulator(
Accumulator accumulator,
List inputTypes,
Session session,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
this.accumulator = requireNonNull(accumulator, "accumulator is null");
this.hash = new MarkDistinctHash(
@@ -119,8 +112,7 @@ private DistinctAccumulator(
inputTypes,
IntStream.range(0, inputTypes.size()).toArray(),
Optional.empty(),
- joinCompiler,
- blockTypeOperators,
+ groupByHashFactory,
UpdateMemory.NOOP);
}
@@ -186,8 +178,7 @@ private DistinctGroupedAccumulator(
GroupedAccumulator accumulator,
List inputTypes,
Session session,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
this.accumulator = requireNonNull(accumulator, "accumulator is null");
this.hash = new MarkDistinctHash(
@@ -198,8 +189,7 @@ private DistinctGroupedAccumulator(
.build(),
IntStream.range(0, inputTypes.size() + 1).toArray(),
Optional.empty(),
- joinCompiler,
- blockTypeOperators,
+ groupByHashFactory,
UpdateMemory.NOOP);
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/GroupedAggregator.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/GroupedAggregator.java
index 05f43e2b38b3..ef5739729f7e 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/GroupedAggregator.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/GroupedAggregator.java
@@ -18,9 +18,12 @@
import io.trino.spi.Page;
import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
+import io.trino.spi.block.ByteArrayBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
import io.trino.spi.type.Type;
import io.trino.sql.planner.plan.AggregationNode;
import io.trino.sql.planner.plan.AggregationNode.Step;
+import org.apache.commons.math3.util.Pair;
import java.util.List;
import java.util.Optional;
@@ -36,17 +39,22 @@ public class GroupedAggregator
private final Type intermediateType;
private final Type finalType;
private final int[] inputChannels;
+ private final int intermediateStateChannel;
+ private final OptionalInt useRawInputChannel;
private final OptionalInt maskChannel;
- public GroupedAggregator(GroupedAccumulator accumulator, Step step, Type intermediateType, Type finalType, List inputChannels, OptionalInt maskChannel)
+ public GroupedAggregator(GroupedAccumulator accumulator, Step step, Type intermediateType, Type finalType, List inputChannels, int intermediateStateChannel, OptionalInt useRawInputChannel, OptionalInt maskChannel)
{
this.accumulator = requireNonNull(accumulator, "accumulator is null");
this.step = requireNonNull(step, "step is null");
this.intermediateType = requireNonNull(intermediateType, "intermediateType is null");
this.finalType = requireNonNull(finalType, "finalType is null");
this.inputChannels = Ints.toArray(requireNonNull(inputChannels, "inputChannels is null"));
+ this.intermediateStateChannel = intermediateStateChannel;
+ this.useRawInputChannel = requireNonNull(useRawInputChannel, "useRawInputChannel is null");
this.maskChannel = requireNonNull(maskChannel, "maskChannel is null");
- checkArgument(step.isInputRaw() || inputChannels.size() == 1, "expected 1 input channel for intermediate aggregation");
+// checkArgument(step.isInputRaw() || inputChannels.size() == 1, "expected 1 input channel for intermediate aggregation");
+ checkArgument(step.isInputRaw() || intermediateStateChannel != -1, "expected intermediateStateChannel for intermediate aggregation but got %s ", intermediateStateChannel);
}
public long getEstimatedSize()
@@ -68,10 +76,42 @@ public void processPage(GroupByIdBlock groupIds, Page page)
{
if (step.isInputRaw()) {
accumulator.addInput(groupIds, page.getColumns(inputChannels), getMaskBlock(page));
+ return;
}
- else {
- accumulator.addIntermediate(groupIds, page.getBlock(inputChannels[0]));
+
+ if (useRawInputChannel.isEmpty()) {
+ // process grouped data
+ accumulator.addIntermediate(groupIds, page.getBlock(intermediateStateChannel));
+ return;
+ }
+ Block useRawInputBlock = page.getBlock(useRawInputChannel.getAsInt());
+ if (useRawInputBlock instanceof RunLengthEncodedBlock) {
+ if (useRawInputBlock.isNull(0)) {
+ // process grouped data
+ accumulator.addIntermediate(groupIds, page.getBlock(intermediateStateChannel));
+ }
+ else {
+ // process raw data
+ accumulator.addInput(groupIds, page.getColumns(inputChannels), getMaskBlock(page));
+ }
+ return;
+ }
+
+ // useRawInputBlock has potentially mixed grouped and raw data
+ Optional maskBlock = Optional.of((getMaskBlock(page).map(mask -> andMasks(mask, useRawInputBlock)).orElse(useRawInputBlock)));
+ accumulator.addInput(groupIds, page.getColumns(inputChannels), maskBlock);
+ Pair filtered = filterByNull(page.getBlock(intermediateStateChannel), groupIds, useRawInputBlock);
+ accumulator.addIntermediate(filtered.getSecond(), filtered.getFirst());
+ }
+
+ private Block andMasks(Block mask1, Block mask2)
+ {
+ int positionCount = mask1.getPositionCount();
+ byte[] mask = new byte[positionCount];
+ for (int i = 0; i < positionCount; i++) {
+ mask[i] = (byte) ((!mask1.isNull(i) && mask1.getByte(i, 0) == 1 && !mask2.isNull(i) && mask2.getByte(i, 0) == 1) ? 1 : 0);
}
+ return new ByteArrayBlock(positionCount, Optional.empty(), mask);
}
private Optional getMaskBlock(Page page)
@@ -107,4 +147,24 @@ public Type getSpillType()
{
return intermediateType;
}
+
+ private static Pair filterByNull(Block block, GroupByIdBlock groupByIdBlock, Block mask)
+ {
+ int positions = mask.getPositionCount();
+
+ int[] ids = new int[positions];
+ int next = 0;
+ for (int i = 0; i < ids.length; ++i) {
+ if (mask.isNull(i)) {
+ ids[next++] = i;
+ }
+ }
+
+ if (next == ids.length) {
+ return Pair.create(block, groupByIdBlock); // no rows were eliminated by the filter
+ }
+ return Pair.create(
+ block.getPositions(ids, 0, next),
+ new GroupByIdBlock(groupByIdBlock.getGroupCount(), groupByIdBlock.getPositions(ids, 0, next)));
+ }
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/InMemoryHashAggregationBuilder.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/InMemoryHashAggregationBuilder.java
index d4cd20cd0944..31a3c49b9bcc 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/InMemoryHashAggregationBuilder.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/InMemoryHashAggregationBuilder.java
@@ -20,22 +20,22 @@
import io.airlift.units.DataSize;
import io.trino.array.IntBigArray;
import io.trino.operator.GroupByHash;
+import io.trino.operator.GroupByHashFactory;
import io.trino.operator.HashCollisionsCounter;
import io.trino.operator.OperatorContext;
import io.trino.operator.TransformWork;
import io.trino.operator.UpdateMemory;
import io.trino.operator.Work;
import io.trino.operator.WorkProcessor;
-import io.trino.operator.WorkProcessor.ProcessState;
import io.trino.operator.aggregation.AggregatorFactory;
import io.trino.operator.aggregation.GroupedAggregator;
import io.trino.spi.Page;
import io.trino.spi.PageBuilder;
+import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
+import io.trino.spi.block.RunLengthEncodedBlock;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.sql.planner.plan.AggregationNode.Step;
-import io.trino.type.BlockTypeOperators;
import it.unimi.dsi.fastutil.ints.AbstractIntIterator;
import it.unimi.dsi.fastutil.ints.IntIterator;
import it.unimi.dsi.fastutil.ints.IntIterators;
@@ -45,8 +45,8 @@
import java.util.Optional;
import java.util.OptionalLong;
-import static io.trino.operator.GroupByHash.createGroupByHash;
import static io.trino.spi.type.BigintType.BIGINT;
+import static io.trino.spi.type.BooleanType.BOOLEAN;
import static java.util.Objects.requireNonNull;
public class InMemoryHashAggregationBuilder
@@ -57,6 +57,8 @@ public class InMemoryHashAggregationBuilder
private final boolean partial;
private final OptionalLong maxPartialMemory;
private final UpdateMemory updateMemory;
+ private final List aggregationInputTypes;
+ private final int maskChannelCount;
private boolean full;
@@ -66,11 +68,12 @@ public InMemoryHashAggregationBuilder(
int expectedGroups,
List groupByTypes,
List groupByChannels,
+ List aggregationInputTypes,
+ int maskChannelCount,
Optional hashChannel,
OperatorContext operatorContext,
Optional maxPartialMemory,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators,
+ GroupByHashFactory groupByHashFactory,
UpdateMemory updateMemory)
{
this(aggregatorFactories,
@@ -78,12 +81,13 @@ public InMemoryHashAggregationBuilder(
expectedGroups,
groupByTypes,
groupByChannels,
+ aggregationInputTypes,
+ maskChannelCount,
hashChannel,
operatorContext,
maxPartialMemory,
Optional.empty(),
- joinCompiler,
- blockTypeOperators,
+ groupByHashFactory,
updateMemory);
}
@@ -93,22 +97,23 @@ public InMemoryHashAggregationBuilder(
int expectedGroups,
List groupByTypes,
List groupByChannels,
+ List aggregationInputTypes,
+ int maskChannelCount,
Optional hashChannel,
OperatorContext operatorContext,
Optional maxPartialMemory,
Optional unspillIntermediateChannelOffset,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators,
+ GroupByHashFactory groupByHashFactory,
UpdateMemory updateMemory)
{
- this.groupByHash = createGroupByHash(
+ this.aggregationInputTypes = ImmutableList.copyOf(aggregationInputTypes);
+ this.maskChannelCount = maskChannelCount;
+ this.groupByHash = groupByHashFactory.createGroupByHash(
operatorContext.getSession(),
groupByTypes,
Ints.toArray(groupByChannels),
hashChannel,
expectedGroups,
- joinCompiler,
- blockTypeOperators,
updateMemory);
this.partial = step.isOutputPartial();
this.maxPartialMemory = maxPartialMemory.map(dataSize -> OptionalLong.of(dataSize.toBytes())).orElseGet(OptionalLong::empty);
@@ -257,6 +262,10 @@ public List buildSpillTypes()
for (GroupedAggregator groupedAggregator : groupedAggregators) {
types.add(groupedAggregator.getSpillType());
}
+ if (partial) {
+ types.addAll(aggregationInputTypes);
+ types.add(BOOLEAN);
+ }
return types;
}
@@ -269,29 +278,34 @@ public int getCapacity()
private WorkProcessor buildResult(IntIterator groupIds)
{
PageBuilder pageBuilder = new PageBuilder(buildTypes());
- return WorkProcessor.create(() -> {
- if (!groupIds.hasNext()) {
- return ProcessState.finished();
- }
-
- pageBuilder.reset();
-
- List types = groupByHash.getTypes();
- while (!pageBuilder.isFull() && groupIds.hasNext()) {
- int groupId = groupIds.nextInt();
-
- groupByHash.appendValuesTo(groupId, pageBuilder);
+ return groupByHash.buildResult(groupIds, pageBuilder, groupedAggregators)
+ .map(page -> {
+ if (partial) {
+ // only from partial step output raw input columns
+ Block[] finalPage = new Block[page.getChannelCount() + maskChannelCount + aggregationInputTypes.size() + 1];
+ for (int i = 0; i < page.getChannelCount(); i++) {
+ finalPage[i] = page.getBlock(i);
+ }
+ int positionCount = page.getPositionCount();
+ for (int i = 0; i < maskChannelCount; i++) {
+ finalPage[page.getChannelCount() + i] = nullRle(BOOLEAN, positionCount);
+ }
+ for (int i = 0; i < aggregationInputTypes.size(); i++) {
+ finalPage[page.getChannelCount() + maskChannelCount + i] = nullRle(aggregationInputTypes.get(i), positionCount);
+ }
+ finalPage[finalPage.length - 1] = nullRle(BOOLEAN, positionCount);
- pageBuilder.declarePosition();
- for (int i = 0; i < groupedAggregators.size(); i++) {
- GroupedAggregator groupedAggregator = groupedAggregators.get(i);
- BlockBuilder output = pageBuilder.getBlockBuilder(types.size() + i);
- groupedAggregator.evaluate(groupId, output);
- }
- }
+ page = Page.wrapBlocksWithoutCopy(positionCount, finalPage);
+ }
+ return page;
+ });
+ }
- return ProcessState.ofResult(pageBuilder.build());
- });
+ public static RunLengthEncodedBlock nullRle(Type type, int positionCount)
+ {
+ return new RunLengthEncodedBlock(
+ type.createBlockBuilder(null, 1).appendNull().build(),
+ positionCount);
}
public List buildTypes()
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/MergingHashAggregationBuilder.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/MergingHashAggregationBuilder.java
index ec53d872f881..b42a82072226 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/MergingHashAggregationBuilder.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/MergingHashAggregationBuilder.java
@@ -17,6 +17,7 @@
import io.airlift.units.DataSize;
import io.trino.memory.context.AggregatedMemoryContext;
import io.trino.memory.context.LocalMemoryContext;
+import io.trino.operator.GroupByHashFactory;
import io.trino.operator.OperatorContext;
import io.trino.operator.WorkProcessor;
import io.trino.operator.WorkProcessor.Transformation;
@@ -24,9 +25,7 @@
import io.trino.operator.aggregation.AggregatorFactory;
import io.trino.spi.Page;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.sql.planner.plan.AggregationNode;
-import io.trino.type.BlockTypeOperators;
import java.io.Closeable;
import java.util.List;
@@ -46,25 +45,27 @@ public class MergingHashAggregationBuilder
private final WorkProcessor sortedPages;
private InMemoryHashAggregationBuilder hashAggregationBuilder;
private final List groupByTypes;
+ private final List aggregationInputTypes;
+ private final int maskChannelCount;
private final LocalMemoryContext memoryContext;
private final long memoryLimitForMerge;
private final int overwriteIntermediateChannelOffset;
- private final JoinCompiler joinCompiler;
- private final BlockTypeOperators blockTypeOperators;
+ private final GroupByHashFactory groupByHashFactory;
public MergingHashAggregationBuilder(
List aggregatorFactories,
AggregationNode.Step step,
int expectedGroups,
List groupByTypes,
+ List aggregationInputTypes,
+ int maskChannelCount,
Optional hashChannel,
OperatorContext operatorContext,
WorkProcessor sortedPages,
AggregatedMemoryContext aggregatedMemoryContext,
long memoryLimitForMerge,
int overwriteIntermediateChannelOffset,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
ImmutableList.Builder groupByPartialChannels = ImmutableList.builder();
for (int i = 0; i < groupByTypes.size(); i++) {
@@ -79,11 +80,12 @@ public MergingHashAggregationBuilder(
this.operatorContext = operatorContext;
this.sortedPages = sortedPages;
this.groupByTypes = groupByTypes;
+ this.aggregationInputTypes = aggregationInputTypes;
+ this.maskChannelCount = maskChannelCount;
this.memoryContext = aggregatedMemoryContext.newLocalMemoryContext(MergingHashAggregationBuilder.class.getSimpleName());
this.memoryLimitForMerge = memoryLimitForMerge;
this.overwriteIntermediateChannelOffset = overwriteIntermediateChannelOffset;
- this.joinCompiler = joinCompiler;
- this.blockTypeOperators = blockTypeOperators;
+ this.groupByHashFactory = groupByHashFactory;
rebuildHashAggregationBuilder();
}
@@ -149,12 +151,13 @@ private void rebuildHashAggregationBuilder()
expectedGroups,
groupByTypes,
groupByPartialChannels,
+ aggregationInputTypes,
+ maskChannelCount,
hashChannel,
operatorContext,
Optional.of(DataSize.succinctBytes(0)),
Optional.of(overwriteIntermediateChannelOffset),
- joinCompiler,
- blockTypeOperators,
+ groupByHashFactory,
// TODO: merging should also yield on memory reservations
() -> true);
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/SpillableHashAggregationBuilder.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/SpillableHashAggregationBuilder.java
index da7ad0f8461a..6480ac6392eb 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/SpillableHashAggregationBuilder.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/builder/SpillableHashAggregationBuilder.java
@@ -18,6 +18,7 @@
import com.google.common.util.concurrent.ListenableFuture;
import io.airlift.units.DataSize;
import io.trino.memory.context.LocalMemoryContext;
+import io.trino.operator.GroupByHashFactory;
import io.trino.operator.HashCollisionsCounter;
import io.trino.operator.MergeHashSort;
import io.trino.operator.OperatorContext;
@@ -28,11 +29,11 @@
import io.trino.spi.type.Type;
import io.trino.spiller.Spiller;
import io.trino.spiller.SpillerFactory;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.sql.planner.plan.AggregationNode;
import io.trino.type.BlockTypeOperators;
import java.io.IOException;
+import java.util.Iterator;
import java.util.List;
import java.util.Optional;
@@ -54,6 +55,8 @@ public class SpillableHashAggregationBuilder
private final int expectedGroups;
private final List groupByTypes;
private final List groupByChannels;
+ private final List aggregationInputTypes;
+ private final int maskChannelCount;
private final Optional hashChannel;
private final OperatorContext operatorContext;
private final LocalMemoryContext localUserMemoryContext;
@@ -64,7 +67,7 @@ public class SpillableHashAggregationBuilder
private Optional merger = Optional.empty();
private Optional mergeHashSort = Optional.empty();
private ListenableFuture spillInProgress = immediateVoidFuture();
- private final JoinCompiler joinCompiler;
+ private final GroupByHashFactory groupByHashFactory;
private final BlockTypeOperators blockTypeOperators;
// todo get rid of that and only use revocable memory
@@ -80,12 +83,14 @@ public SpillableHashAggregationBuilder(
int expectedGroups,
List groupByTypes,
List groupByChannels,
+ List aggregationInputTypes,
+ int maskChannelCount,
Optional hashChannel,
OperatorContext operatorContext,
DataSize memoryLimitForMerge,
DataSize memoryLimitForMergeWithMemory,
SpillerFactory spillerFactory,
- JoinCompiler joinCompiler,
+ GroupByHashFactory groupByHashFactory,
BlockTypeOperators blockTypeOperators)
{
this.aggregatorFactories = aggregatorFactories;
@@ -93,6 +98,8 @@ public SpillableHashAggregationBuilder(
this.expectedGroups = expectedGroups;
this.groupByTypes = groupByTypes;
this.groupByChannels = groupByChannels;
+ this.aggregationInputTypes = aggregationInputTypes;
+ this.maskChannelCount = maskChannelCount;
this.hashChannel = hashChannel;
this.operatorContext = operatorContext;
this.localUserMemoryContext = operatorContext.localUserMemoryContext();
@@ -100,7 +107,7 @@ public SpillableHashAggregationBuilder(
this.memoryLimitForMerge = memoryLimitForMerge.toBytes();
this.memoryLimitForMergeWithMemory = memoryLimitForMergeWithMemory.toBytes();
this.spillerFactory = spillerFactory;
- this.joinCompiler = joinCompiler;
+ this.groupByHashFactory = groupByHashFactory;
this.blockTypeOperators = blockTypeOperators;
rebuildHashAggregationBuilder();
@@ -257,7 +264,9 @@ private ListenableFuture spillToDisk()
}
// start spilling process with current content of the hashAggregationBuilder builder...
- spillInProgress = spiller.get().spill(hashAggregationBuilder.buildHashSortedResult().iterator());
+ Iterator pageIterator = hashAggregationBuilder.buildHashSortedResult().iterator();
+ ImmutableList pages = ImmutableList.copyOf(pageIterator);
+ spillInProgress = spiller.get().spill(pages.iterator());
// ... and immediately create new hashAggregationBuilder so effectively memory ownership
// over hashAggregationBuilder is transferred from this thread to a spilling thread
rebuildHashAggregationBuilder();
@@ -310,14 +319,15 @@ private WorkProcessor mergeSortedPages(WorkProcessor sortedPages, lo
step,
expectedGroups,
groupByTypes,
+ aggregationInputTypes,
+ maskChannelCount,
hashChannel,
operatorContext,
sortedPages,
operatorContext.aggregateUserMemoryContext(),
memoryLimitForMerge,
hashAggregationBuilder.getKeyChannels(),
- joinCompiler,
- blockTypeOperators));
+ groupByHashFactory));
return merger.get().buildResult();
}
@@ -336,11 +346,12 @@ private void rebuildHashAggregationBuilder()
expectedGroups,
groupByTypes,
groupByChannels,
+ aggregationInputTypes,
+ maskChannelCount,
hashChannel,
operatorContext,
Optional.of(DataSize.succinctBytes(0)),
- joinCompiler,
- blockTypeOperators,
+ groupByHashFactory,
() -> {
updateMemory();
// TODO: Support GroupByHash yielding in spillable hash aggregation (https://github.com/trinodb/trino/issues/460)
diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/partial/SkipAggregationBuilder.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/partial/SkipAggregationBuilder.java
index 3e9f00b9ad8d..d5fc29cc0aae 100644
--- a/core/trino-main/src/main/java/io/trino/operator/aggregation/partial/SkipAggregationBuilder.java
+++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/partial/SkipAggregationBuilder.java
@@ -13,10 +13,10 @@
*/
package io.trino.operator.aggregation.partial;
+import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.ListenableFuture;
import io.trino.memory.context.LocalMemoryContext;
import io.trino.operator.CompletedWork;
-import io.trino.operator.GroupByIdBlock;
import io.trino.operator.HashCollisionsCounter;
import io.trino.operator.Work;
import io.trino.operator.WorkProcessor;
@@ -26,7 +26,7 @@
import io.trino.spi.Page;
import io.trino.spi.block.Block;
import io.trino.spi.block.BlockBuilder;
-import io.trino.spi.block.LongArrayBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
import javax.annotation.Nullable;
@@ -35,6 +35,8 @@
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.collect.ImmutableList.toImmutableList;
+import static io.trino.operator.aggregation.builder.InMemoryHashAggregationBuilder.nullRle;
+import static io.trino.spi.type.BooleanType.BOOLEAN;
import static java.util.Objects.requireNonNull;
/**
@@ -50,18 +52,24 @@ public class SkipAggregationBuilder
@Nullable
private Page currentPage;
private final int[] hashChannels;
+ private final int[] aggregationInputChannels;
+ private final int[] maskBlockChannels;
public SkipAggregationBuilder(
List groupByChannels,
Optional inputHashChannel,
List aggregatorFactories,
- LocalMemoryContext memoryContext)
+ LocalMemoryContext memoryContext,
+ List aggregationInputChannels,
+ List maskBlockChannels)
{
this.memoryContext = requireNonNull(memoryContext, "memoryContext is null");
this.groupedAggregators = requireNonNull(aggregatorFactories, "aggregatorFactories is null")
.stream()
.map(AggregatorFactory::createGroupedAggregator)
.collect(toImmutableList());
+ this.aggregationInputChannels = Ints.toArray(aggregationInputChannels);
+ this.maskBlockChannels = Ints.toArray(maskBlockChannels);
this.hashChannels = new int[groupByChannels.size() + (inputHashChannel.isPresent() ? 1 : 0)];
for (int i = 0; i < groupByChannels.size(); i++) {
hashChannels[i] = groupByChannels.get(i);
@@ -128,63 +136,29 @@ public void finishMemoryRevoke()
private Page buildOutputPage(Page page)
{
- populateInitialAccumulatorState(page);
-
- BlockBuilder[] outputBuilders = serializeAccumulatorState(page.getPositionCount());
-
- return constructOutputPage(page, outputBuilders);
- }
-
- private void populateInitialAccumulatorState(Page page)
- {
- GroupByIdBlock groupByIdBlock = getGroupByIdBlock(page.getPositionCount());
- for (GroupedAggregator groupedAggregator : groupedAggregators) {
- groupedAggregator.processPage(groupByIdBlock, page);
+ Block[] outputBlocks = new Block[hashChannels.length + groupedAggregators.size() + maskBlockChannels.length + aggregationInputChannels.length + 1];
+ int blockOffset = 0;
+ for (int i = 0; i < hashChannels.length; i++, blockOffset++) {
+ outputBlocks[blockOffset] = page.getBlock(hashChannels[i]);
}
- }
-
- private GroupByIdBlock getGroupByIdBlock(int positionCount)
- {
- return new GroupByIdBlock(
- positionCount,
- new LongArrayBlock(positionCount, Optional.empty(), consecutive(positionCount)));
- }
-
- private BlockBuilder[] serializeAccumulatorState(int positionCount)
- {
- BlockBuilder[] outputBuilders = new BlockBuilder[groupedAggregators.size()];
- for (int i = 0; i < outputBuilders.length; i++) {
- outputBuilders[i] = groupedAggregators.get(i).getType().createBlockBuilder(null, positionCount);
- }
-
- for (int position = 0; position < positionCount; position++) {
- for (int i = 0; i < groupedAggregators.size(); i++) {
- GroupedAggregator groupedAggregator = groupedAggregators.get(i);
- BlockBuilder output = outputBuilders[i];
- groupedAggregator.evaluate(position, output);
- }
+ for (int i = 0; i < groupedAggregators.size(); i++, blockOffset++) {
+ outputBlocks[blockOffset] = nullRle(groupedAggregators.get(i).getType(), page.getPositionCount());
}
- return outputBuilders;
- }
-
- private Page constructOutputPage(Page page, BlockBuilder[] outputBuilders)
- {
- Block[] outputBlocks = new Block[hashChannels.length + outputBuilders.length];
- for (int i = 0; i < hashChannels.length; i++) {
- outputBlocks[i] = page.getBlock(hashChannels[i]);
+ for (int i = 0; i < maskBlockChannels.length; i++, blockOffset++) {
+ outputBlocks[blockOffset] = page.getBlock(maskBlockChannels[i]);
}
- for (int i = 0; i < outputBuilders.length; i++) {
- outputBlocks[hashChannels.length + i] = outputBuilders[i].build();
+ for (int i = 0; i < aggregationInputChannels.length; i++, blockOffset++) {
+ outputBlocks[blockOffset] = page.getBlock(aggregationInputChannels[i]);
}
- return new Page(page.getPositionCount(), outputBlocks);
+ outputBlocks[blockOffset] = booleanRle(true, page.getPositionCount());
+ Page outputPage = new Page(page.getPositionCount(), outputBlocks);
+ return outputPage;
}
- private static long[] consecutive(int positionCount)
+ public static RunLengthEncodedBlock booleanRle(boolean value, int positionCount)
{
- long[] longs = new long[positionCount];
- for (int i = 0; i < positionCount; i++) {
- longs[i] = i;
- }
- return longs;
+ BlockBuilder valueBuilder = BOOLEAN.createBlockBuilder(null, 1);
+ BOOLEAN.writeBoolean(valueBuilder, value);
+ return new RunLengthEncodedBlock(valueBuilder.build(), positionCount);
}
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/AbstractHashTableValuesAppender.java b/core/trino-main/src/main/java/io/trino/operator/hash/AbstractHashTableValuesAppender.java
new file mode 100644
index 000000000000..bde4dbee643c
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/AbstractHashTableValuesAppender.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+import io.trino.operator.WorkProcessor;
+import io.trino.operator.aggregation.GroupedAggregator;
+import io.trino.spi.Page;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+import it.unimi.dsi.fastutil.ints.IntIterator;
+
+import java.util.List;
+
+import static io.trino.operator.WorkProcessor.ProcessState;
+import static io.trino.operator.WorkProcessor.create;
+
+public abstract class AbstractHashTableValuesAppender
+ implements HashTableValuesAppender
+{
+ private static final int BATCH_SIZE = 64;
+
+ // we use this class to isolate (using class loader) the method below.
+ public WorkProcessor buildResult(HashTableData hashTableData, IntIterator groupIds, PageBuilder pageBuilder, List groupedAggregators, int accumulatorChannelOffset, boolean outputHash)
+ {
+ return create(() -> {
+ if (!groupIds.hasNext()) {
+ return ProcessState.finished();
+ }
+
+ pageBuilder.reset();
+ int[] groupIdBatch = new int[BATCH_SIZE];
+ int[] positionsBatch = new int[BATCH_SIZE];
+ int[] valuesOffsetBatch = new int[BATCH_SIZE];
+ boolean[] isNullBatch = new boolean[BATCH_SIZE];
+ while (!pageBuilder.isFull() && groupIds.hasNext()) {
+ int currentBatchSize = 0;
+ for (; currentBatchSize < groupIdBatch.length && groupIds.hasNext(); currentBatchSize++) {
+ groupIdBatch[currentBatchSize] = groupIds.nextInt();
+ }
+
+ appendValuesTo(hashTableData, currentBatchSize, groupIdBatch, positionsBatch, valuesOffsetBatch, isNullBatch, pageBuilder, 0, outputHash);
+
+ pageBuilder.declarePositions(currentBatchSize);
+ for (int i = 0; i < groupedAggregators.size(); i++) {
+ GroupedAggregator groupedAggregator = groupedAggregators.get(i);
+ BlockBuilder output = pageBuilder.getBlockBuilder(accumulatorChannelOffset + i);
+ for (int j = 0; j < currentBatchSize; j++) {
+ int groupId = groupIdBatch[j];
+ groupedAggregator.evaluate(groupId, output);
+ }
+ }
+ }
+
+ return ProcessState.ofResult(pageBuilder.build());
+ });
+ }
+
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ appendValuesTo(data, groupIdBatch[i], pageBuilder, outputChannelOffset, outputHash);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/ColumnValueExtractor.java b/core/trino-main/src/main/java/io/trino/operator/hash/ColumnValueExtractor.java
new file mode 100644
index 000000000000..978a09c2db64
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/ColumnValueExtractor.java
@@ -0,0 +1,469 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilder;
+import io.trino.spi.type.AbstractIntType;
+import io.trino.spi.type.AbstractLongType;
+import io.trino.spi.type.BooleanType;
+import io.trino.spi.type.DecimalType;
+import io.trino.spi.type.DoubleType;
+import io.trino.spi.type.RealType;
+import io.trino.spi.type.SmallintType;
+import io.trino.spi.type.TimeWithTimeZoneType;
+import io.trino.spi.type.TimestampType;
+import io.trino.spi.type.TimestampWithTimeZoneType;
+import io.trino.spi.type.TinyintType;
+import io.trino.spi.type.Type;
+import io.trino.spi.type.UuidType;
+import io.trino.type.IpAddressType;
+
+import java.util.Optional;
+
+import static java.lang.Float.intBitsToFloat;
+
+/**
+ * Block vs FastByteBuffer adapter.
+ */
+public interface ColumnValueExtractor
+{
+ int INT96_BYTES = Long.BYTES + Integer.BYTES;
+ int INT128_BYTES = Long.BYTES + Long.BYTES;
+ ByteValueExtractor BYTE_VALUE_EXTRACTOR = new ByteValueExtractor();
+ ShortValueExtractor SHORT_VALUE_EXTRACTOR = new ShortValueExtractor();
+ IntValueExtractor INT_VALUE_EXTRACTOR = new IntValueExtractor();
+ FloatValueExtractor FLOAT_VALUE_EXTRACTOR = new FloatValueExtractor();
+ LongValueExtractor LONG_VALUE_EXTRACTOR = new LongValueExtractor();
+ DoubleValueExtractor DOUBLE_VALUE_EXTRACTOR = new DoubleValueExtractor();
+ Int96ValueExtractor INT_96_VALUE_EXTRACTOR = new Int96ValueExtractor();
+ Int128ValueExtractor INT_128_VALUE_EXTRACTOR = new Int128ValueExtractor();
+
+ static boolean isSupported(Type type)
+ {
+ return columnValueExtractor(type).isPresent();
+ }
+
+ boolean isFixedSize();
+
+ int getSize();
+
+ void putValue(FastByteBuffer buffer, int offset, Block block, int position);
+
+ void appendValue(FastByteBuffer buffer, int offset, BlockBuilder blockBuilder);
+
+ boolean valueEquals(FastByteBuffer serialized, int offset, Block block, int position);
+
+ int getSerializedValueLength(Block block, int position);
+
+ static ColumnValueExtractor requiredColumnValueExtractor(Type type)
+ {
+ return columnValueExtractor(type).orElseThrow(() -> new IllegalArgumentException("Type " + type + " is not supported!"));
+ }
+
+ static Optional columnValueExtractor(Type type)
+ {
+ if (type instanceof BooleanType || type instanceof TinyintType) {
+ return Optional.of(BYTE_VALUE_EXTRACTOR);
+ }
+ if (type instanceof SmallintType) {
+ return Optional.of(SHORT_VALUE_EXTRACTOR);
+ }
+ if (type instanceof RealType) {
+ // RealType is stored as int but has different equals semantics
+ return Optional.of(FLOAT_VALUE_EXTRACTOR);
+ }
+ if (type instanceof AbstractIntType) {
+ return Optional.of(INT_VALUE_EXTRACTOR);
+ }
+ if (type instanceof DoubleType) {
+ return Optional.of(DOUBLE_VALUE_EXTRACTOR);
+ }
+ if (type instanceof AbstractLongType ||
+ (type instanceof DecimalType && ((DecimalType) type).isShort()) ||
+ (type instanceof TimestampType && ((TimestampType) type).isShort()) ||
+ (type instanceof TimeWithTimeZoneType && ((TimeWithTimeZoneType) type).isShort()) ||
+ (type instanceof TimestampWithTimeZoneType && ((TimestampWithTimeZoneType) type).isShort())) {
+ return Optional.of(LONG_VALUE_EXTRACTOR);
+ }
+
+ if ((type instanceof TimestampType && !((TimestampType) type).isShort()) ||
+ (type instanceof TimeWithTimeZoneType && !((TimeWithTimeZoneType) type).isShort()) ||
+ (type instanceof TimestampWithTimeZoneType && !((TimestampWithTimeZoneType) type).isShort())) {
+ return Optional.of(INT_96_VALUE_EXTRACTOR);
+ }
+ if (type instanceof IpAddressType ||
+ type instanceof UuidType ||
+ (type instanceof DecimalType && !((DecimalType) type).isShort())) {
+ return Optional.of(INT_128_VALUE_EXTRACTOR);
+ }
+ return Optional.empty();
+ }
+
+ class ByteValueExtractor
+ extends AbstractColumnValueExtractor
+ implements ColumnValueExtractor
+ {
+ public ByteValueExtractor()
+ {
+ super(Byte.BYTES);
+ }
+
+ @Override
+ public void putValue(FastByteBuffer buffer, int offset, Block block, int position)
+ {
+ buffer.put(offset, block.getByte(position, 0));
+ }
+
+ @Override
+ public void appendValue(FastByteBuffer buffer, int offset, BlockBuilder blockBuilder)
+ {
+ blockBuilder.writeByte(buffer.get(offset)).closeEntry();
+ }
+
+ @Override
+ public boolean valueEquals(FastByteBuffer serialized, int offset, Block block, int position)
+ {
+ return serialized.get(offset) == block.getByte(position, 0);
+ }
+
+ @Override
+ public int getSerializedValueLength(Block block, int position)
+ {
+ return Byte.BYTES;
+ }
+
+ @Override
+ public boolean isFixedSize()
+ {
+ return true;
+ }
+ }
+
+ class ShortValueExtractor
+ extends AbstractColumnValueExtractor
+ implements ColumnValueExtractor
+ {
+ public ShortValueExtractor()
+ {
+ super(Short.BYTES);
+ }
+
+ @Override
+ public void putValue(FastByteBuffer buffer, int offset, Block block, int position)
+ {
+ buffer.putShort(offset, block.getShort(position, 0));
+ }
+
+ @Override
+ public void appendValue(FastByteBuffer buffer, int offset, BlockBuilder blockBuilder)
+ {
+ blockBuilder.writeShort(buffer.getShort(offset)).closeEntry();
+ }
+
+ @Override
+ public boolean valueEquals(FastByteBuffer serialized, int offset, Block block, int position)
+ {
+ return serialized.getShort(offset) == block.getShort(position, 0);
+ }
+
+ @Override
+ public int getSerializedValueLength(Block block, int position)
+ {
+ return Short.BYTES;
+ }
+
+ @Override
+ public boolean isFixedSize()
+ {
+ return true;
+ }
+ }
+
+ class IntValueExtractor
+ extends AbstractColumnValueExtractor
+ implements ColumnValueExtractor
+ {
+ public IntValueExtractor()
+ {
+ super(Integer.BYTES);
+ }
+
+ @Override
+ public void putValue(FastByteBuffer buffer, int offset, Block block, int position)
+ {
+ buffer.putInt(offset, block.getInt(position, 0));
+ }
+
+ @Override
+ public void appendValue(FastByteBuffer buffer, int offset, BlockBuilder blockBuilder)
+ {
+ blockBuilder.writeInt(buffer.getInt(offset)).closeEntry();
+ }
+
+ @Override
+ public boolean valueEquals(FastByteBuffer serialized, int offset, Block block, int position)
+ {
+ return serialized.getInt(offset) == block.getInt(position, 0);
+ }
+
+ @Override
+ public int getSerializedValueLength(Block block, int position)
+ {
+ return Integer.BYTES;
+ }
+
+ @Override
+ public boolean isFixedSize()
+ {
+ return true;
+ }
+ }
+
+ class FloatValueExtractor
+ extends AbstractColumnValueExtractor
+ implements ColumnValueExtractor
+ {
+ public FloatValueExtractor()
+ {
+ super(Integer.BYTES);
+ }
+
+ @Override
+ public void putValue(FastByteBuffer buffer, int offset, Block block, int position)
+ {
+ buffer.putInt(offset, block.getInt(position, 0));
+ }
+
+ @Override
+ public void appendValue(FastByteBuffer buffer, int offset, BlockBuilder blockBuilder)
+ {
+ blockBuilder.writeInt(buffer.getInt(offset)).closeEntry();
+ }
+
+ @Override
+ public boolean valueEquals(FastByteBuffer serialized, int offset, Block block, int position)
+ {
+ float leftFloat = intBitsToFloat(serialized.getInt(offset));
+ float rightFloat = intBitsToFloat(block.getInt(position, 0));
+ if (Float.isNaN(leftFloat) && Float.isNaN(rightFloat)) {
+ return true;
+ }
+ return leftFloat == rightFloat;
+ }
+
+ @Override
+ public int getSerializedValueLength(Block block, int position)
+ {
+ return Integer.BYTES;
+ }
+
+ @Override
+ public boolean isFixedSize()
+ {
+ return true;
+ }
+ }
+
+ class LongValueExtractor
+ extends AbstractColumnValueExtractor
+ implements ColumnValueExtractor
+ {
+ public LongValueExtractor()
+ {
+ super(Long.BYTES);
+ }
+
+ @Override
+ public void putValue(FastByteBuffer buffer, int offset, Block block, int position)
+ {
+ buffer.putLong(offset, block.getLong(position, 0));
+ }
+
+ @Override
+ public void appendValue(FastByteBuffer buffer, int offset, BlockBuilder blockBuilder)
+ {
+ blockBuilder.writeLong(buffer.getLong(offset)).closeEntry();
+ }
+
+ @Override
+ public boolean valueEquals(FastByteBuffer serialized, int offset, Block block, int position)
+ {
+ return serialized.getLong(offset) == block.getLong(position, 0);
+ }
+
+ @Override
+ public int getSerializedValueLength(Block block, int position)
+ {
+ return Long.BYTES;
+ }
+
+ @Override
+ public boolean isFixedSize()
+ {
+ return true;
+ }
+ }
+
+ class DoubleValueExtractor
+ extends AbstractColumnValueExtractor
+ implements ColumnValueExtractor
+ {
+ public DoubleValueExtractor()
+ {
+ super(Long.BYTES);
+ }
+
+ @Override
+ public void putValue(FastByteBuffer buffer, int offset, Block block, int position)
+ {
+ buffer.putLong(offset, block.getLong(position, 0));
+ }
+
+ @Override
+ public void appendValue(FastByteBuffer buffer, int offset, BlockBuilder blockBuilder)
+ {
+ blockBuilder.writeLong(buffer.getLong(offset)).closeEntry();
+ }
+
+ @Override
+ public boolean valueEquals(FastByteBuffer serialized, int offset, Block block, int position)
+ {
+ double left = Double.longBitsToDouble(serialized.getLong(offset));
+ double right = Double.longBitsToDouble(block.getLong(position, 0));
+ if (Double.isNaN(left) && Double.isNaN(right)) {
+ return true;
+ }
+ return left == right;
+ }
+
+ @Override
+ public int getSerializedValueLength(Block block, int position)
+ {
+ return Long.BYTES;
+ }
+
+ @Override
+ public boolean isFixedSize()
+ {
+ return true;
+ }
+ }
+
+ class Int96ValueExtractor
+ extends AbstractColumnValueExtractor
+ implements ColumnValueExtractor
+ {
+ public Int96ValueExtractor()
+ {
+ super(Long.BYTES + Integer.BYTES);
+ }
+
+ @Override
+ public void putValue(FastByteBuffer buffer, int offset, Block block, int position)
+ {
+ buffer.putLong(offset, block.getLong(position, 0));
+ buffer.putInt(offset + Long.BYTES, block.getInt(position, Long.BYTES));
+ }
+
+ @Override
+ public void appendValue(FastByteBuffer buffer, int offset, BlockBuilder blockBuilder)
+ {
+ blockBuilder.writeLong(buffer.getLong(offset));
+ blockBuilder.writeInt(buffer.getInt(offset + Long.BYTES));
+ blockBuilder.closeEntry();
+ }
+
+ @Override
+ public boolean valueEquals(FastByteBuffer serialized, int offset, Block block, int position)
+ {
+ return serialized.getLong(offset) == block.getLong(position, 0) &&
+ serialized.getInt(offset + Long.BYTES) == block.getInt(position, Long.BYTES);
+ }
+
+ @Override
+ public int getSerializedValueLength(Block block, int position)
+ {
+ return INT96_BYTES;
+ }
+
+ @Override
+ public boolean isFixedSize()
+ {
+ return true;
+ }
+ }
+
+ class Int128ValueExtractor
+ extends AbstractColumnValueExtractor
+ implements ColumnValueExtractor
+ {
+ public Int128ValueExtractor()
+ {
+ super(Long.BYTES + Long.BYTES);
+ }
+
+ @Override
+ public void putValue(FastByteBuffer buffer, int offset, Block block, int position)
+ {
+ buffer.putLong(offset, block.getLong(position, 0));
+ buffer.putLong(offset + Long.BYTES, block.getLong(position, Long.BYTES));
+ }
+
+ @Override
+ public void appendValue(FastByteBuffer buffer, int offset, BlockBuilder blockBuilder)
+ {
+ blockBuilder.writeLong(buffer.getLong(offset));
+ blockBuilder.writeLong(buffer.getLong(offset + Long.BYTES));
+ blockBuilder.closeEntry();
+ }
+
+ @Override
+ public boolean valueEquals(FastByteBuffer serialized, int offset, Block block, int position)
+ {
+ return serialized.getLong(offset) == block.getLong(position, 0) &&
+ serialized.getLong(offset + Long.BYTES) == block.getLong(position, Long.BYTES);
+ }
+
+ @Override
+ public int getSerializedValueLength(Block block, int position)
+ {
+ return INT128_BYTES;
+ }
+
+ @Override
+ public boolean isFixedSize()
+ {
+ return true;
+ }
+ }
+
+ abstract class AbstractColumnValueExtractor
+ implements ColumnValueExtractor
+ {
+ private final int sizeInBytes;
+
+ protected AbstractColumnValueExtractor(int sizeInBytes)
+ {
+ this.sizeInBytes = sizeInBytes;
+ }
+
+ @Override
+ public int getSize()
+ {
+ return sizeInBytes;
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/GroupByHashTable.java b/core/trino-main/src/main/java/io/trino/operator/hash/GroupByHashTable.java
new file mode 100644
index 000000000000..73ecc07e0bb7
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/GroupByHashTable.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+import io.trino.spi.Page;
+
+public interface GroupByHashTable
+{
+ int putIfAbsent(HashTableData data, int position, Page page);
+
+ boolean contains(HashTableData data, int position, Page page);
+
+ boolean contains(HashTableData data, int position, Page page, long rawHash);
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/GroupByHashTableEntries.java b/core/trino-main/src/main/java/io/trino/operator/hash/GroupByHashTableEntries.java
new file mode 100644
index 000000000000..391eefdd2b73
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/GroupByHashTableEntries.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+public interface GroupByHashTableEntries
+{
+ int getEntrySize();
+
+ long getHash(int position);
+
+ byte isNull(int position, int i);
+
+ int capacity();
+
+ int getGroupId(int position);
+
+ void putHash(int position, long hash);
+
+ void putGroupId(int position, int groupId);
+
+ String toString(int position);
+
+ long getEstimatedSize();
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/HashTableData.java b/core/trino-main/src/main/java/io/trino/operator/hash/HashTableData.java
new file mode 100644
index 000000000000..4df9989ddeca
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/HashTableData.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+public interface HashTableData
+{
+ int getPosition(int groupId);
+
+ int getHashPosition(long rawHash);
+
+ boolean needRehash();
+
+ int getGroupId(int hashPosition);
+
+ int addNewGroupId(int hashPosition);
+
+ GroupByHashTableEntries entries();
+
+ int nextPosition(int hashPosition);
+
+ void markCollision();
+
+ long getEstimatedSize();
+
+ long getHashCollisions();
+
+ int getHashTableSize();
+
+ int getCapacity();
+
+ int maxFill();
+
+ long getRawHash(int groupId);
+
+ HashTableData resize(int newCapacity);
+
+ int entriesPosition(int hashPosition, int groupId);
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/HashTableDataGroupByHash.java b/core/trino-main/src/main/java/io/trino/operator/hash/HashTableDataGroupByHash.java
new file mode 100644
index 000000000000..ed5ccadf7d90
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/HashTableDataGroupByHash.java
@@ -0,0 +1,668 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import io.trino.operator.GroupByHash;
+import io.trino.operator.GroupByIdBlock;
+import io.trino.operator.HashGenerator;
+import io.trino.operator.InterpretedHashGenerator;
+import io.trino.operator.MultiChannelGroupByHash.DictionaryLookBack;
+import io.trino.operator.PrecomputedHashGenerator;
+import io.trino.operator.UpdateMemory;
+import io.trino.operator.Work;
+import io.trino.operator.WorkProcessor;
+import io.trino.operator.aggregation.GroupedAggregator;
+import io.trino.operator.hash.fixedwidth.SingleTableHashTableData;
+import io.trino.spi.Page;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.TrinoException;
+import io.trino.spi.block.Block;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.LongArrayBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import io.trino.spi.type.Type;
+import io.trino.type.BlockTypeOperators;
+import it.unimi.dsi.fastutil.ints.IntIterator;
+import org.openjdk.jol.info.ClassLayout;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Verify.verify;
+import static io.trino.spi.StandardErrorCode.GENERIC_INSUFFICIENT_RESOURCES;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static io.trino.util.HashCollisionsEstimator.estimateNumberOfHashCollisions;
+import static it.unimi.dsi.fastutil.HashCommon.arraySize;
+import static java.lang.Math.toIntExact;
+import static java.util.Objects.requireNonNull;
+
+public class HashTableDataGroupByHash
+ implements GroupByHash
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(HashTableDataGroupByHash.class).instanceSize();
+
+ static final float FILL_RATIO = 0.75f;
+
+ private final Optional inputHashChannel;
+ private final int[] hashChannelsWithHash;
+
+ // the hash table with values + groupId entries.
+
+ private HashTableData hashTableData;
+ private double expectedHashCollisions;
+ private final boolean processDictionary;
+ private DictionaryLookBack dictionaryLookBack;
+
+ // reserve enough memory before rehash
+ private final UpdateMemory updateMemory;
+ private long preallocatedMemoryInBytes;
+ private long currentPageSizeInBytes;
+ private final List types;
+
+ private final HashTableManager hashTableManager;
+
+ public HashTableDataGroupByHash(
+ IsolatedHashTableFactory isolatedHashTableFactory,
+ List extends Type> hashTypes,
+ int[] hashChannels,
+ Optional inputHashChannel,
+ int expectedSize,
+ boolean processDictionary,
+ UpdateMemory updateMemory,
+ BlockTypeOperators blockTypeOperators)
+ {
+ this.processDictionary = processDictionary;
+ checkArgument(expectedSize > 0, "expectedSize must be greater than zero");
+
+ checkArgument(hashChannels.length > 0, "hashChannels.length must be at least 1");
+ requireNonNull(inputHashChannel, "inputHashChannel is null");
+
+ if (inputHashChannel.isPresent()) {
+ int channel = inputHashChannel.get();
+ this.inputHashChannel = Optional.of(hashChannels.length);
+ this.hashChannelsWithHash = Arrays.copyOf(hashChannels, hashChannels.length + 1);
+ // map inputHashChannel to last column
+ this.hashChannelsWithHash[hashChannels.length] = channel;
+ }
+ else {
+ this.inputHashChannel = Optional.empty();
+ this.hashChannelsWithHash = hashChannels;
+ }
+
+ this.hashTableManager = new HashTableManager(
+ this,
+ isolatedHashTableFactory,
+ hashTypes,
+ createHashGenerator(hashTypes, this.inputHashChannel, blockTypeOperators));
+ this.hashTableData = new SingleTableHashTableData(hashTypes.size(), arraySize(expectedSize, FILL_RATIO), hashTableManager.valuesLTotalLength());
+
+ // This interface is used for actively reserving memory (push model) for rehash.
+ // The caller can also query memory usage on this object (pull model)
+ this.updateMemory = requireNonNull(updateMemory, "updateMemory is null");
+ this.types = inputHashChannel.isPresent() ? ImmutableList.copyOf(Iterables.concat(hashTypes, ImmutableList.of(BIGINT))) : ImmutableList.copyOf(hashTypes);
+ }
+
+ @Override
+ public long getEstimatedSize()
+ {
+ return INSTANCE_SIZE +
+ hashTableData.getEstimatedSize() +
+ preallocatedMemoryInBytes;
+ }
+
+ @Override
+ public long getHashCollisions()
+ {
+ return hashTableData.getHashCollisions();
+ }
+
+ @Override
+ public double getExpectedHashCollisions()
+ {
+ return expectedHashCollisions + estimateNumberOfHashCollisions(getGroupCount(), hashTableData.getCapacity());
+ }
+
+ @Override
+ public List getTypes()
+ {
+ return types;
+ }
+
+ @Override
+ public int getGroupCount()
+ {
+ return hashTableData.getHashTableSize();
+ }
+
+ @Override
+ public void appendValuesTo(int groupId, PageBuilder pageBuilder)
+ {
+ hashTableManager.valuesAppender().appendValuesTo(hashTableData, groupId, pageBuilder, 0, inputHashChannel.isPresent());
+ }
+
+ @Override
+ public WorkProcessor buildResult(IntIterator groupIds, PageBuilder pageBuilder, List groupedAggregators)
+ {
+ return hashTableManager.valuesAppender().buildResult(hashTableData, groupIds, pageBuilder, groupedAggregators, types.size(), inputHashChannel.isPresent());
+ }
+
+ @Override
+ public Work> addPage(Page page)
+ {
+ currentPageSizeInBytes = page.getRetainedSizeInBytes();
+
+ Page minimalPage = page.getColumns(hashChannelsWithHash);
+ if (isRunLengthEncoded(minimalPage)) {
+ return new AddRunLengthEncodedPageWork(this, hashTableManager.getHashTable(minimalPage), minimalPage);
+ }
+ if (canProcessDictionary(minimalPage)) {
+ return hashTableManager.getAAddDictionaryPageWork(minimalPage);
+ }
+
+ return hashTableManager.getAddPageWork(minimalPage);
+ }
+
+ @Override
+ public Work getGroupIds(Page page)
+ {
+ currentPageSizeInBytes = page.getRetainedSizeInBytes();
+
+ Page minimalPage = page.getColumns(hashChannelsWithHash);
+ if (isRunLengthEncoded(minimalPage)) {
+ return new GetRunLengthEncodedGroupIdsWork(this, hashTableManager.getHashTable(minimalPage), minimalPage);
+ }
+ if (canProcessDictionary(minimalPage)) {
+ return hashTableManager.getGetDictionaryGroupIdsWorkFactory(minimalPage);
+ }
+ return hashTableManager.getGetGroupIdsWorkFactory(minimalPage);
+ }
+
+ @Override
+ public boolean contains(int position, Page page)
+ {
+ return hashTableManager.getForContains(page).contains(hashTableData, position, page);
+ }
+
+ @Override
+ public boolean contains(int position, Page page, long rawHash)
+ {
+ return hashTableManager.getForContains(page).contains(hashTableData, position, page, rawHash);
+ }
+
+ @Override
+ public long getRawHash(int groupId)
+ {
+ return hashTableData.getRawHash(groupId);
+ }
+
+ @VisibleForTesting
+ @Override
+ public int getCapacity()
+ {
+ return hashTableData.getCapacity();
+ }
+
+ public boolean tryRehash()
+ {
+ return tryRehash(hashTableData.getCapacity() * 2L);
+ }
+
+ private boolean tryRehash(long newCapacityLong)
+ {
+ if (newCapacityLong > Integer.MAX_VALUE) {
+ throw new TrinoException(GENERIC_INSUFFICIENT_RESOURCES, "Size of hash table cannot exceed 1 billion entries");
+ }
+ int newCapacity = toIntExact(newCapacityLong);
+ // An estimate of how much extra memory is needed before we can go ahead and expand the hash table.
+ // This includes the new capacity for values, groupIds, and valuesByGroupId as well as the size of the current page
+ preallocatedMemoryInBytes = (newCapacity - hashTableData.getCapacity()) * (long) (Long.BYTES + Integer.BYTES) + (calculateMaxFill(newCapacity) - hashTableData.maxFill()) * Long.BYTES + currentPageSizeInBytes;
+ if (!updateMemory.update()) {
+ // reserved memory but has exceeded the limit
+ return false;
+ }
+ preallocatedMemoryInBytes = 0;
+
+ expectedHashCollisions += estimateNumberOfHashCollisions(getGroupCount(), hashTableData.getCapacity());
+
+ hashTableData = hashTableData.resize(newCapacity);
+ return true;
+ }
+
+ public static int calculateMaxFill(int hashSize)
+ {
+ checkArgument(hashSize > 0, "hashSize must be greater than 0");
+ int maxFill = (int) Math.ceil(hashSize * FILL_RATIO);
+ if (maxFill == hashSize) {
+ maxFill--;
+ }
+ checkArgument(hashSize > maxFill, "hashSize must be larger than maxFill");
+ return maxFill;
+ }
+
+ public HashTableData getHashTableData()
+ {
+ return hashTableData;
+ }
+
+ public static class AddPageWork
+ implements Work
+ {
+ private final GroupByHashTable hashTable;
+ private final HashTableDataGroupByHash groupByHash;
+ private final Page page;
+ private int lastPosition;
+
+ public AddPageWork(HashTableDataGroupByHash groupByHash, GroupByHashTable hashTable, Page page)
+ {
+ this.groupByHash = requireNonNull(groupByHash, "groupByHash is null");
+ this.page = requireNonNull(page, "page is null");
+ this.hashTable = requireNonNull(hashTable, "hashTable is null");
+ }
+
+ @Override
+ public boolean process()
+ {
+ int positionCount = page.getPositionCount();
+ checkState(lastPosition < positionCount, "position count out of bound");
+
+ // needRehash() == false indicates we have reached capacity boundary and a rehash is needed.
+ // We can only proceed if tryRehash() successfully did a rehash.
+ if (groupByHash.getHashTableData().needRehash() && !groupByHash.tryRehash()) {
+ return false;
+ }
+
+ HashTableData data = groupByHash.getHashTableData();
+ while (lastPosition < positionCount) {
+ hashTable.putIfAbsent(data, lastPosition, page);
+ lastPosition++;
+ if (data.needRehash()) {
+ if (!groupByHash.tryRehash()) {
+ // memory not available
+ break;
+ }
+ data = groupByHash.getHashTableData();
+ }
+ }
+ return lastPosition == positionCount;
+ }
+
+ @Override
+ public Void getResult()
+ {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ private class AddRunLengthEncodedPageWork
+ implements Work
+ {
+ private final HashTableDataGroupByHash groupByHash;
+ private final GroupByHashTable hashTable;
+ private final Page page;
+
+ private boolean finished;
+
+ public AddRunLengthEncodedPageWork(HashTableDataGroupByHash groupByHash, GroupByHashTable hashTable, Page page)
+ {
+ this.groupByHash = requireNonNull(groupByHash, "groupByHash is null");
+ this.page = requireNonNull(page, "page is null");
+ this.hashTable = requireNonNull(hashTable, "hashTable is null");
+ }
+
+ @Override
+ public boolean process()
+ {
+ checkState(!finished);
+ if (page.getPositionCount() == 0) {
+ finished = true;
+ return true;
+ }
+
+ // needRehash() == false indicates we have reached capacity boundary and a rehash is needed.
+ // We can only proceed if tryRehash() successfully did a rehash.
+ if (groupByHash.getHashTableData().needRehash() && !groupByHash.tryRehash()) {
+ return false;
+ }
+
+ HashTableData data = groupByHash.getHashTableData();
+ // Only needs to process the first row since it is Run Length Encoded
+ hashTable.putIfAbsent(data, 0, page);
+ finished = true;
+
+ return true;
+ }
+
+ @Override
+ public Void getResult()
+ {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ private class GetRunLengthEncodedGroupIdsWork
+ implements Work
+ {
+ private final HashTableDataGroupByHash groupByHash;
+ private final GroupByHashTable hashTable;
+ private final Page page;
+
+ int groupId = -1;
+ private boolean processFinished;
+ private boolean resultProduced;
+
+ public GetRunLengthEncodedGroupIdsWork(HashTableDataGroupByHash groupByHash, GroupByHashTable hashTable, Page page)
+ {
+ this.groupByHash = requireNonNull(groupByHash, "groupByHash is null");
+ this.page = requireNonNull(page, "page is null");
+ this.hashTable = requireNonNull(hashTable, "hashTable is null");
+ }
+
+ @Override
+ public boolean process()
+ {
+ checkState(!processFinished);
+ if (page.getPositionCount() == 0) {
+ processFinished = true;
+ return true;
+ }
+
+ // needRehash() == false indicates we have reached capacity boundary and a rehash is needed.
+ // We can only proceed if tryRehash() successfully did a rehash.
+ if (groupByHash.getHashTableData().needRehash() && !groupByHash.tryRehash()) {
+ return false;
+ }
+
+ HashTableData data = groupByHash.getHashTableData();
+ // Only needs to process the first row since it is Run Length Encoded
+ groupId = hashTable.putIfAbsent(data, 0, page);
+
+ processFinished = true;
+ return true;
+ }
+
+ @Override
+ public GroupByIdBlock getResult()
+ {
+ checkState(processFinished);
+ checkState(!resultProduced);
+ resultProduced = true;
+
+ return new GroupByIdBlock(
+ groupByHash.getGroupCount(),
+ new RunLengthEncodedBlock(
+ BIGINT.createFixedSizeBlockBuilder(1).writeLong(groupId).build(),
+ page.getPositionCount()));
+ }
+ }
+
+ public static class AddDictionaryPageWork
+ implements Work
+ {
+ private final HashTableDataGroupByHash groupByHash;
+ private final GroupByHashTable hashTable;
+ private final Page page;
+ private final Page dictionaryPage;
+ private final DictionaryBlock dictionaryBlock;
+
+ private int lastPosition;
+
+ public AddDictionaryPageWork(HashTableDataGroupByHash groupByHash, GroupByHashTable hashTable, Page page)
+ {
+ this.groupByHash = groupByHash;
+ this.hashTable = hashTable;
+ verify(groupByHash.canProcessDictionary(page), "invalid call to addDictionaryPage");
+ this.page = requireNonNull(page, "page is null");
+ this.dictionaryBlock = (DictionaryBlock) page.getBlock(0);
+ groupByHash.updateDictionaryLookBack(dictionaryBlock.getDictionary());
+ this.dictionaryPage = groupByHash.createPageWithExtractedDictionary(page);
+ }
+
+ @Override
+ public boolean process()
+ {
+ int positionCount = page.getPositionCount();
+ checkState(lastPosition < positionCount, "position count out of bound");
+
+ // needRehash() == false indicates we have reached capacity boundary and a rehash is needed.
+ // We can only proceed if tryRehash() successfully did a rehash.
+ if (groupByHash.getHashTableData().needRehash() && !groupByHash.tryRehash()) {
+ return false;
+ }
+
+ HashTableData data = groupByHash.getHashTableData();
+ while (lastPosition < positionCount) {
+ int positionInDictionary = dictionaryBlock.getId(lastPosition);
+ groupByHash.registerGroupId(hashTable, data, dictionaryPage, positionInDictionary);
+ lastPosition++;
+ if (data.needRehash()) {
+ if (!groupByHash.tryRehash()) {
+ // memory not available
+ break;
+ }
+ data = groupByHash.getHashTableData();
+ }
+ }
+ return lastPosition == positionCount;
+ }
+
+ @Override
+ public Void getResult()
+ {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ public static class GetDictionaryGroupIdsWork
+ implements Work
+ {
+ private final long[] groupIds;
+ private final HashTableDataGroupByHash groupByHash;
+ private final GroupByHashTable hashTable;
+ private final Page page;
+ private final Page dictionaryPage;
+ private final DictionaryBlock dictionaryBlock;
+
+ private boolean finished;
+ private int lastPosition;
+
+ public GetDictionaryGroupIdsWork(HashTableDataGroupByHash groupByHash, GroupByHashTable hashTable, Page page)
+ {
+ this.groupByHash = groupByHash;
+ this.hashTable = hashTable;
+ verify(groupByHash.canProcessDictionary(page), "invalid call to addDictionaryPage");
+ this.page = requireNonNull(page, "page is null");
+ this.dictionaryBlock = (DictionaryBlock) page.getBlock(0);
+ groupByHash.updateDictionaryLookBack(dictionaryBlock.getDictionary());
+ this.dictionaryPage = groupByHash.createPageWithExtractedDictionary(page);
+ this.groupIds = new long[page.getPositionCount()];
+ }
+
+ @Override
+ public boolean process()
+ {
+ int positionCount = page.getPositionCount();
+ checkState(lastPosition < positionCount, "position count out of bound");
+
+ // needRehash() == false indicates we have reached capacity boundary and a rehash is needed.
+ // We can only proceed if tryRehash() successfully did a rehash.
+ if (groupByHash.getHashTableData().needRehash() && !groupByHash.tryRehash()) {
+ return false;
+ }
+
+ HashTableData data = groupByHash.getHashTableData();
+ while (lastPosition < positionCount) {
+ int positionInDictionary = dictionaryBlock.getId(lastPosition);
+ int groupId = groupByHash.registerGroupId(hashTable, data, dictionaryPage, positionInDictionary);
+ groupIds[lastPosition] = groupId;
+ lastPosition++;
+ if (data.needRehash()) {
+ if (!groupByHash.tryRehash()) {
+ // memory not available
+ break;
+ }
+ data = groupByHash.getHashTableData();
+ }
+ }
+ return lastPosition == positionCount;
+ }
+
+ @Override
+ public GroupByIdBlock getResult()
+ {
+ checkState(lastPosition == page.getPositionCount(), "process has not yet finished");
+ checkState(!finished, "result has produced");
+ finished = true;
+ return new GroupByIdBlock(groupByHash.getGroupCount(), new LongArrayBlock(page.getPositionCount(), Optional.empty(), groupIds));
+ }
+ }
+
+ public static class GetGroupIdsWork
+ implements Work
+ {
+ private final HashTableDataGroupByHash groupByHash;
+ private final GroupByHashTable hashTable;
+ private final long[] groupIds;
+ private final Page page;
+ private boolean finished;
+ private int lastPosition;
+
+ public GetGroupIdsWork(HashTableDataGroupByHash groupByHash, GroupByHashTable hashTable, Page page)
+ {
+ this.groupByHash = requireNonNull(groupByHash, "groupByHash is null");
+ this.hashTable = requireNonNull(hashTable, "hashTable is null");
+ this.page = requireNonNull(page, "page is null");
+ this.groupIds = new long[page.getPositionCount()];
+ }
+
+ @Override
+ public boolean process()
+ {
+ int positionCount = page.getPositionCount();
+ checkState(lastPosition < positionCount, "position count out of bound");
+ checkState(!finished);
+
+ // needRehash() == false indicates we have reached capacity boundary and a rehash is needed.
+ // We can only proceed if tryRehash() successfully did a rehash.
+ if (groupByHash.getHashTableData().needRehash() && !groupByHash.tryRehash()) {
+ return false;
+ }
+
+ HashTableData data = groupByHash.getHashTableData();
+ while (lastPosition < positionCount && !data.needRehash()) {
+ // output the group id for this row
+ int groupId = hashTable.putIfAbsent(data, lastPosition, page);
+ groupIds[lastPosition] = groupId;
+ lastPosition++;
+ if (data.needRehash()) {
+ if (!groupByHash.tryRehash()) {
+ // memory not available
+ break;
+ }
+ data = groupByHash.getHashTableData();
+ }
+ }
+ return lastPosition == positionCount;
+ }
+
+ @Override
+ public GroupByIdBlock getResult()
+ {
+ checkState(lastPosition == page.getPositionCount(), "process has not yet finished");
+ checkState(!finished, "result has produced");
+ finished = true;
+ return new GroupByIdBlock(groupByHash.getGroupCount(), new LongArrayBlock(page.getPositionCount(), Optional.empty(), groupIds));
+ }
+ }
+
+ public boolean canProcessDictionary(Page page)
+ {
+ if (!this.processDictionary || page.getChannelCount() > 1 || !(page.getBlock(0) instanceof DictionaryBlock)) {
+ return false;
+ }
+
+ if (inputHashChannel.isPresent()) {
+ Block inputHashBlock = page.getBlock(inputHashChannel.get());
+ DictionaryBlock inputDataBlock = (DictionaryBlock) page.getBlock(0);
+
+ if (!(inputHashBlock instanceof DictionaryBlock)) {
+ // data channel is dictionary encoded but hash channel is not
+ return false;
+ }
+ if (!((DictionaryBlock) inputHashBlock).getDictionarySourceId().equals(inputDataBlock.getDictionarySourceId())) {
+ // dictionarySourceIds of data block and hash block do not match
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ private boolean isRunLengthEncoded(Page page)
+ {
+ for (int i = 0; i < page.getChannelCount(); i++) {
+ if (!(page.getBlock(i) instanceof RunLengthEncodedBlock)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ // For a page that contains DictionaryBlocks, create a new page in which
+ // the dictionaries from the DictionaryBlocks are extracted into the corresponding channels
+ // From Page(DictionaryBlock1, DictionaryBlock2) create new page with Page(dictionary1, dictionary2)
+ public Page createPageWithExtractedDictionary(Page page)
+ {
+ Block[] blocks = new Block[page.getChannelCount()];
+ Block dictionary = ((DictionaryBlock) page.getBlock(0)).getDictionary();
+
+ // extract data dictionary
+ blocks[0] = dictionary;
+
+ // extract hash dictionary
+ inputHashChannel.ifPresent(channel -> blocks[channel] = ((DictionaryBlock) page.getBlock(channel)).getDictionary());
+
+ return new Page(dictionary.getPositionCount(), blocks);
+ }
+
+ public void updateDictionaryLookBack(Block dictionary)
+ {
+ if (dictionaryLookBack == null || dictionaryLookBack.getDictionary() != dictionary) {
+ dictionaryLookBack = new DictionaryLookBack(dictionary);
+ }
+ }
+
+ public int registerGroupId(GroupByHashTable hashTable, HashTableData data, Page page, int positionInDictionary)
+ {
+ if (dictionaryLookBack.isProcessed(positionInDictionary)) {
+ return dictionaryLookBack.getGroupId(positionInDictionary);
+ }
+
+ int groupId = hashTable.putIfAbsent(data, positionInDictionary, page);
+ dictionaryLookBack.setProcessed(positionInDictionary, groupId);
+ return groupId;
+ }
+
+ private static HashGenerator createHashGenerator(List extends Type> hashTypes, Optional inputHashChannel, BlockTypeOperators blockTypeOperators)
+ {
+ return inputHashChannel.isPresent() ?
+ new PrecomputedHashGenerator(inputHashChannel.get()) :
+ InterpretedHashGenerator.createPositionalWithTypes(ImmutableList.copyOf(hashTypes), blockTypeOperators);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/HashTableManager.java b/core/trino-main/src/main/java/io/trino/operator/hash/HashTableManager.java
new file mode 100644
index 000000000000..12cd1e23911e
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/HashTableManager.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+import com.google.common.collect.ImmutableList;
+import io.trino.operator.GroupByIdBlock;
+import io.trino.operator.HashGenerator;
+import io.trino.operator.Work;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+import io.trino.spi.type.Type;
+
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+public class HashTableManager
+{
+ private final HashTableDataGroupByHash groupByHash;
+ private final IsolatedHashTableFactory isolatedHashTableFactory;
+ private final HashGenerator hashGenerator;
+ private final FixedWidthEntryStructure structure;
+ private final HashTableValuesAppender valuesAppender;
+ private GroupByHashTable containsHashTable;
+ private Page lastContainsPage;
+
+ public HashTableManager(
+ HashTableDataGroupByHash groupByHash,
+ IsolatedHashTableFactory isolatedHashTableFactory,
+ List extends Type> hashTypes,
+ HashGenerator hashGenerator)
+ {
+ this.groupByHash = groupByHash;
+ this.isolatedHashTableFactory = isolatedHashTableFactory;
+ this.hashGenerator = hashGenerator;
+ this.structure = new FixedWidthEntryStructure(hashTypes.stream()
+ .map(ColumnValueExtractor::requiredColumnValueExtractor)
+ .peek(columnValueExtractor -> checkArgument(columnValueExtractor.isFixedSize(), "channel is not fixed width " + columnValueExtractor))
+ .toArray(ColumnValueExtractor[]::new));
+ this.valuesAppender = isolatedHashTableFactory.createAppender(structure);
+ }
+
+ public GroupByHashTable getForContains(Page page)
+ {
+ if (containsHashTable == null || page != lastContainsPage) {
+ containsHashTable = isolatedHashTableFactory.createHashTable(structure, blockTypes(page), hashGenerator);
+ lastContainsPage = page;
+ }
+ return containsHashTable;
+ }
+
+ public Work getAddPageWork(Page page)
+ {
+ WorkFactory workFactory = isolatedHashTableFactory.createAddPageWorkFactory(structure, blockTypes(page));
+ return workFactory.create(groupByHash, getHashTable(page), page);
+ }
+
+ public Work getAAddDictionaryPageWork(Page page)
+ {
+ WorkFactory workFactory = isolatedHashTableFactory.createAAddDictionaryPageWork(structure, blockTypes(page));
+ return workFactory.create(groupByHash, getHashTable(page), page);
+ }
+
+ public Work getGetGroupIdsWorkFactory(Page page)
+ {
+ WorkFactory workFactory = isolatedHashTableFactory.createGetGroupIdsWorkFactory(structure, blockTypes(page));
+ return workFactory.create(groupByHash, getHashTable(page), page);
+ }
+
+ public Work getGetDictionaryGroupIdsWorkFactory(Page page)
+ {
+ WorkFactory workFactory = isolatedHashTableFactory.createGetDictionaryGroupIdsWorkFactory(structure, blockTypes(page));
+ return workFactory.create(groupByHash, getHashTable(page), page);
+ }
+
+ public GroupByHashTable getHashTable(Page page)
+ {
+ return isolatedHashTableFactory.createHashTable(structure, blockTypes(page), hashGenerator);
+ }
+
+ public HashTableValuesAppender valuesAppender()
+ {
+ return valuesAppender;
+ }
+
+ public int valuesLTotalLength()
+ {
+ return structure.getValuesLength();
+ }
+
+ private List> blockTypes(Page page)
+ {
+ ImmutableList.Builder> result = ImmutableList.builder();
+ for (int i = 0; i < structure.getHashChannelsCount(); i++) {
+ result.add(page.getBlock(i).getClass());
+ }
+ return result.build();
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/HashTableRowHandler.java b/core/trino-main/src/main/java/io/trino/operator/hash/HashTableRowHandler.java
new file mode 100644
index 000000000000..c1114dd34e92
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/HashTableRowHandler.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+import io.trino.spi.Page;
+
+public interface HashTableRowHandler
+{
+ void putEntry(GroupByHashTableEntries entries, int entriesPosition, int groupId, Page page, int position, long rawHash);
+
+ boolean keyEquals(GroupByHashTableEntries entries, int entriesPosition, Page page, int position, long rawHash);
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/HashTableValuesAppender.java b/core/trino-main/src/main/java/io/trino/operator/hash/HashTableValuesAppender.java
new file mode 100644
index 000000000000..f5acc1aeb78c
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/HashTableValuesAppender.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+import io.trino.operator.WorkProcessor;
+import io.trino.operator.aggregation.GroupedAggregator;
+import io.trino.spi.Page;
+import io.trino.spi.PageBuilder;
+import it.unimi.dsi.fastutil.ints.IntIterator;
+
+import java.util.List;
+
+public interface HashTableValuesAppender
+{
+ void appendValuesTo(HashTableData hashTableData, int groupId, PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash);
+
+ WorkProcessor buildResult(HashTableData hashTableData, IntIterator groupIds, PageBuilder pageBuilder, List groupedAggregators, int accumulatorChannelOffset, boolean outputHash);
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/IsolatedHashTableFactory.java b/core/trino-main/src/main/java/io/trino/operator/hash/IsolatedHashTableFactory.java
new file mode 100644
index 000000000000..fea7df8c2ca0
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/IsolatedHashTableFactory.java
@@ -0,0 +1,250 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import io.airlift.bytecode.DynamicClassLoader;
+import io.trino.operator.GroupByIdBlock;
+import io.trino.operator.HashGenerator;
+import io.trino.operator.Work;
+import io.trino.operator.hash.HashTableDataGroupByHash.AddDictionaryPageWork;
+import io.trino.operator.hash.HashTableDataGroupByHash.AddPageWork;
+import io.trino.operator.hash.HashTableDataGroupByHash.GetDictionaryGroupIdsWork;
+import io.trino.operator.hash.HashTableDataGroupByHash.GetGroupIdsWork;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.GenericFixedWidthHashTableRowHandler;
+import io.trino.operator.hash.fixedwidth.GenericFixedWidthHashTableValuesAppender;
+import io.trino.operator.hash.fixedwidth.gen.FixedWidthHashTableRowHandler1Channels;
+import io.trino.operator.hash.fixedwidth.gen.FixedWidthHashTableValuesAppender1Channels;
+import io.trino.spi.block.Block;
+import io.trino.sql.gen.IsolatedClass;
+
+import javax.inject.Inject;
+
+import java.lang.reflect.Constructor;
+import java.util.List;
+import java.util.Objects;
+
+import static java.lang.String.format;
+import static java.util.Objects.requireNonNull;
+
+public class IsolatedHashTableFactory
+{
+ public static final int MAX_SUPPORTED_CHANNELS = 20;
+ private static final Class extends HashTableRowHandler>[] DEDICATED_ROW_HANDLERS;
+ private static final Class extends HashTableValuesAppender>[] DEDICATED_VALUES_APPENDERS;
+
+ private final LoadingCache hashTableCache;
+ private final LoadingCache appenderCache;
+ private final LoadingCache> addPageWorkFactoryCache;
+ private final LoadingCache> addDictionaryPageWorkFactoryCache;
+ private final LoadingCache> getGroupIdsWorkFactoryCache;
+ private final LoadingCache> getDictionaryGroupIdsWorkFactoryCache;
+
+ static {
+ // setup dedicated per number of channels row handler and values appender classes
+ DEDICATED_ROW_HANDLERS = new Class[MAX_SUPPORTED_CHANNELS];
+ DEDICATED_VALUES_APPENDERS = new Class[MAX_SUPPORTED_CHANNELS];
+ for (int i = 0; i < DEDICATED_ROW_HANDLERS.length; i++) {
+ try {
+ DEDICATED_ROW_HANDLERS[i] = (Class extends HashTableRowHandler>) Class.forName(FixedWidthHashTableRowHandler1Channels.class.getPackageName() + "." +
+ format("FixedWidthHashTableRowHandler%dChannels", i + 1));
+ DEDICATED_VALUES_APPENDERS[i] = (Class extends HashTableValuesAppender>) Class.forName(FixedWidthHashTableValuesAppender1Channels.class.getPackageName() + "." +
+ format("FixedWidthHashTableValuesAppender%dChannels", i + 1));
+ }
+ catch (ClassNotFoundException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ @Inject
+ public IsolatedHashTableFactory()
+ {
+ this.hashTableCache = CacheBuilder.newBuilder()
+ .maximumSize(1024 * 16)
+ .build(CacheLoader.from(key -> isolateHashTable(key.getStructure(), key.getHashGenerator())));
+
+ this.appenderCache = CacheBuilder.newBuilder()
+ .maximumSize(1024 * 16)
+ .build(CacheLoader.from(this::isolateAppender));
+
+ this.addPageWorkFactoryCache = CacheBuilder.newBuilder()
+ .maximumSize(1024 * 16)
+ .build(CacheLoader.from(key -> isolateWorkFactory(AddPageWork.class)));
+
+ this.addDictionaryPageWorkFactoryCache = CacheBuilder.newBuilder()
+ .maximumSize(1024 * 16)
+ .build(CacheLoader.from(key -> isolateWorkFactory(AddDictionaryPageWork.class)));
+
+ this.getGroupIdsWorkFactoryCache = CacheBuilder.newBuilder()
+ .maximumSize(1024 * 16)
+ .build(CacheLoader.from(key -> isolateWorkFactory(GetGroupIdsWork.class)));
+
+ this.getDictionaryGroupIdsWorkFactoryCache = CacheBuilder.newBuilder()
+ .maximumSize(1024 * 16)
+ .build(CacheLoader.from(key -> isolateWorkFactory(GetDictionaryGroupIdsWork.class)));
+ }
+
+ public GroupByHashTable createHashTable(FixedWidthEntryStructure structure, List> blockTypes, HashGenerator hashGenerator)
+ {
+ return hashTableCache.getUnchecked(new HashTableCacheKey(structure, blockTypes, hashGenerator));
+ }
+
+ public WorkFactory createAddPageWorkFactory(FixedWidthEntryStructure structure, List> blockTypes)
+ {
+ return addPageWorkFactoryCache.getUnchecked(new StructureAndBlockCacheKey(structure, blockTypes));
+ }
+
+ public WorkFactory createAAddDictionaryPageWork(FixedWidthEntryStructure structure, List> blockTypes)
+ {
+ return addDictionaryPageWorkFactoryCache.getUnchecked(new StructureAndBlockCacheKey(structure, blockTypes));
+ }
+
+ public WorkFactory createGetGroupIdsWorkFactory(FixedWidthEntryStructure structure, List> blockTypes)
+ {
+ return getGroupIdsWorkFactoryCache.getUnchecked(new StructureAndBlockCacheKey(structure, blockTypes));
+ }
+
+ public WorkFactory createGetDictionaryGroupIdsWorkFactory(FixedWidthEntryStructure structure, List> blockTypes)
+ {
+ return getDictionaryGroupIdsWorkFactoryCache.getUnchecked(new StructureAndBlockCacheKey(structure, blockTypes));
+ }
+
+ public HashTableValuesAppender createAppender(FixedWidthEntryStructure structure)
+ {
+ return appenderCache.getUnchecked(structure);
+ }
+
+ private HashTableValuesAppender isolateAppender(FixedWidthEntryStructure structure)
+ {
+ Class extends HashTableValuesAppender> appenderClass = GenericFixedWidthHashTableValuesAppender.class;
+ if (structure.getHashChannelsCount() <= DEDICATED_VALUES_APPENDERS.length) {
+ appenderClass = DEDICATED_VALUES_APPENDERS[structure.getHashChannelsCount() - 1];
+ }
+
+ Class extends HashTableValuesAppender> valuesAppenderClass = IsolatedClass.isolateClass(
+ new DynamicClassLoader(HashTableValuesAppender.class.getClassLoader()),
+ HashTableValuesAppender.class,
+ appenderClass,
+ AbstractHashTableValuesAppender.class);
+ try {
+ return valuesAppenderClass.getConstructor(FixedWidthEntryStructure.class).newInstance(structure);
+ }
+ catch (ReflectiveOperationException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private WorkFactory isolateWorkFactory(Class extends Work> workClass)
+ {
+ Class extends Work> isolatedWorkClass = (Class extends Work>) IsolatedClass.isolateClass(
+ new DynamicClassLoader(HashTableDataGroupByHash.class.getClassLoader()),
+ Work.class,
+ workClass);
+ return new WorkFactory(isolatedWorkClass);
+ }
+
+ private GroupByHashTable isolateHashTable(FixedWidthEntryStructure structure, HashGenerator hashGenerator)
+ {
+ Class extends HashTableRowHandler> rowHandlerClass = GenericFixedWidthHashTableRowHandler.class;
+ if (structure.getHashChannelsCount() <= DEDICATED_ROW_HANDLERS.length) {
+ rowHandlerClass = DEDICATED_ROW_HANDLERS[structure.getHashChannelsCount() - 1];
+ }
+
+ DynamicClassLoader dynamicClassLoader = new DynamicClassLoader(GroupByHashTable.class.getClassLoader());
+
+ try {
+ Class extends HashTableRowHandler> isolatedRowHandlerClass = IsolatedClass.isolateClass(
+ dynamicClassLoader,
+ HashTableRowHandler.class,
+ rowHandlerClass,
+ RowHandlerGroupByHashTable.class);
+
+ Class extends GroupByHashTable> hashTableClass = (Class extends GroupByHashTable>) dynamicClassLoader
+ .loadClass(RowHandlerGroupByHashTable.class.getName());
+
+ HashTableRowHandler rowHandler = isolatedRowHandlerClass.getConstructor(FixedWidthEntryStructure.class).newInstance(structure);
+
+ Constructor extends GroupByHashTable> hashTableConstructor = hashTableClass.getConstructor(HashGenerator.class, HashTableRowHandler.class);
+
+ return hashTableConstructor.newInstance(hashGenerator, rowHandler);
+ }
+ catch (ReflectiveOperationException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private static class HashTableCacheKey
+ extends StructureAndBlockCacheKey
+ {
+ private final HashGenerator hashGenerator;
+
+ private HashTableCacheKey(FixedWidthEntryStructure structure, List> blockTypes, HashGenerator hashGenerator)
+ {
+ super(structure, blockTypes);
+ this.hashGenerator = hashGenerator;
+ }
+
+ public HashGenerator getHashGenerator()
+ {
+ return hashGenerator;
+ }
+ }
+
+ private static class StructureAndBlockCacheKey
+ {
+ private final FixedWidthEntryStructure structure;
+ private final List> blockTypes;
+
+ private StructureAndBlockCacheKey(
+ FixedWidthEntryStructure structure,
+ List> blockTypes)
+ {
+ this.structure = requireNonNull(structure, "structure is null");
+ this.blockTypes = requireNonNull(blockTypes, "blockTypes is null");
+ }
+
+ public FixedWidthEntryStructure getStructure()
+ {
+ return structure;
+ }
+
+ public List> getBlockTypes()
+ {
+ return blockTypes;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof StructureAndBlockCacheKey)) {
+ return false;
+ }
+ StructureAndBlockCacheKey that = (StructureAndBlockCacheKey) o;
+ return structure.equals(that.structure) && blockTypes.equals(that.blockTypes);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(structure, blockTypes);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/RowHandlerGroupByHashTable.java b/core/trino-main/src/main/java/io/trino/operator/hash/RowHandlerGroupByHashTable.java
new file mode 100644
index 000000000000..ce2857e29cfd
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/RowHandlerGroupByHashTable.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+import io.trino.operator.HashGenerator;
+import io.trino.spi.Page;
+
+import static java.util.Objects.requireNonNull;
+
+public class RowHandlerGroupByHashTable
+ implements GroupByHashTable
+{
+ private final HashGenerator hashGenerator;
+ private final HashTableRowHandler rowHandler;
+
+ public RowHandlerGroupByHashTable(HashGenerator hashGenerator, HashTableRowHandler rowHandler)
+ {
+ this.hashGenerator = requireNonNull(hashGenerator, "hashGenerator is null");
+ this.rowHandler = requireNonNull(rowHandler, "rowHandler is null");
+ }
+
+ public int putIfAbsent(HashTableData data, int position, Page page)
+ {
+ long rawHash = hashGenerator.hashPosition(position, page);
+ int hashPosition = data.getHashPosition(rawHash);
+ // look for an empty slot or a slot containing this key
+ while (true) {
+ int current = data.getGroupId(hashPosition);
+
+ if (current == -1) {
+ // empty slot found
+ return addNewGroupId(data, position, page, rawHash, hashPosition);
+ }
+
+ if (rowHandler.keyEquals(data.entries(), data.entriesPosition(hashPosition, current), page, position, rawHash)) {
+ return current;
+ }
+
+ hashPosition = data.nextPosition(hashPosition);
+ data.markCollision();
+ }
+ }
+
+ private int addNewGroupId(HashTableData data, int position, Page page, long rawHash, int hashPosition)
+ {
+ int groupId = data.addNewGroupId(hashPosition);
+ rowHandler.putEntry(data.entries(), data.entriesPosition(hashPosition, groupId), groupId, page, position, rawHash);
+
+ return groupId;
+ }
+
+ @Override
+ public boolean contains(HashTableData data, int position, Page page)
+ {
+ long rawHash = hashGenerator.hashPosition(position, page);
+ return contains(data, position, page, rawHash);
+ }
+
+ @Override
+ public boolean contains(HashTableData data, int position, Page page, long rawHash)
+ {
+ int hashPosition = data.getHashPosition(rawHash);
+ // look for an empty slot or a slot containing this key
+ int current;
+ while ((current = data.getGroupId(hashPosition)) != -1) {
+ if (rowHandler.keyEquals(data.entries(), data.entriesPosition(hashPosition, current), page, position, rawHash)) {
+ // found an existing slot for this key
+ return true;
+ }
+ hashPosition = data.nextPosition(hashPosition);
+ }
+
+ return false;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/WorkFactory.java b/core/trino-main/src/main/java/io/trino/operator/hash/WorkFactory.java
new file mode 100644
index 000000000000..d4ba5d713164
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/WorkFactory.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash;
+
+import io.trino.operator.Work;
+import io.trino.spi.Page;
+
+import java.lang.reflect.Constructor;
+
+public class WorkFactory
+{
+ private Constructor extends Work> constructor;
+
+ public WorkFactory(Class extends Work> workClass)
+ {
+ try {
+ constructor = workClass.getConstructor(HashTableDataGroupByHash.class, GroupByHashTable.class, Page.class);
+ }
+ catch (NoSuchMethodException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ public Work create(HashTableDataGroupByHash groupByHash, GroupByHashTable groupByHashTable, Page page)
+ {
+ try {
+ return constructor.newInstance(groupByHash, groupByHashTable, page);
+ }
+ catch (ReflectiveOperationException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/ArrayFastByteBuffer.java b/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/ArrayFastByteBuffer.java
new file mode 100644
index 000000000000..40a204dd05f9
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/ArrayFastByteBuffer.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fastbb;
+
+import io.airlift.slice.Slice;
+import io.airlift.slice.Slices;
+import sun.misc.Unsafe;
+
+import java.lang.reflect.Field;
+
+public class ArrayFastByteBuffer
+ implements FastByteBuffer
+{
+ private static final Unsafe UNSAFE = getUnsafe();
+ private static final int BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+ private static Unsafe getUnsafe()
+ {
+ try {
+ Field f = Unsafe.class.getDeclaredField("theUnsafe");
+ f.setAccessible(true);
+ return (Unsafe) f.get(null);
+ }
+ catch (Exception ex) {
+ throw new RuntimeException(ex);
+ }
+ }
+
+ private final byte[] array;
+ private final Slice slice; // Slice that wraps the array for operations that require Slice
+ private final int capacity;
+
+ public ArrayFastByteBuffer(int capacity)
+ {
+ this.capacity = capacity;
+ this.array = new byte[capacity];
+ this.slice = Slices.wrappedBuffer(array);
+ }
+
+ @Override
+ public void copyFrom(FastByteBuffer src, int srcPosition, int destPosition, int length)
+ {
+ ArrayFastByteBuffer srcArray = (ArrayFastByteBuffer) src;
+ System.arraycopy(srcArray.array, srcPosition, array, destPosition, length);
+ }
+
+ @Override
+ public void putInt(int position, int value)
+ {
+ UNSAFE.putInt(array, BYTE_ARRAY_BASE_OFFSET + position, value);
+ }
+
+ @Override
+ public int getInt(int position)
+ {
+ return UNSAFE.getInt(array, BYTE_ARRAY_BASE_OFFSET + position);
+ }
+
+ @Override
+ public int capacity()
+ {
+ return capacity;
+ }
+
+ @Override
+ public void putLong(int position, long value)
+ {
+ UNSAFE.putLong(array, BYTE_ARRAY_BASE_OFFSET + position, value);
+ }
+
+ @Override
+ public byte get(int position)
+ {
+ // alternative
+// return UNSAFE.getByte(array, BYTE_ARRAY_BASE_OFFSET + position);
+ return array[position];
+ }
+
+ @Override
+ public void put(int position, byte value)
+ {
+ array[position] = value;
+ // alternative
+// UNSAFE.putByte(array, BYTE_ARRAY_BASE_OFFSET + position, value);
+ }
+
+ @Override
+ public short getShort(int position)
+ {
+ return UNSAFE.getShort(array, BYTE_ARRAY_BASE_OFFSET + position);
+ }
+
+ @Override
+ public void putShort(int position, short value)
+ {
+ UNSAFE.putShort(array, BYTE_ARRAY_BASE_OFFSET + position, value);
+ }
+
+ @Override
+ public Slice asSlice()
+ {
+ return slice;
+ }
+
+ @Override
+ public long getLong(int position)
+ {
+ return UNSAFE.getLong(array, BYTE_ARRAY_BASE_OFFSET + position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/FastByteBuffer.java b/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/FastByteBuffer.java
new file mode 100644
index 000000000000..96bfcd05a529
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/FastByteBuffer.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fastbb;
+
+import io.airlift.slice.Slice;
+
+/**
+ * Like byte array but with fast read/write for other primitive types like int, long, short.
+ */
+public interface FastByteBuffer
+{
+ static FastByteBuffer allocate(int capacity)
+ {
+ return new VarHandleFastByteBuffer(capacity);
+ }
+
+ void copyFrom(FastByteBuffer src, int srcPosition, int destPosition, int length);
+
+ void putInt(int position, int value);
+
+ int getInt(int position);
+
+ int capacity();
+
+ void putLong(int position, long value);
+
+ byte get(int position);
+
+ void put(int position, byte value);
+
+ long getLong(int position);
+
+ default void putByteUnsigned(int position, int value)
+ {
+ put(position, (byte) value);
+ }
+
+ default int getByteUnsigned(int position)
+ {
+ return Byte.toUnsignedInt(get(position));
+ }
+
+ short getShort(int position);
+
+ void putShort(int position, short value);
+
+ Slice asSlice();
+
+ default String toString(int position, int length)
+ {
+ int iMax = position + length - 1;
+ if (iMax == -1) {
+ return "[]";
+ }
+
+ StringBuilder b = new StringBuilder();
+ b.append('[');
+ for (int i = position; ; i++) {
+ b.append(get(i));
+ if (i == iMax) {
+ return b.append(']').toString();
+ }
+ b.append(", ");
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/SliceFastByteBuffer.java b/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/SliceFastByteBuffer.java
new file mode 100644
index 000000000000..562d4362ba3b
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/SliceFastByteBuffer.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fastbb;
+
+import io.airlift.slice.Slice;
+import io.airlift.slice.Slices;
+
+public class SliceFastByteBuffer
+ implements FastByteBuffer
+{
+ private final Slice slice;
+
+ public SliceFastByteBuffer(int capacity)
+ {
+ slice = Slices.allocate(capacity);
+ }
+
+ @Override
+ public void copyFrom(FastByteBuffer src, int srcPosition, int destPosition, int length)
+ {
+ slice.setBytes(destPosition, src.asSlice(), srcPosition, length);
+ }
+
+ @Override
+ public void putInt(int position, int value)
+ {
+ slice.setInt(position, value);
+ }
+
+ @Override
+ public int getInt(int position)
+ {
+ return slice.getInt(position);
+ }
+
+ @Override
+ public int capacity()
+ {
+ return slice.length();
+ }
+
+ @Override
+ public void putLong(int position, long value)
+ {
+ slice.setLong(position, value);
+ }
+
+ @Override
+ public long getLong(int position)
+ {
+ return slice.getLong(position);
+ }
+
+ @Override
+ public byte get(int position)
+ {
+ return slice.getByte(position);
+ }
+
+ @Override
+ public void put(int position, byte value)
+ {
+ slice.setByte(position, value);
+ }
+
+ @Override
+ public short getShort(int position)
+ {
+ return slice.getShort(position);
+ }
+
+ @Override
+ public void putShort(int position, short value)
+ {
+ slice.setShort(position, value);
+ }
+
+ @Override
+ public Slice asSlice()
+ {
+ return slice;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/VarHandleFastByteBuffer.java b/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/VarHandleFastByteBuffer.java
new file mode 100644
index 000000000000..74deb7db4293
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fastbb/VarHandleFastByteBuffer.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fastbb;
+
+import io.airlift.slice.Slice;
+import io.airlift.slice.Slices;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.nio.ByteOrder;
+
+public class VarHandleFastByteBuffer
+ implements FastByteBuffer
+{
+ private static final VarHandle LONG_ARRAY_HANDLE = MethodHandles.byteArrayViewVarHandle(long[].class, ByteOrder.nativeOrder());
+ private static final VarHandle INT_ARRAY_HANDLE = MethodHandles.byteArrayViewVarHandle(int[].class, ByteOrder.nativeOrder());
+ private static final VarHandle SHORT_ARRAY_HANDLE = MethodHandles.byteArrayViewVarHandle(short[].class, ByteOrder.nativeOrder());
+
+ private final byte[] array;
+ private final Slice slice; // Slice that wraps the array for operations that require Slice
+ private final int capacity;
+
+ public VarHandleFastByteBuffer(int capacity)
+ {
+ this.capacity = capacity;
+ this.array = new byte[capacity];
+ this.slice = Slices.wrappedBuffer(array);
+ }
+
+ @Override
+ public void copyFrom(FastByteBuffer src, int srcPosition, int destPosition, int length)
+ {
+ VarHandleFastByteBuffer srcArray = (VarHandleFastByteBuffer) src;
+ System.arraycopy(srcArray.array, srcPosition, array, destPosition, length);
+ }
+
+ @Override
+ public void putInt(int position, int value)
+ {
+ INT_ARRAY_HANDLE.set(array, position, value);
+ }
+
+ @Override
+ public int getInt(int position)
+ {
+ return (int) INT_ARRAY_HANDLE.get(array, position);
+ }
+
+ @Override
+ public int capacity()
+ {
+ return capacity;
+ }
+
+ @Override
+ public void putLong(int position, long value)
+ {
+ LONG_ARRAY_HANDLE.set(array, position, value);
+ }
+
+ @Override
+ public long getLong(int position)
+ {
+ return (long) LONG_ARRAY_HANDLE.get(array, position);
+ }
+
+ @Override
+ public byte get(int position)
+ {
+ return array[position];
+ }
+
+ @Override
+ public void put(int position, byte value)
+ {
+ array[position] = value;
+ }
+
+ @Override
+ public short getShort(int position)
+ {
+ return (short) SHORT_ARRAY_HANDLE.get(array, position);
+ }
+
+ @Override
+ public void putShort(int position, short value)
+ {
+ SHORT_ARRAY_HANDLE.set(array, position, value);
+ }
+
+ @Override
+ public Slice asSlice()
+ {
+ return slice;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/FixedWidthEntryStructure.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/FixedWidthEntryStructure.java
new file mode 100644
index 000000000000..3fee1b38d674
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/FixedWidthEntryStructure.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Value object that defines column types and offset for the values stored in the fixed width hash table row.
+ */
+public class FixedWidthEntryStructure
+{
+ private final int hashChannelsCount;
+ private final ColumnValueExtractor[] columnValueExtractors;
+ private final int[] valuesOffsets;
+ private final int valuesLength;
+
+ public FixedWidthEntryStructure(ColumnValueExtractor[] columnValueExtractors)
+ {
+ this.columnValueExtractors = requireNonNull(columnValueExtractors, "columnValueExtractors is null");
+ this.hashChannelsCount = columnValueExtractors.length;
+
+ valuesOffsets = new int[hashChannelsCount];
+ int mainBufferOffset = 0;
+ for (int i = 0; i < columnValueExtractors.length; i++) {
+ valuesOffsets[i] = mainBufferOffset;
+ checkArgument(columnValueExtractors[i].isFixedSize());
+ mainBufferOffset += columnValueExtractors[i].getSize();
+ }
+ this.valuesLength = mainBufferOffset;
+ }
+
+ public int getHashChannelsCount()
+ {
+ return hashChannelsCount;
+ }
+
+ public ColumnValueExtractor[] getColumnValueExtractors()
+ {
+ return columnValueExtractors;
+ }
+
+ public int[] getValuesOffsets()
+ {
+ return valuesOffsets;
+ }
+
+ public int getValuesLength()
+ {
+ return valuesLength;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ FixedWidthEntryStructure that = (FixedWidthEntryStructure) o;
+ return hashChannelsCount == that.hashChannelsCount && valuesLength == that.valuesLength && Arrays.equals(columnValueExtractors, that.columnValueExtractors) && Arrays.equals(valuesOffsets, that.valuesOffsets);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ int result = Objects.hash(hashChannelsCount, valuesLength);
+ result = 31 * result + Arrays.hashCode(columnValueExtractors);
+ result = 31 * result + Arrays.hashCode(valuesOffsets);
+ return result;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/FixedWidthGroupByHashTableEntries.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/FixedWidthGroupByHashTableEntries.java
new file mode 100644
index 000000000000..00e4acabe4f4
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/FixedWidthGroupByHashTableEntries.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth;
+
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+
+public class FixedWidthGroupByHashTableEntries
+ implements GroupByHashTableEntries
+{
+ // Memory layout per entry
+ // 0 - 4 : group id
+ // 4 - 12 : hash
+ // 12 - 12 + channelCount : isNull
+ // 12 + channelCount - 12 + channelCount + valuesLength : values
+
+ private final FastByteBuffer buffer;
+ private final int channelCount;
+ private final int maxEntryCount;
+ private final int entrySize;
+
+ private final int hashOffset;
+ private final int isNullOffset;
+ private final int contentLength; // number of bytes from the hash offset to the end of the entry
+ private final int valuesOffset;
+ private final int valuesLength;
+
+ public static FixedWidthGroupByHashTableEntries allocate(
+ int maxEntryCount,
+ int channelCount,
+ int valuesLength)
+ {
+ // first 4 bytes are groupId
+ int hashOffset = Integer.BYTES;
+ int isNullOffset = hashOffset + Long.BYTES;
+ int valuesOffset = isNullOffset + channelCount /* isNull */;
+ int entrySize = valuesOffset + valuesLength;
+ int contentLength = entrySize - hashOffset;
+ FastByteBuffer mainBuffer = createBuffer(maxEntryCount, entrySize);
+ return new FixedWidthGroupByHashTableEntries(
+ mainBuffer,
+ channelCount,
+ maxEntryCount,
+ entrySize,
+ hashOffset,
+ isNullOffset,
+ contentLength,
+ valuesOffset,
+ valuesLength);
+ }
+
+ private FixedWidthGroupByHashTableEntries(
+ FastByteBuffer buffer,
+ int channelCount,
+ int maxEntryCount,
+ int entrySize,
+ int hashOffset,
+ int isNullOffset,
+ int contentLength,
+ int valuesOffset,
+ int valuesLength)
+ {
+ this.buffer = buffer;
+ this.channelCount = channelCount;
+ this.maxEntryCount = maxEntryCount;
+ this.entrySize = entrySize;
+ this.hashOffset = hashOffset;
+ this.isNullOffset = isNullOffset;
+ this.contentLength = contentLength;
+ this.valuesOffset = valuesOffset;
+ this.valuesLength = valuesLength;
+ }
+
+ private static FastByteBuffer createBuffer(int entryCount, int entrySize)
+ {
+ final FastByteBuffer mainBuffer = FastByteBuffer.allocate(entryCount * entrySize);
+ // set groupIds to -1
+ for (int i = 0; i <= mainBuffer.capacity() - entrySize; i += entrySize) {
+ mainBuffer.putInt(i, -1);
+ }
+
+ return mainBuffer;
+ }
+
+ @Override
+ public int getEntrySize()
+ {
+ return entrySize;
+ }
+
+ @Override
+ public long getHash(int position)
+ {
+ return buffer.getLong(position + hashOffset);
+ }
+
+ @Override
+ public void putHash(int position, long hash)
+ {
+ buffer.putLong(position + hashOffset, hash);
+ }
+
+ public void copyEntryFrom(FixedWidthGroupByHashTableEntries src, int srcPosition, int toPosition)
+ {
+ buffer.copyFrom(src.buffer, srcPosition, toPosition, src.getEntrySize());
+ }
+
+ public int getValuesOffset(int position)
+ {
+ return position + valuesOffset;
+ }
+
+ @Override
+ public byte isNull(int position, int i)
+ {
+ return buffer.get(position + isNullOffset + i);
+ }
+
+ public void putIsNull(int position, int channelIndex, byte isNull)
+ {
+ buffer.put(position + isNullOffset + channelIndex, isNull);
+ }
+
+ public void putIsNull(int position, byte[] isNull)
+ {
+ for (int i = 0; i < isNull.length; i++) {
+ buffer.put(position + isNullOffset + i, isNull[i]);
+ }
+ }
+
+ @Override
+ public int getGroupId(int position)
+ {
+ return buffer.getInt(position);
+ }
+
+ @Override
+ public void putGroupId(int position, int groupId)
+ {
+ buffer.putInt(position, groupId);
+ }
+
+ @Override
+ public int capacity()
+ {
+ return buffer.capacity();
+ }
+
+ @Override
+ public long getEstimatedSize()
+ {
+ return buffer.capacity();
+ }
+
+ @Override
+ public String toString(int position)
+ {
+ StringBuilder sb = new StringBuilder("[");
+
+ sb.append("groupId=").append(getGroupId(position)).append("\n");
+
+ sb.append("hash=").append(getHash(position)).append("\n");
+ sb.append("isNull=[");
+ for (int i = 0; i < channelCount; i++) {
+ sb.append(isNull(position, i)).append(", ");
+ }
+ sb.append("]").append("\n");
+ sb.append("values=[");
+ sb.append(buffer.toString(position + valuesOffset, valuesLength));
+ sb.append("]").append("\n");
+
+ return sb.toString();
+ }
+
+ public FastByteBuffer getBuffer()
+ {
+ return buffer;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/FixedWidthHashTableData.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/FixedWidthHashTableData.java
new file mode 100644
index 000000000000..56e219b3af7b
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/FixedWidthHashTableData.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth;
+
+import io.trino.operator.hash.HashTableData;
+
+import static io.trino.operator.hash.HashTableDataGroupByHash.calculateMaxFill;
+
+public abstract class FixedWidthHashTableData
+ implements HashTableData
+{
+ private final int hashCapacity;
+ private final int maxFill;
+
+ protected final int hashChannelsCount;
+ protected final int mask;
+ protected final int valuesLength;
+ protected final FixedWidthGroupByHashTableEntries entries;
+ protected final int entrySize;
+
+ protected int hashTableSize;
+ protected long hashCollisions;
+
+ public FixedWidthHashTableData(int hashChannelsCount, int hashCapacity, int valuesLength, long hashCollisions, FixedWidthGroupByHashTableEntries entries)
+ {
+ this.hashChannelsCount = hashChannelsCount;
+ this.hashCapacity = hashCapacity;
+ this.valuesLength = valuesLength;
+ this.hashCollisions = hashCollisions;
+ this.maxFill = calculateMaxFill(hashCapacity);
+ this.mask = hashCapacity - 1;
+ this.entries = entries;
+ this.entrySize = entries.getEntrySize();
+ }
+
+ public int getHashTableSize()
+ {
+ return hashTableSize;
+ }
+
+ @Override
+ public boolean needRehash()
+ {
+ return hashTableSize >= maxFill;
+ }
+
+ public int maxFill()
+ {
+ return maxFill;
+ }
+
+ @Override
+ public FixedWidthGroupByHashTableEntries entries()
+ {
+ return entries;
+ }
+
+ @Override
+ public void markCollision()
+ {
+ hashCollisions++;
+ }
+
+ public long getHashCollisions()
+ {
+ return hashCollisions;
+ }
+
+ public long getRawHash(int groupId)
+ {
+ return entries.getHash(getPosition(groupId));
+ }
+
+ public int getCapacity()
+ {
+ return hashCapacity;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/GenericFixedWidthHashTableRowHandler.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/GenericFixedWidthHashTableRowHandler.java
new file mode 100644
index 000000000000..65612c8eb32a
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/GenericFixedWidthHashTableRowHandler.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static java.util.Objects.requireNonNull;
+
+public class GenericFixedWidthHashTableRowHandler
+ implements HashTableRowHandler
+{
+ private final FixedWidthEntryStructure structure;
+
+ public GenericFixedWidthHashTableRowHandler(FixedWidthEntryStructure structure)
+ {
+ this.structure = requireNonNull(structure, "structure is null");
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page, int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+
+ ColumnValueExtractor[] columnValueExtractors = structure.getColumnValueExtractors();
+ int[] valuesOffsets = structure.getValuesOffsets();
+ for (int i = 0; i < structure.getHashChannelsCount(); i++) {
+ Block block = page.getBlock(i);
+ byte valueIsNull = (byte) (block.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, i, valueIsNull);
+ columnValueExtractors[i].putValue(buffer, valuesOffset + valuesOffsets[i], block, position);
+ }
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries entries, int entriesPosition, Page page, int position, long rawHash)
+ {
+// if (rawHash != entries.getHash(entriesPosition)) {
+// return false;
+// }
+
+ FixedWidthGroupByHashTableEntries table = (FixedWidthGroupByHashTableEntries) entries;
+
+ int valuesOffset = table.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = table.getBuffer();
+ ColumnValueExtractor[] columnValueExtractors = structure.getColumnValueExtractors();
+ int[] valuesOffsets = structure.getValuesOffsets();
+ for (int i = 0; i < structure.getHashChannelsCount(); i++) {
+ Block block = page.getBlock(i);
+
+ boolean blockValueNull = block.isNull(position);
+ boolean tableValueIsNull = table.isNull(entriesPosition, i) == 1;
+
+ if (!(blockValueNull == tableValueIsNull &&
+ columnValueExtractors[i].valueEquals(buffer, valuesOffset + valuesOffsets[i], block, position))) {
+ return false;
+ }
+ }
+ return true;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/GenericFixedWidthHashTableValuesAppender.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/GenericFixedWidthHashTableValuesAppender.java
new file mode 100644
index 000000000000..24e555d5a7c5
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/GenericFixedWidthHashTableValuesAppender.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+public class GenericFixedWidthHashTableValuesAppender
+ extends AbstractHashTableValuesAppender
+{
+ private final FixedWidthEntryStructure structure;
+
+ public GenericFixedWidthHashTableValuesAppender(FixedWidthEntryStructure structure)
+ {
+ this.structure = requireNonNull(structure, "structure is null");
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ ColumnValueExtractor[] columnValueExtractors = structure.getColumnValueExtractors();
+ int[] valuesOffsets = structure.getValuesOffsets();
+
+ for (int i = 0; i < structure.getHashChannelsCount(); i++, outputChannelOffset++) {
+ BlockBuilder blockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, i) == 1) {
+ blockBuilder.appendNull();
+ }
+ else {
+ columnValueExtractors[i].appendValue(buffer, valuesOffset + valuesOffsets[i], blockBuilder);
+ }
+ }
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/SeparateTableHashTableData.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/SeparateTableHashTableData.java
new file mode 100644
index 000000000000..94cbb236d608
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/SeparateTableHashTableData.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth;
+
+import java.util.Arrays;
+
+import static io.airlift.slice.SizeOf.sizeOf;
+import static io.trino.operator.hash.HashTableDataGroupByHash.calculateMaxFill;
+import static it.unimi.dsi.fastutil.HashCommon.murmurHash3;
+
+public class SeparateTableHashTableData
+ extends FixedWidthHashTableData
+{
+ // hash table that keeps group ids
+ private final int[] hashTable;
+
+ public SeparateTableHashTableData(int hashChannelsCount, int hashCapacity, int valuesLength)
+ {
+ this(hashChannelsCount, hashCapacity, valuesLength, 0);
+ }
+
+ public SeparateTableHashTableData(int hashChannelsCount, int hashCapacity, int valuesLength, long hashCollisions)
+ {
+ super(hashChannelsCount, hashCapacity, valuesLength, hashCollisions, FixedWidthGroupByHashTableEntries.allocate(calculateMaxFill(hashCapacity), hashChannelsCount, valuesLength));
+ this.hashTable = new int[hashCapacity];
+ Arrays.fill(hashTable, -1);
+ }
+
+ private void copyFrom(SeparateTableHashTableData other)
+ {
+ FixedWidthGroupByHashTableEntries otherEntries = other.entries();
+ FixedWidthGroupByHashTableEntries thisEntries = this.entries();
+ int[] otherHashTable = other.hashTable;
+ int[] thisHashTable = this.hashTable;
+ for (int i = 0; i < otherHashTable.length; i++) {
+ int groupId = otherHashTable[i];
+ if (groupId != -1) {
+ int entriesPosition = groupId * otherEntries.getEntrySize();
+ int hashPosition = getHashPosition(otherEntries.getHash(entriesPosition));
+ // look for an empty slot or a slot containing this key
+ while (thisHashTable[hashPosition] != -1) {
+ hashPosition = nextPosition(hashPosition);
+ hashCollisions++;
+ }
+ thisHashTable[hashPosition] = groupId;
+ }
+ }
+ // TODO lysy: we dont need to copy this (can chain arrays like in BigArray). Not sure if it's better though (we need to pay for BigInt indirection on every access).
+ thisEntries.getBuffer().copyFrom(otherEntries.getBuffer(), 0, 0, other.getHashTableSize() * entrySize);
+
+ hashTableSize += other.getHashTableSize();
+ }
+
+ @Override
+ public int getPosition(int groupId)
+ {
+ return groupId * entrySize;
+ }
+
+ @Override
+ public int getHashPosition(long rawHash)
+ {
+ return (int) (murmurHash3(rawHash) & mask);
+ }
+
+ @Override
+ public int getGroupId(int hashPosition)
+ {
+ return hashTable[hashPosition];
+ }
+
+ @Override
+ public int addNewGroupId(int hashPosition)
+ {
+ int groupId = hashTableSize++;
+ hashTable[hashPosition] = groupId;
+ return groupId;
+ }
+
+ @Override
+ public int nextPosition(int hashPosition)
+ {
+ return (hashPosition + 1) & mask;
+ }
+
+ public SeparateTableHashTableData resize(int newCapacity)
+ {
+ SeparateTableHashTableData newHashTableData = new SeparateTableHashTableData(
+ hashChannelsCount,
+ newCapacity,
+ valuesLength,
+ getHashCollisions());
+
+ newHashTableData.copyFrom(this);
+ return newHashTableData;
+ }
+
+ @Override
+ public int entriesPosition(int hashPosition, int groupId)
+ {
+ return getPosition(groupId);
+ }
+
+ public long getEstimatedSize()
+ {
+ return entries.getEstimatedSize() + sizeOf(hashTable);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/SingleTableHashTableData.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/SingleTableHashTableData.java
new file mode 100644
index 000000000000..85c43aa6001c
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/SingleTableHashTableData.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth;
+
+import java.util.Arrays;
+
+import static io.airlift.slice.SizeOf.sizeOf;
+import static it.unimi.dsi.fastutil.HashCommon.murmurHash3;
+
+public class SingleTableHashTableData
+ extends FixedWidthHashTableData
+{
+ private final int[] groupToHashPosition;
+
+ public SingleTableHashTableData(int hashChannelsCount, int hashCapacity, int valuesLength)
+ {
+ this(hashChannelsCount, hashCapacity, valuesLength, 0, new int[0]);
+ }
+
+ public SingleTableHashTableData(int hashChannelsCount, int hashCapacity, int valuesLength, long hashCollisions, int[] groupToHashPosition)
+ {
+ super(hashChannelsCount, hashCapacity, valuesLength, hashCollisions, FixedWidthGroupByHashTableEntries.allocate(hashCapacity, hashChannelsCount, valuesLength));
+ this.groupToHashPosition = groupToHashPosition.length >= maxFill() ? groupToHashPosition : Arrays.copyOf(groupToHashPosition, maxFill());
+ }
+
+ private void copyFrom(SingleTableHashTableData other)
+ {
+ FixedWidthGroupByHashTableEntries otherHashTable = other.entries;
+ FixedWidthGroupByHashTableEntries thisHashTable = this.entries;
+ int entrySize = entries.getEntrySize();
+ for (int i = 0; i <= otherHashTable.capacity() - entrySize; i += entrySize) {
+ if (otherHashTable.getGroupId(i) != -1) {
+ int hashPosition = getHashPosition(otherHashTable.getHash(i));
+ // look for an empty slot or a slot containing this key
+ while (thisHashTable.getGroupId(hashPosition) != -1) {
+ hashPosition = hashPosition + entrySize;
+ if (hashPosition >= thisHashTable.capacity()) {
+ hashPosition = 0;
+ }
+ hashCollisions++;
+ }
+ thisHashTable.copyEntryFrom(otherHashTable, i, hashPosition);
+ groupToHashPosition[otherHashTable.getGroupId(i)] = hashPosition;
+ }
+ }
+ hashTableSize += other.getHashTableSize();
+ }
+
+ @Override
+ public int getPosition(int groupId)
+ {
+ return groupToHashPosition[groupId];
+ }
+
+ @Override
+ public int getHashPosition(long rawHash)
+ {
+ return (int) (murmurHash3(rawHash) & mask) * entries.getEntrySize();
+ }
+
+ @Override
+ public int getGroupId(int hashPosition)
+ {
+ return entries.getGroupId(hashPosition);
+ }
+
+ @Override
+ public int addNewGroupId(int hashPosition)
+ {
+ int groupId = hashTableSize++;
+ groupToHashPosition[groupId] = hashPosition;
+ return groupId;
+ }
+
+ @Override
+ public int nextPosition(int hashPosition)
+ {
+ hashPosition = hashPosition + entrySize;
+ if (hashPosition >= entries.capacity()) {
+ hashPosition = 0;
+ }
+ return hashPosition;
+ }
+
+ public SingleTableHashTableData resize(int newCapacity)
+ {
+ SingleTableHashTableData newHashTableData = new SingleTableHashTableData(
+ hashChannelsCount,
+ newCapacity,
+ valuesLength, hashCollisions,
+ groupToHashPosition);
+
+ newHashTableData.copyFrom(this);
+ return newHashTableData;
+ }
+
+ @Override
+ public int entriesPosition(int hashPosition, int groupId)
+ {
+ return hashPosition;
+ }
+
+ public long getEstimatedSize()
+ {
+ return entries.getEstimatedSize() + sizeOf(groupToHashPosition);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler10Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler10Channels.java
new file mode 100644
index 000000000000..bb066632f2f4
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler10Channels.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler10Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ public FixedWidthHashTableRowHandler10Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 10);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ return block9ValueNull || columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler11Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler11Channels.java
new file mode 100644
index 000000000000..912bfa38ad3d
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler11Channels.java
@@ -0,0 +1,264 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler11Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ public FixedWidthHashTableRowHandler11Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 11);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ Block block10 = page.getBlock(10);
+ byte value10IsNull = (byte) (block10.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 10, value10IsNull);
+ columnValueExtractor10.putValue(buffer, valuesOffset + value10Offset, block10, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ if (!block9ValueNull && !columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position)) {
+ return false;
+ }
+ Block block10 = page.getBlock(10);
+ boolean block10ValueNull = block10.isNull(position);
+ boolean entriesValue10IsNull = entries.isNull(entriesPosition, 10) == 1;
+ if (block10ValueNull != entriesValue10IsNull) {
+ return false;
+ }
+ return block10ValueNull || columnValueExtractor10.valueEquals(buffer, valuesOffset + value10Offset, block10, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler12Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler12Channels.java
new file mode 100644
index 000000000000..934c4362c740
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler12Channels.java
@@ -0,0 +1,283 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler12Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ public FixedWidthHashTableRowHandler12Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 12);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ Block block10 = page.getBlock(10);
+ byte value10IsNull = (byte) (block10.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 10, value10IsNull);
+ columnValueExtractor10.putValue(buffer, valuesOffset + value10Offset, block10, position);
+ Block block11 = page.getBlock(11);
+ byte value11IsNull = (byte) (block11.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 11, value11IsNull);
+ columnValueExtractor11.putValue(buffer, valuesOffset + value11Offset, block11, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ if (!block9ValueNull && !columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position)) {
+ return false;
+ }
+ Block block10 = page.getBlock(10);
+ boolean block10ValueNull = block10.isNull(position);
+ boolean entriesValue10IsNull = entries.isNull(entriesPosition, 10) == 1;
+ if (block10ValueNull != entriesValue10IsNull) {
+ return false;
+ }
+ if (!block10ValueNull && !columnValueExtractor10.valueEquals(buffer, valuesOffset + value10Offset, block10, position)) {
+ return false;
+ }
+ Block block11 = page.getBlock(11);
+ boolean block11ValueNull = block11.isNull(position);
+ boolean entriesValue11IsNull = entries.isNull(entriesPosition, 11) == 1;
+ if (block11ValueNull != entriesValue11IsNull) {
+ return false;
+ }
+ return block11ValueNull || columnValueExtractor11.valueEquals(buffer, valuesOffset + value11Offset, block11, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler13Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler13Channels.java
new file mode 100644
index 000000000000..ce5851a31717
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler13Channels.java
@@ -0,0 +1,302 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler13Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ public FixedWidthHashTableRowHandler13Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 13);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ Block block10 = page.getBlock(10);
+ byte value10IsNull = (byte) (block10.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 10, value10IsNull);
+ columnValueExtractor10.putValue(buffer, valuesOffset + value10Offset, block10, position);
+ Block block11 = page.getBlock(11);
+ byte value11IsNull = (byte) (block11.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 11, value11IsNull);
+ columnValueExtractor11.putValue(buffer, valuesOffset + value11Offset, block11, position);
+ Block block12 = page.getBlock(12);
+ byte value12IsNull = (byte) (block12.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 12, value12IsNull);
+ columnValueExtractor12.putValue(buffer, valuesOffset + value12Offset, block12, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ if (!block9ValueNull && !columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position)) {
+ return false;
+ }
+ Block block10 = page.getBlock(10);
+ boolean block10ValueNull = block10.isNull(position);
+ boolean entriesValue10IsNull = entries.isNull(entriesPosition, 10) == 1;
+ if (block10ValueNull != entriesValue10IsNull) {
+ return false;
+ }
+ if (!block10ValueNull && !columnValueExtractor10.valueEquals(buffer, valuesOffset + value10Offset, block10, position)) {
+ return false;
+ }
+ Block block11 = page.getBlock(11);
+ boolean block11ValueNull = block11.isNull(position);
+ boolean entriesValue11IsNull = entries.isNull(entriesPosition, 11) == 1;
+ if (block11ValueNull != entriesValue11IsNull) {
+ return false;
+ }
+ if (!block11ValueNull && !columnValueExtractor11.valueEquals(buffer, valuesOffset + value11Offset, block11, position)) {
+ return false;
+ }
+ Block block12 = page.getBlock(12);
+ boolean block12ValueNull = block12.isNull(position);
+ boolean entriesValue12IsNull = entries.isNull(entriesPosition, 12) == 1;
+ if (block12ValueNull != entriesValue12IsNull) {
+ return false;
+ }
+ return block12ValueNull || columnValueExtractor12.valueEquals(buffer, valuesOffset + value12Offset, block12, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler14Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler14Channels.java
new file mode 100644
index 000000000000..ad88d90356e8
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler14Channels.java
@@ -0,0 +1,321 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler14Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ public FixedWidthHashTableRowHandler14Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 14);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ Block block10 = page.getBlock(10);
+ byte value10IsNull = (byte) (block10.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 10, value10IsNull);
+ columnValueExtractor10.putValue(buffer, valuesOffset + value10Offset, block10, position);
+ Block block11 = page.getBlock(11);
+ byte value11IsNull = (byte) (block11.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 11, value11IsNull);
+ columnValueExtractor11.putValue(buffer, valuesOffset + value11Offset, block11, position);
+ Block block12 = page.getBlock(12);
+ byte value12IsNull = (byte) (block12.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 12, value12IsNull);
+ columnValueExtractor12.putValue(buffer, valuesOffset + value12Offset, block12, position);
+ Block block13 = page.getBlock(13);
+ byte value13IsNull = (byte) (block13.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 13, value13IsNull);
+ columnValueExtractor13.putValue(buffer, valuesOffset + value13Offset, block13, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ if (!block9ValueNull && !columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position)) {
+ return false;
+ }
+ Block block10 = page.getBlock(10);
+ boolean block10ValueNull = block10.isNull(position);
+ boolean entriesValue10IsNull = entries.isNull(entriesPosition, 10) == 1;
+ if (block10ValueNull != entriesValue10IsNull) {
+ return false;
+ }
+ if (!block10ValueNull && !columnValueExtractor10.valueEquals(buffer, valuesOffset + value10Offset, block10, position)) {
+ return false;
+ }
+ Block block11 = page.getBlock(11);
+ boolean block11ValueNull = block11.isNull(position);
+ boolean entriesValue11IsNull = entries.isNull(entriesPosition, 11) == 1;
+ if (block11ValueNull != entriesValue11IsNull) {
+ return false;
+ }
+ if (!block11ValueNull && !columnValueExtractor11.valueEquals(buffer, valuesOffset + value11Offset, block11, position)) {
+ return false;
+ }
+ Block block12 = page.getBlock(12);
+ boolean block12ValueNull = block12.isNull(position);
+ boolean entriesValue12IsNull = entries.isNull(entriesPosition, 12) == 1;
+ if (block12ValueNull != entriesValue12IsNull) {
+ return false;
+ }
+ if (!block12ValueNull && !columnValueExtractor12.valueEquals(buffer, valuesOffset + value12Offset, block12, position)) {
+ return false;
+ }
+ Block block13 = page.getBlock(13);
+ boolean block13ValueNull = block13.isNull(position);
+ boolean entriesValue13IsNull = entries.isNull(entriesPosition, 13) == 1;
+ if (block13ValueNull != entriesValue13IsNull) {
+ return false;
+ }
+ return block13ValueNull || columnValueExtractor13.valueEquals(buffer, valuesOffset + value13Offset, block13, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler15Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler15Channels.java
new file mode 100644
index 000000000000..ca9e69446242
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler15Channels.java
@@ -0,0 +1,340 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler15Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ public FixedWidthHashTableRowHandler15Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 15);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ Block block10 = page.getBlock(10);
+ byte value10IsNull = (byte) (block10.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 10, value10IsNull);
+ columnValueExtractor10.putValue(buffer, valuesOffset + value10Offset, block10, position);
+ Block block11 = page.getBlock(11);
+ byte value11IsNull = (byte) (block11.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 11, value11IsNull);
+ columnValueExtractor11.putValue(buffer, valuesOffset + value11Offset, block11, position);
+ Block block12 = page.getBlock(12);
+ byte value12IsNull = (byte) (block12.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 12, value12IsNull);
+ columnValueExtractor12.putValue(buffer, valuesOffset + value12Offset, block12, position);
+ Block block13 = page.getBlock(13);
+ byte value13IsNull = (byte) (block13.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 13, value13IsNull);
+ columnValueExtractor13.putValue(buffer, valuesOffset + value13Offset, block13, position);
+ Block block14 = page.getBlock(14);
+ byte value14IsNull = (byte) (block14.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 14, value14IsNull);
+ columnValueExtractor14.putValue(buffer, valuesOffset + value14Offset, block14, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ if (!block9ValueNull && !columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position)) {
+ return false;
+ }
+ Block block10 = page.getBlock(10);
+ boolean block10ValueNull = block10.isNull(position);
+ boolean entriesValue10IsNull = entries.isNull(entriesPosition, 10) == 1;
+ if (block10ValueNull != entriesValue10IsNull) {
+ return false;
+ }
+ if (!block10ValueNull && !columnValueExtractor10.valueEquals(buffer, valuesOffset + value10Offset, block10, position)) {
+ return false;
+ }
+ Block block11 = page.getBlock(11);
+ boolean block11ValueNull = block11.isNull(position);
+ boolean entriesValue11IsNull = entries.isNull(entriesPosition, 11) == 1;
+ if (block11ValueNull != entriesValue11IsNull) {
+ return false;
+ }
+ if (!block11ValueNull && !columnValueExtractor11.valueEquals(buffer, valuesOffset + value11Offset, block11, position)) {
+ return false;
+ }
+ Block block12 = page.getBlock(12);
+ boolean block12ValueNull = block12.isNull(position);
+ boolean entriesValue12IsNull = entries.isNull(entriesPosition, 12) == 1;
+ if (block12ValueNull != entriesValue12IsNull) {
+ return false;
+ }
+ if (!block12ValueNull && !columnValueExtractor12.valueEquals(buffer, valuesOffset + value12Offset, block12, position)) {
+ return false;
+ }
+ Block block13 = page.getBlock(13);
+ boolean block13ValueNull = block13.isNull(position);
+ boolean entriesValue13IsNull = entries.isNull(entriesPosition, 13) == 1;
+ if (block13ValueNull != entriesValue13IsNull) {
+ return false;
+ }
+ if (!block13ValueNull && !columnValueExtractor13.valueEquals(buffer, valuesOffset + value13Offset, block13, position)) {
+ return false;
+ }
+ Block block14 = page.getBlock(14);
+ boolean block14ValueNull = block14.isNull(position);
+ boolean entriesValue14IsNull = entries.isNull(entriesPosition, 14) == 1;
+ if (block14ValueNull != entriesValue14IsNull) {
+ return false;
+ }
+ return block14ValueNull || columnValueExtractor14.valueEquals(buffer, valuesOffset + value14Offset, block14, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler16Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler16Channels.java
new file mode 100644
index 000000000000..bdc37b0c56a9
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler16Channels.java
@@ -0,0 +1,359 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler16Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final ColumnValueExtractor columnValueExtractor15;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ private final int value15Offset;
+
+ public FixedWidthHashTableRowHandler16Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 16);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ columnValueExtractor15 = structure.getColumnValueExtractors()[15];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ value15Offset = structure.getValuesOffsets()[15];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ Block block10 = page.getBlock(10);
+ byte value10IsNull = (byte) (block10.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 10, value10IsNull);
+ columnValueExtractor10.putValue(buffer, valuesOffset + value10Offset, block10, position);
+ Block block11 = page.getBlock(11);
+ byte value11IsNull = (byte) (block11.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 11, value11IsNull);
+ columnValueExtractor11.putValue(buffer, valuesOffset + value11Offset, block11, position);
+ Block block12 = page.getBlock(12);
+ byte value12IsNull = (byte) (block12.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 12, value12IsNull);
+ columnValueExtractor12.putValue(buffer, valuesOffset + value12Offset, block12, position);
+ Block block13 = page.getBlock(13);
+ byte value13IsNull = (byte) (block13.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 13, value13IsNull);
+ columnValueExtractor13.putValue(buffer, valuesOffset + value13Offset, block13, position);
+ Block block14 = page.getBlock(14);
+ byte value14IsNull = (byte) (block14.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 14, value14IsNull);
+ columnValueExtractor14.putValue(buffer, valuesOffset + value14Offset, block14, position);
+ Block block15 = page.getBlock(15);
+ byte value15IsNull = (byte) (block15.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 15, value15IsNull);
+ columnValueExtractor15.putValue(buffer, valuesOffset + value15Offset, block15, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ if (!block9ValueNull && !columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position)) {
+ return false;
+ }
+ Block block10 = page.getBlock(10);
+ boolean block10ValueNull = block10.isNull(position);
+ boolean entriesValue10IsNull = entries.isNull(entriesPosition, 10) == 1;
+ if (block10ValueNull != entriesValue10IsNull) {
+ return false;
+ }
+ if (!block10ValueNull && !columnValueExtractor10.valueEquals(buffer, valuesOffset + value10Offset, block10, position)) {
+ return false;
+ }
+ Block block11 = page.getBlock(11);
+ boolean block11ValueNull = block11.isNull(position);
+ boolean entriesValue11IsNull = entries.isNull(entriesPosition, 11) == 1;
+ if (block11ValueNull != entriesValue11IsNull) {
+ return false;
+ }
+ if (!block11ValueNull && !columnValueExtractor11.valueEquals(buffer, valuesOffset + value11Offset, block11, position)) {
+ return false;
+ }
+ Block block12 = page.getBlock(12);
+ boolean block12ValueNull = block12.isNull(position);
+ boolean entriesValue12IsNull = entries.isNull(entriesPosition, 12) == 1;
+ if (block12ValueNull != entriesValue12IsNull) {
+ return false;
+ }
+ if (!block12ValueNull && !columnValueExtractor12.valueEquals(buffer, valuesOffset + value12Offset, block12, position)) {
+ return false;
+ }
+ Block block13 = page.getBlock(13);
+ boolean block13ValueNull = block13.isNull(position);
+ boolean entriesValue13IsNull = entries.isNull(entriesPosition, 13) == 1;
+ if (block13ValueNull != entriesValue13IsNull) {
+ return false;
+ }
+ if (!block13ValueNull && !columnValueExtractor13.valueEquals(buffer, valuesOffset + value13Offset, block13, position)) {
+ return false;
+ }
+ Block block14 = page.getBlock(14);
+ boolean block14ValueNull = block14.isNull(position);
+ boolean entriesValue14IsNull = entries.isNull(entriesPosition, 14) == 1;
+ if (block14ValueNull != entriesValue14IsNull) {
+ return false;
+ }
+ if (!block14ValueNull && !columnValueExtractor14.valueEquals(buffer, valuesOffset + value14Offset, block14, position)) {
+ return false;
+ }
+ Block block15 = page.getBlock(15);
+ boolean block15ValueNull = block15.isNull(position);
+ boolean entriesValue15IsNull = entries.isNull(entriesPosition, 15) == 1;
+ if (block15ValueNull != entriesValue15IsNull) {
+ return false;
+ }
+ return block15ValueNull || columnValueExtractor15.valueEquals(buffer, valuesOffset + value15Offset, block15, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler17Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler17Channels.java
new file mode 100644
index 000000000000..3bb85e2bbbdb
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler17Channels.java
@@ -0,0 +1,378 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler17Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final ColumnValueExtractor columnValueExtractor15;
+
+ private final ColumnValueExtractor columnValueExtractor16;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ private final int value15Offset;
+
+ private final int value16Offset;
+
+ public FixedWidthHashTableRowHandler17Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 17);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ columnValueExtractor15 = structure.getColumnValueExtractors()[15];
+ columnValueExtractor16 = structure.getColumnValueExtractors()[16];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ value15Offset = structure.getValuesOffsets()[15];
+ value16Offset = structure.getValuesOffsets()[16];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ Block block10 = page.getBlock(10);
+ byte value10IsNull = (byte) (block10.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 10, value10IsNull);
+ columnValueExtractor10.putValue(buffer, valuesOffset + value10Offset, block10, position);
+ Block block11 = page.getBlock(11);
+ byte value11IsNull = (byte) (block11.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 11, value11IsNull);
+ columnValueExtractor11.putValue(buffer, valuesOffset + value11Offset, block11, position);
+ Block block12 = page.getBlock(12);
+ byte value12IsNull = (byte) (block12.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 12, value12IsNull);
+ columnValueExtractor12.putValue(buffer, valuesOffset + value12Offset, block12, position);
+ Block block13 = page.getBlock(13);
+ byte value13IsNull = (byte) (block13.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 13, value13IsNull);
+ columnValueExtractor13.putValue(buffer, valuesOffset + value13Offset, block13, position);
+ Block block14 = page.getBlock(14);
+ byte value14IsNull = (byte) (block14.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 14, value14IsNull);
+ columnValueExtractor14.putValue(buffer, valuesOffset + value14Offset, block14, position);
+ Block block15 = page.getBlock(15);
+ byte value15IsNull = (byte) (block15.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 15, value15IsNull);
+ columnValueExtractor15.putValue(buffer, valuesOffset + value15Offset, block15, position);
+ Block block16 = page.getBlock(16);
+ byte value16IsNull = (byte) (block16.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 16, value16IsNull);
+ columnValueExtractor16.putValue(buffer, valuesOffset + value16Offset, block16, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ if (!block9ValueNull && !columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position)) {
+ return false;
+ }
+ Block block10 = page.getBlock(10);
+ boolean block10ValueNull = block10.isNull(position);
+ boolean entriesValue10IsNull = entries.isNull(entriesPosition, 10) == 1;
+ if (block10ValueNull != entriesValue10IsNull) {
+ return false;
+ }
+ if (!block10ValueNull && !columnValueExtractor10.valueEquals(buffer, valuesOffset + value10Offset, block10, position)) {
+ return false;
+ }
+ Block block11 = page.getBlock(11);
+ boolean block11ValueNull = block11.isNull(position);
+ boolean entriesValue11IsNull = entries.isNull(entriesPosition, 11) == 1;
+ if (block11ValueNull != entriesValue11IsNull) {
+ return false;
+ }
+ if (!block11ValueNull && !columnValueExtractor11.valueEquals(buffer, valuesOffset + value11Offset, block11, position)) {
+ return false;
+ }
+ Block block12 = page.getBlock(12);
+ boolean block12ValueNull = block12.isNull(position);
+ boolean entriesValue12IsNull = entries.isNull(entriesPosition, 12) == 1;
+ if (block12ValueNull != entriesValue12IsNull) {
+ return false;
+ }
+ if (!block12ValueNull && !columnValueExtractor12.valueEquals(buffer, valuesOffset + value12Offset, block12, position)) {
+ return false;
+ }
+ Block block13 = page.getBlock(13);
+ boolean block13ValueNull = block13.isNull(position);
+ boolean entriesValue13IsNull = entries.isNull(entriesPosition, 13) == 1;
+ if (block13ValueNull != entriesValue13IsNull) {
+ return false;
+ }
+ if (!block13ValueNull && !columnValueExtractor13.valueEquals(buffer, valuesOffset + value13Offset, block13, position)) {
+ return false;
+ }
+ Block block14 = page.getBlock(14);
+ boolean block14ValueNull = block14.isNull(position);
+ boolean entriesValue14IsNull = entries.isNull(entriesPosition, 14) == 1;
+ if (block14ValueNull != entriesValue14IsNull) {
+ return false;
+ }
+ if (!block14ValueNull && !columnValueExtractor14.valueEquals(buffer, valuesOffset + value14Offset, block14, position)) {
+ return false;
+ }
+ Block block15 = page.getBlock(15);
+ boolean block15ValueNull = block15.isNull(position);
+ boolean entriesValue15IsNull = entries.isNull(entriesPosition, 15) == 1;
+ if (block15ValueNull != entriesValue15IsNull) {
+ return false;
+ }
+ if (!block15ValueNull && !columnValueExtractor15.valueEquals(buffer, valuesOffset + value15Offset, block15, position)) {
+ return false;
+ }
+ Block block16 = page.getBlock(16);
+ boolean block16ValueNull = block16.isNull(position);
+ boolean entriesValue16IsNull = entries.isNull(entriesPosition, 16) == 1;
+ if (block16ValueNull != entriesValue16IsNull) {
+ return false;
+ }
+ return block16ValueNull || columnValueExtractor16.valueEquals(buffer, valuesOffset + value16Offset, block16, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler18Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler18Channels.java
new file mode 100644
index 000000000000..70dda7be283d
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler18Channels.java
@@ -0,0 +1,397 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler18Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final ColumnValueExtractor columnValueExtractor15;
+
+ private final ColumnValueExtractor columnValueExtractor16;
+
+ private final ColumnValueExtractor columnValueExtractor17;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ private final int value15Offset;
+
+ private final int value16Offset;
+
+ private final int value17Offset;
+
+ public FixedWidthHashTableRowHandler18Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 18);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ columnValueExtractor15 = structure.getColumnValueExtractors()[15];
+ columnValueExtractor16 = structure.getColumnValueExtractors()[16];
+ columnValueExtractor17 = structure.getColumnValueExtractors()[17];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ value15Offset = structure.getValuesOffsets()[15];
+ value16Offset = structure.getValuesOffsets()[16];
+ value17Offset = structure.getValuesOffsets()[17];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ Block block10 = page.getBlock(10);
+ byte value10IsNull = (byte) (block10.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 10, value10IsNull);
+ columnValueExtractor10.putValue(buffer, valuesOffset + value10Offset, block10, position);
+ Block block11 = page.getBlock(11);
+ byte value11IsNull = (byte) (block11.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 11, value11IsNull);
+ columnValueExtractor11.putValue(buffer, valuesOffset + value11Offset, block11, position);
+ Block block12 = page.getBlock(12);
+ byte value12IsNull = (byte) (block12.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 12, value12IsNull);
+ columnValueExtractor12.putValue(buffer, valuesOffset + value12Offset, block12, position);
+ Block block13 = page.getBlock(13);
+ byte value13IsNull = (byte) (block13.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 13, value13IsNull);
+ columnValueExtractor13.putValue(buffer, valuesOffset + value13Offset, block13, position);
+ Block block14 = page.getBlock(14);
+ byte value14IsNull = (byte) (block14.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 14, value14IsNull);
+ columnValueExtractor14.putValue(buffer, valuesOffset + value14Offset, block14, position);
+ Block block15 = page.getBlock(15);
+ byte value15IsNull = (byte) (block15.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 15, value15IsNull);
+ columnValueExtractor15.putValue(buffer, valuesOffset + value15Offset, block15, position);
+ Block block16 = page.getBlock(16);
+ byte value16IsNull = (byte) (block16.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 16, value16IsNull);
+ columnValueExtractor16.putValue(buffer, valuesOffset + value16Offset, block16, position);
+ Block block17 = page.getBlock(17);
+ byte value17IsNull = (byte) (block17.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 17, value17IsNull);
+ columnValueExtractor17.putValue(buffer, valuesOffset + value17Offset, block17, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ if (!block9ValueNull && !columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position)) {
+ return false;
+ }
+ Block block10 = page.getBlock(10);
+ boolean block10ValueNull = block10.isNull(position);
+ boolean entriesValue10IsNull = entries.isNull(entriesPosition, 10) == 1;
+ if (block10ValueNull != entriesValue10IsNull) {
+ return false;
+ }
+ if (!block10ValueNull && !columnValueExtractor10.valueEquals(buffer, valuesOffset + value10Offset, block10, position)) {
+ return false;
+ }
+ Block block11 = page.getBlock(11);
+ boolean block11ValueNull = block11.isNull(position);
+ boolean entriesValue11IsNull = entries.isNull(entriesPosition, 11) == 1;
+ if (block11ValueNull != entriesValue11IsNull) {
+ return false;
+ }
+ if (!block11ValueNull && !columnValueExtractor11.valueEquals(buffer, valuesOffset + value11Offset, block11, position)) {
+ return false;
+ }
+ Block block12 = page.getBlock(12);
+ boolean block12ValueNull = block12.isNull(position);
+ boolean entriesValue12IsNull = entries.isNull(entriesPosition, 12) == 1;
+ if (block12ValueNull != entriesValue12IsNull) {
+ return false;
+ }
+ if (!block12ValueNull && !columnValueExtractor12.valueEquals(buffer, valuesOffset + value12Offset, block12, position)) {
+ return false;
+ }
+ Block block13 = page.getBlock(13);
+ boolean block13ValueNull = block13.isNull(position);
+ boolean entriesValue13IsNull = entries.isNull(entriesPosition, 13) == 1;
+ if (block13ValueNull != entriesValue13IsNull) {
+ return false;
+ }
+ if (!block13ValueNull && !columnValueExtractor13.valueEquals(buffer, valuesOffset + value13Offset, block13, position)) {
+ return false;
+ }
+ Block block14 = page.getBlock(14);
+ boolean block14ValueNull = block14.isNull(position);
+ boolean entriesValue14IsNull = entries.isNull(entriesPosition, 14) == 1;
+ if (block14ValueNull != entriesValue14IsNull) {
+ return false;
+ }
+ if (!block14ValueNull && !columnValueExtractor14.valueEquals(buffer, valuesOffset + value14Offset, block14, position)) {
+ return false;
+ }
+ Block block15 = page.getBlock(15);
+ boolean block15ValueNull = block15.isNull(position);
+ boolean entriesValue15IsNull = entries.isNull(entriesPosition, 15) == 1;
+ if (block15ValueNull != entriesValue15IsNull) {
+ return false;
+ }
+ if (!block15ValueNull && !columnValueExtractor15.valueEquals(buffer, valuesOffset + value15Offset, block15, position)) {
+ return false;
+ }
+ Block block16 = page.getBlock(16);
+ boolean block16ValueNull = block16.isNull(position);
+ boolean entriesValue16IsNull = entries.isNull(entriesPosition, 16) == 1;
+ if (block16ValueNull != entriesValue16IsNull) {
+ return false;
+ }
+ if (!block16ValueNull && !columnValueExtractor16.valueEquals(buffer, valuesOffset + value16Offset, block16, position)) {
+ return false;
+ }
+ Block block17 = page.getBlock(17);
+ boolean block17ValueNull = block17.isNull(position);
+ boolean entriesValue17IsNull = entries.isNull(entriesPosition, 17) == 1;
+ if (block17ValueNull != entriesValue17IsNull) {
+ return false;
+ }
+ return block17ValueNull || columnValueExtractor17.valueEquals(buffer, valuesOffset + value17Offset, block17, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler19Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler19Channels.java
new file mode 100644
index 000000000000..cdf0f13c4347
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler19Channels.java
@@ -0,0 +1,416 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler19Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final ColumnValueExtractor columnValueExtractor15;
+
+ private final ColumnValueExtractor columnValueExtractor16;
+
+ private final ColumnValueExtractor columnValueExtractor17;
+
+ private final ColumnValueExtractor columnValueExtractor18;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ private final int value15Offset;
+
+ private final int value16Offset;
+
+ private final int value17Offset;
+
+ private final int value18Offset;
+
+ public FixedWidthHashTableRowHandler19Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 19);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ columnValueExtractor15 = structure.getColumnValueExtractors()[15];
+ columnValueExtractor16 = structure.getColumnValueExtractors()[16];
+ columnValueExtractor17 = structure.getColumnValueExtractors()[17];
+ columnValueExtractor18 = structure.getColumnValueExtractors()[18];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ value15Offset = structure.getValuesOffsets()[15];
+ value16Offset = structure.getValuesOffsets()[16];
+ value17Offset = structure.getValuesOffsets()[17];
+ value18Offset = structure.getValuesOffsets()[18];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ Block block10 = page.getBlock(10);
+ byte value10IsNull = (byte) (block10.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 10, value10IsNull);
+ columnValueExtractor10.putValue(buffer, valuesOffset + value10Offset, block10, position);
+ Block block11 = page.getBlock(11);
+ byte value11IsNull = (byte) (block11.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 11, value11IsNull);
+ columnValueExtractor11.putValue(buffer, valuesOffset + value11Offset, block11, position);
+ Block block12 = page.getBlock(12);
+ byte value12IsNull = (byte) (block12.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 12, value12IsNull);
+ columnValueExtractor12.putValue(buffer, valuesOffset + value12Offset, block12, position);
+ Block block13 = page.getBlock(13);
+ byte value13IsNull = (byte) (block13.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 13, value13IsNull);
+ columnValueExtractor13.putValue(buffer, valuesOffset + value13Offset, block13, position);
+ Block block14 = page.getBlock(14);
+ byte value14IsNull = (byte) (block14.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 14, value14IsNull);
+ columnValueExtractor14.putValue(buffer, valuesOffset + value14Offset, block14, position);
+ Block block15 = page.getBlock(15);
+ byte value15IsNull = (byte) (block15.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 15, value15IsNull);
+ columnValueExtractor15.putValue(buffer, valuesOffset + value15Offset, block15, position);
+ Block block16 = page.getBlock(16);
+ byte value16IsNull = (byte) (block16.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 16, value16IsNull);
+ columnValueExtractor16.putValue(buffer, valuesOffset + value16Offset, block16, position);
+ Block block17 = page.getBlock(17);
+ byte value17IsNull = (byte) (block17.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 17, value17IsNull);
+ columnValueExtractor17.putValue(buffer, valuesOffset + value17Offset, block17, position);
+ Block block18 = page.getBlock(18);
+ byte value18IsNull = (byte) (block18.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 18, value18IsNull);
+ columnValueExtractor18.putValue(buffer, valuesOffset + value18Offset, block18, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ if (!block9ValueNull && !columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position)) {
+ return false;
+ }
+ Block block10 = page.getBlock(10);
+ boolean block10ValueNull = block10.isNull(position);
+ boolean entriesValue10IsNull = entries.isNull(entriesPosition, 10) == 1;
+ if (block10ValueNull != entriesValue10IsNull) {
+ return false;
+ }
+ if (!block10ValueNull && !columnValueExtractor10.valueEquals(buffer, valuesOffset + value10Offset, block10, position)) {
+ return false;
+ }
+ Block block11 = page.getBlock(11);
+ boolean block11ValueNull = block11.isNull(position);
+ boolean entriesValue11IsNull = entries.isNull(entriesPosition, 11) == 1;
+ if (block11ValueNull != entriesValue11IsNull) {
+ return false;
+ }
+ if (!block11ValueNull && !columnValueExtractor11.valueEquals(buffer, valuesOffset + value11Offset, block11, position)) {
+ return false;
+ }
+ Block block12 = page.getBlock(12);
+ boolean block12ValueNull = block12.isNull(position);
+ boolean entriesValue12IsNull = entries.isNull(entriesPosition, 12) == 1;
+ if (block12ValueNull != entriesValue12IsNull) {
+ return false;
+ }
+ if (!block12ValueNull && !columnValueExtractor12.valueEquals(buffer, valuesOffset + value12Offset, block12, position)) {
+ return false;
+ }
+ Block block13 = page.getBlock(13);
+ boolean block13ValueNull = block13.isNull(position);
+ boolean entriesValue13IsNull = entries.isNull(entriesPosition, 13) == 1;
+ if (block13ValueNull != entriesValue13IsNull) {
+ return false;
+ }
+ if (!block13ValueNull && !columnValueExtractor13.valueEquals(buffer, valuesOffset + value13Offset, block13, position)) {
+ return false;
+ }
+ Block block14 = page.getBlock(14);
+ boolean block14ValueNull = block14.isNull(position);
+ boolean entriesValue14IsNull = entries.isNull(entriesPosition, 14) == 1;
+ if (block14ValueNull != entriesValue14IsNull) {
+ return false;
+ }
+ if (!block14ValueNull && !columnValueExtractor14.valueEquals(buffer, valuesOffset + value14Offset, block14, position)) {
+ return false;
+ }
+ Block block15 = page.getBlock(15);
+ boolean block15ValueNull = block15.isNull(position);
+ boolean entriesValue15IsNull = entries.isNull(entriesPosition, 15) == 1;
+ if (block15ValueNull != entriesValue15IsNull) {
+ return false;
+ }
+ if (!block15ValueNull && !columnValueExtractor15.valueEquals(buffer, valuesOffset + value15Offset, block15, position)) {
+ return false;
+ }
+ Block block16 = page.getBlock(16);
+ boolean block16ValueNull = block16.isNull(position);
+ boolean entriesValue16IsNull = entries.isNull(entriesPosition, 16) == 1;
+ if (block16ValueNull != entriesValue16IsNull) {
+ return false;
+ }
+ if (!block16ValueNull && !columnValueExtractor16.valueEquals(buffer, valuesOffset + value16Offset, block16, position)) {
+ return false;
+ }
+ Block block17 = page.getBlock(17);
+ boolean block17ValueNull = block17.isNull(position);
+ boolean entriesValue17IsNull = entries.isNull(entriesPosition, 17) == 1;
+ if (block17ValueNull != entriesValue17IsNull) {
+ return false;
+ }
+ if (!block17ValueNull && !columnValueExtractor17.valueEquals(buffer, valuesOffset + value17Offset, block17, position)) {
+ return false;
+ }
+ Block block18 = page.getBlock(18);
+ boolean block18ValueNull = block18.isNull(position);
+ boolean entriesValue18IsNull = entries.isNull(entriesPosition, 18) == 1;
+ if (block18ValueNull != entriesValue18IsNull) {
+ return false;
+ }
+ return block18ValueNull || columnValueExtractor18.valueEquals(buffer, valuesOffset + value18Offset, block18, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler1Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler1Channels.java
new file mode 100644
index 000000000000..6a71d415814e
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler1Channels.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler1Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ public FixedWidthHashTableRowHandler1Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 1);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ return block0ValueNull || columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler20Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler20Channels.java
new file mode 100644
index 000000000000..729afbd73568
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler20Channels.java
@@ -0,0 +1,435 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler20Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final ColumnValueExtractor columnValueExtractor15;
+
+ private final ColumnValueExtractor columnValueExtractor16;
+
+ private final ColumnValueExtractor columnValueExtractor17;
+
+ private final ColumnValueExtractor columnValueExtractor18;
+
+ private final ColumnValueExtractor columnValueExtractor19;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ private final int value15Offset;
+
+ private final int value16Offset;
+
+ private final int value17Offset;
+
+ private final int value18Offset;
+
+ private final int value19Offset;
+
+ public FixedWidthHashTableRowHandler20Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 20);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ columnValueExtractor15 = structure.getColumnValueExtractors()[15];
+ columnValueExtractor16 = structure.getColumnValueExtractors()[16];
+ columnValueExtractor17 = structure.getColumnValueExtractors()[17];
+ columnValueExtractor18 = structure.getColumnValueExtractors()[18];
+ columnValueExtractor19 = structure.getColumnValueExtractors()[19];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ value15Offset = structure.getValuesOffsets()[15];
+ value16Offset = structure.getValuesOffsets()[16];
+ value17Offset = structure.getValuesOffsets()[17];
+ value18Offset = structure.getValuesOffsets()[18];
+ value19Offset = structure.getValuesOffsets()[19];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ Block block9 = page.getBlock(9);
+ byte value9IsNull = (byte) (block9.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 9, value9IsNull);
+ columnValueExtractor9.putValue(buffer, valuesOffset + value9Offset, block9, position);
+ Block block10 = page.getBlock(10);
+ byte value10IsNull = (byte) (block10.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 10, value10IsNull);
+ columnValueExtractor10.putValue(buffer, valuesOffset + value10Offset, block10, position);
+ Block block11 = page.getBlock(11);
+ byte value11IsNull = (byte) (block11.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 11, value11IsNull);
+ columnValueExtractor11.putValue(buffer, valuesOffset + value11Offset, block11, position);
+ Block block12 = page.getBlock(12);
+ byte value12IsNull = (byte) (block12.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 12, value12IsNull);
+ columnValueExtractor12.putValue(buffer, valuesOffset + value12Offset, block12, position);
+ Block block13 = page.getBlock(13);
+ byte value13IsNull = (byte) (block13.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 13, value13IsNull);
+ columnValueExtractor13.putValue(buffer, valuesOffset + value13Offset, block13, position);
+ Block block14 = page.getBlock(14);
+ byte value14IsNull = (byte) (block14.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 14, value14IsNull);
+ columnValueExtractor14.putValue(buffer, valuesOffset + value14Offset, block14, position);
+ Block block15 = page.getBlock(15);
+ byte value15IsNull = (byte) (block15.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 15, value15IsNull);
+ columnValueExtractor15.putValue(buffer, valuesOffset + value15Offset, block15, position);
+ Block block16 = page.getBlock(16);
+ byte value16IsNull = (byte) (block16.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 16, value16IsNull);
+ columnValueExtractor16.putValue(buffer, valuesOffset + value16Offset, block16, position);
+ Block block17 = page.getBlock(17);
+ byte value17IsNull = (byte) (block17.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 17, value17IsNull);
+ columnValueExtractor17.putValue(buffer, valuesOffset + value17Offset, block17, position);
+ Block block18 = page.getBlock(18);
+ byte value18IsNull = (byte) (block18.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 18, value18IsNull);
+ columnValueExtractor18.putValue(buffer, valuesOffset + value18Offset, block18, position);
+ Block block19 = page.getBlock(19);
+ byte value19IsNull = (byte) (block19.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 19, value19IsNull);
+ columnValueExtractor19.putValue(buffer, valuesOffset + value19Offset, block19, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ if (!block8ValueNull && !columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position)) {
+ return false;
+ }
+ Block block9 = page.getBlock(9);
+ boolean block9ValueNull = block9.isNull(position);
+ boolean entriesValue9IsNull = entries.isNull(entriesPosition, 9) == 1;
+ if (block9ValueNull != entriesValue9IsNull) {
+ return false;
+ }
+ if (!block9ValueNull && !columnValueExtractor9.valueEquals(buffer, valuesOffset + value9Offset, block9, position)) {
+ return false;
+ }
+ Block block10 = page.getBlock(10);
+ boolean block10ValueNull = block10.isNull(position);
+ boolean entriesValue10IsNull = entries.isNull(entriesPosition, 10) == 1;
+ if (block10ValueNull != entriesValue10IsNull) {
+ return false;
+ }
+ if (!block10ValueNull && !columnValueExtractor10.valueEquals(buffer, valuesOffset + value10Offset, block10, position)) {
+ return false;
+ }
+ Block block11 = page.getBlock(11);
+ boolean block11ValueNull = block11.isNull(position);
+ boolean entriesValue11IsNull = entries.isNull(entriesPosition, 11) == 1;
+ if (block11ValueNull != entriesValue11IsNull) {
+ return false;
+ }
+ if (!block11ValueNull && !columnValueExtractor11.valueEquals(buffer, valuesOffset + value11Offset, block11, position)) {
+ return false;
+ }
+ Block block12 = page.getBlock(12);
+ boolean block12ValueNull = block12.isNull(position);
+ boolean entriesValue12IsNull = entries.isNull(entriesPosition, 12) == 1;
+ if (block12ValueNull != entriesValue12IsNull) {
+ return false;
+ }
+ if (!block12ValueNull && !columnValueExtractor12.valueEquals(buffer, valuesOffset + value12Offset, block12, position)) {
+ return false;
+ }
+ Block block13 = page.getBlock(13);
+ boolean block13ValueNull = block13.isNull(position);
+ boolean entriesValue13IsNull = entries.isNull(entriesPosition, 13) == 1;
+ if (block13ValueNull != entriesValue13IsNull) {
+ return false;
+ }
+ if (!block13ValueNull && !columnValueExtractor13.valueEquals(buffer, valuesOffset + value13Offset, block13, position)) {
+ return false;
+ }
+ Block block14 = page.getBlock(14);
+ boolean block14ValueNull = block14.isNull(position);
+ boolean entriesValue14IsNull = entries.isNull(entriesPosition, 14) == 1;
+ if (block14ValueNull != entriesValue14IsNull) {
+ return false;
+ }
+ if (!block14ValueNull && !columnValueExtractor14.valueEquals(buffer, valuesOffset + value14Offset, block14, position)) {
+ return false;
+ }
+ Block block15 = page.getBlock(15);
+ boolean block15ValueNull = block15.isNull(position);
+ boolean entriesValue15IsNull = entries.isNull(entriesPosition, 15) == 1;
+ if (block15ValueNull != entriesValue15IsNull) {
+ return false;
+ }
+ if (!block15ValueNull && !columnValueExtractor15.valueEquals(buffer, valuesOffset + value15Offset, block15, position)) {
+ return false;
+ }
+ Block block16 = page.getBlock(16);
+ boolean block16ValueNull = block16.isNull(position);
+ boolean entriesValue16IsNull = entries.isNull(entriesPosition, 16) == 1;
+ if (block16ValueNull != entriesValue16IsNull) {
+ return false;
+ }
+ if (!block16ValueNull && !columnValueExtractor16.valueEquals(buffer, valuesOffset + value16Offset, block16, position)) {
+ return false;
+ }
+ Block block17 = page.getBlock(17);
+ boolean block17ValueNull = block17.isNull(position);
+ boolean entriesValue17IsNull = entries.isNull(entriesPosition, 17) == 1;
+ if (block17ValueNull != entriesValue17IsNull) {
+ return false;
+ }
+ if (!block17ValueNull && !columnValueExtractor17.valueEquals(buffer, valuesOffset + value17Offset, block17, position)) {
+ return false;
+ }
+ Block block18 = page.getBlock(18);
+ boolean block18ValueNull = block18.isNull(position);
+ boolean entriesValue18IsNull = entries.isNull(entriesPosition, 18) == 1;
+ if (block18ValueNull != entriesValue18IsNull) {
+ return false;
+ }
+ if (!block18ValueNull && !columnValueExtractor18.valueEquals(buffer, valuesOffset + value18Offset, block18, position)) {
+ return false;
+ }
+ Block block19 = page.getBlock(19);
+ boolean block19ValueNull = block19.isNull(position);
+ boolean entriesValue19IsNull = entries.isNull(entriesPosition, 19) == 1;
+ if (block19ValueNull != entriesValue19IsNull) {
+ return false;
+ }
+ return block19ValueNull || columnValueExtractor19.valueEquals(buffer, valuesOffset + value19Offset, block19, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler2Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler2Channels.java
new file mode 100644
index 000000000000..d76b3a196a66
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler2Channels.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler2Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final int value1Offset;
+
+ public FixedWidthHashTableRowHandler2Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 2);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ value1Offset = structure.getValuesOffsets()[1];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ return block1ValueNull || columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler3Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler3Channels.java
new file mode 100644
index 000000000000..758d2a881e96
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler3Channels.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler3Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ public FixedWidthHashTableRowHandler3Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 3);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ return block2ValueNull || columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler4Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler4Channels.java
new file mode 100644
index 000000000000..bed6d0c9f2fe
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler4Channels.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler4Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ public FixedWidthHashTableRowHandler4Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 4);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ return block3ValueNull || columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler5Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler5Channels.java
new file mode 100644
index 000000000000..8118b0e7de19
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler5Channels.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler5Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ public FixedWidthHashTableRowHandler5Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 5);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ return block4ValueNull || columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler6Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler6Channels.java
new file mode 100644
index 000000000000..4d73d4e82e8c
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler6Channels.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler6Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ public FixedWidthHashTableRowHandler6Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 6);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ return block5ValueNull || columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler7Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler7Channels.java
new file mode 100644
index 000000000000..bd41e360044a
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler7Channels.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler7Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ public FixedWidthHashTableRowHandler7Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 7);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ return block6ValueNull || columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler8Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler8Channels.java
new file mode 100644
index 000000000000..210ac5d2d3c6
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler8Channels.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler8Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ public FixedWidthHashTableRowHandler8Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 8);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ return block7ValueNull || columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler9Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler9Channels.java
new file mode 100644
index 000000000000..3d2a1dce96c1
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableRowHandler9Channels.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.GroupByHashTableEntries;
+import io.trino.operator.hash.HashTableRowHandler;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableRowHandlerGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableRowHandler9Channels
+ implements HashTableRowHandler
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ public FixedWidthHashTableRowHandler9Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 9);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ }
+
+ @Override
+ public void putEntry(GroupByHashTableEntries data, int entriesPosition, int groupId, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ entries.putGroupId(entriesPosition, groupId);
+ entries.putHash(entriesPosition, rawHash);
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ byte value0IsNull = (byte) (block0.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 0, value0IsNull);
+ columnValueExtractor0.putValue(buffer, valuesOffset, block0, position);
+ Block block1 = page.getBlock(1);
+ byte value1IsNull = (byte) (block1.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 1, value1IsNull);
+ columnValueExtractor1.putValue(buffer, valuesOffset + value1Offset, block1, position);
+ Block block2 = page.getBlock(2);
+ byte value2IsNull = (byte) (block2.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 2, value2IsNull);
+ columnValueExtractor2.putValue(buffer, valuesOffset + value2Offset, block2, position);
+ Block block3 = page.getBlock(3);
+ byte value3IsNull = (byte) (block3.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 3, value3IsNull);
+ columnValueExtractor3.putValue(buffer, valuesOffset + value3Offset, block3, position);
+ Block block4 = page.getBlock(4);
+ byte value4IsNull = (byte) (block4.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 4, value4IsNull);
+ columnValueExtractor4.putValue(buffer, valuesOffset + value4Offset, block4, position);
+ Block block5 = page.getBlock(5);
+ byte value5IsNull = (byte) (block5.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 5, value5IsNull);
+ columnValueExtractor5.putValue(buffer, valuesOffset + value5Offset, block5, position);
+ Block block6 = page.getBlock(6);
+ byte value6IsNull = (byte) (block6.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 6, value6IsNull);
+ columnValueExtractor6.putValue(buffer, valuesOffset + value6Offset, block6, position);
+ Block block7 = page.getBlock(7);
+ byte value7IsNull = (byte) (block7.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 7, value7IsNull);
+ columnValueExtractor7.putValue(buffer, valuesOffset + value7Offset, block7, position);
+ Block block8 = page.getBlock(8);
+ byte value8IsNull = (byte) (block8.isNull(position) ? 1 : 0);
+ entries.putIsNull(entriesPosition, 8, value8IsNull);
+ columnValueExtractor8.putValue(buffer, valuesOffset + value8Offset, block8, position);
+ }
+
+ @Override
+ public boolean keyEquals(GroupByHashTableEntries data, int entriesPosition, Page page,
+ int position, long rawHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data;
+ int valuesOffset = entries.getValuesOffset(entriesPosition);
+ FastByteBuffer buffer = entries.getBuffer();
+ Block block0 = page.getBlock(0);
+ boolean block0ValueNull = block0.isNull(position);
+ boolean entriesValue0IsNull = entries.isNull(entriesPosition, 0) == 1;
+ if (block0ValueNull != entriesValue0IsNull) {
+ return false;
+ }
+ if (!block0ValueNull && !columnValueExtractor0.valueEquals(buffer, valuesOffset, block0, position)) {
+ return false;
+ }
+ Block block1 = page.getBlock(1);
+ boolean block1ValueNull = block1.isNull(position);
+ boolean entriesValue1IsNull = entries.isNull(entriesPosition, 1) == 1;
+ if (block1ValueNull != entriesValue1IsNull) {
+ return false;
+ }
+ if (!block1ValueNull && !columnValueExtractor1.valueEquals(buffer, valuesOffset + value1Offset, block1, position)) {
+ return false;
+ }
+ Block block2 = page.getBlock(2);
+ boolean block2ValueNull = block2.isNull(position);
+ boolean entriesValue2IsNull = entries.isNull(entriesPosition, 2) == 1;
+ if (block2ValueNull != entriesValue2IsNull) {
+ return false;
+ }
+ if (!block2ValueNull && !columnValueExtractor2.valueEquals(buffer, valuesOffset + value2Offset, block2, position)) {
+ return false;
+ }
+ Block block3 = page.getBlock(3);
+ boolean block3ValueNull = block3.isNull(position);
+ boolean entriesValue3IsNull = entries.isNull(entriesPosition, 3) == 1;
+ if (block3ValueNull != entriesValue3IsNull) {
+ return false;
+ }
+ if (!block3ValueNull && !columnValueExtractor3.valueEquals(buffer, valuesOffset + value3Offset, block3, position)) {
+ return false;
+ }
+ Block block4 = page.getBlock(4);
+ boolean block4ValueNull = block4.isNull(position);
+ boolean entriesValue4IsNull = entries.isNull(entriesPosition, 4) == 1;
+ if (block4ValueNull != entriesValue4IsNull) {
+ return false;
+ }
+ if (!block4ValueNull && !columnValueExtractor4.valueEquals(buffer, valuesOffset + value4Offset, block4, position)) {
+ return false;
+ }
+ Block block5 = page.getBlock(5);
+ boolean block5ValueNull = block5.isNull(position);
+ boolean entriesValue5IsNull = entries.isNull(entriesPosition, 5) == 1;
+ if (block5ValueNull != entriesValue5IsNull) {
+ return false;
+ }
+ if (!block5ValueNull && !columnValueExtractor5.valueEquals(buffer, valuesOffset + value5Offset, block5, position)) {
+ return false;
+ }
+ Block block6 = page.getBlock(6);
+ boolean block6ValueNull = block6.isNull(position);
+ boolean entriesValue6IsNull = entries.isNull(entriesPosition, 6) == 1;
+ if (block6ValueNull != entriesValue6IsNull) {
+ return false;
+ }
+ if (!block6ValueNull && !columnValueExtractor6.valueEquals(buffer, valuesOffset + value6Offset, block6, position)) {
+ return false;
+ }
+ Block block7 = page.getBlock(7);
+ boolean block7ValueNull = block7.isNull(position);
+ boolean entriesValue7IsNull = entries.isNull(entriesPosition, 7) == 1;
+ if (block7ValueNull != entriesValue7IsNull) {
+ return false;
+ }
+ if (!block7ValueNull && !columnValueExtractor7.valueEquals(buffer, valuesOffset + value7Offset, block7, position)) {
+ return false;
+ }
+ Block block8 = page.getBlock(8);
+ boolean block8ValueNull = block8.isNull(position);
+ boolean entriesValue8IsNull = entries.isNull(entriesPosition, 8) == 1;
+ if (block8ValueNull != entriesValue8IsNull) {
+ return false;
+ }
+ return block8ValueNull || columnValueExtractor8.valueEquals(buffer, valuesOffset + value8Offset, block8, position);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender10Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender10Channels.java
new file mode 100644
index 000000000000..75ede6d6c9c5
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender10Channels.java
@@ -0,0 +1,463 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender10Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ public FixedWidthHashTableValuesAppender10Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 10);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender11Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender11Channels.java
new file mode 100644
index 000000000000..96de88572145
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender11Channels.java
@@ -0,0 +1,502 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender11Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ public FixedWidthHashTableValuesAppender11Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 11);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 10) == 1;
+ }
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffsetBatch[i] + value10Offset, blockBuilder10);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ if (entries.isNull(position, 10) == 1) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffset + value10Offset, blockBuilder10);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender12Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender12Channels.java
new file mode 100644
index 000000000000..b5d701e5e3cb
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender12Channels.java
@@ -0,0 +1,541 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender12Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ public FixedWidthHashTableValuesAppender12Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 12);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 10) == 1;
+ }
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffsetBatch[i] + value10Offset, blockBuilder10);
+ }
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 11) == 1;
+ }
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffsetBatch[i] + value11Offset, blockBuilder11);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ if (entries.isNull(position, 10) == 1) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffset + value10Offset, blockBuilder10);
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ if (entries.isNull(position, 11) == 1) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffset + value11Offset, blockBuilder11);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender13Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender13Channels.java
new file mode 100644
index 000000000000..1386c75e3e84
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender13Channels.java
@@ -0,0 +1,580 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender13Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ public FixedWidthHashTableValuesAppender13Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 13);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 10) == 1;
+ }
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffsetBatch[i] + value10Offset, blockBuilder10);
+ }
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 11) == 1;
+ }
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffsetBatch[i] + value11Offset, blockBuilder11);
+ }
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 12) == 1;
+ }
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffsetBatch[i] + value12Offset, blockBuilder12);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ if (entries.isNull(position, 10) == 1) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffset + value10Offset, blockBuilder10);
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ if (entries.isNull(position, 11) == 1) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffset + value11Offset, blockBuilder11);
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ if (entries.isNull(position, 12) == 1) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffset + value12Offset, blockBuilder12);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender14Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender14Channels.java
new file mode 100644
index 000000000000..e8d0d300b205
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender14Channels.java
@@ -0,0 +1,619 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender14Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ public FixedWidthHashTableValuesAppender14Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 14);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 10) == 1;
+ }
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffsetBatch[i] + value10Offset, blockBuilder10);
+ }
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 11) == 1;
+ }
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffsetBatch[i] + value11Offset, blockBuilder11);
+ }
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 12) == 1;
+ }
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffsetBatch[i] + value12Offset, blockBuilder12);
+ }
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 13) == 1;
+ }
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffsetBatch[i] + value13Offset, blockBuilder13);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ if (entries.isNull(position, 10) == 1) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffset + value10Offset, blockBuilder10);
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ if (entries.isNull(position, 11) == 1) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffset + value11Offset, blockBuilder11);
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ if (entries.isNull(position, 12) == 1) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffset + value12Offset, blockBuilder12);
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ if (entries.isNull(position, 13) == 1) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffset + value13Offset, blockBuilder13);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender15Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender15Channels.java
new file mode 100644
index 000000000000..b4befd5fc6ad
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender15Channels.java
@@ -0,0 +1,658 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender15Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ public FixedWidthHashTableValuesAppender15Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 15);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 10) == 1;
+ }
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffsetBatch[i] + value10Offset, blockBuilder10);
+ }
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 11) == 1;
+ }
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffsetBatch[i] + value11Offset, blockBuilder11);
+ }
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 12) == 1;
+ }
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffsetBatch[i] + value12Offset, blockBuilder12);
+ }
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 13) == 1;
+ }
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffsetBatch[i] + value13Offset, blockBuilder13);
+ }
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 14) == 1;
+ }
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffsetBatch[i] + value14Offset, blockBuilder14);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ if (entries.isNull(position, 10) == 1) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffset + value10Offset, blockBuilder10);
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ if (entries.isNull(position, 11) == 1) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffset + value11Offset, blockBuilder11);
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ if (entries.isNull(position, 12) == 1) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffset + value12Offset, blockBuilder12);
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ if (entries.isNull(position, 13) == 1) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffset + value13Offset, blockBuilder13);
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ if (entries.isNull(position, 14) == 1) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffset + value14Offset, blockBuilder14);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender16Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender16Channels.java
new file mode 100644
index 000000000000..41befa22b3c2
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender16Channels.java
@@ -0,0 +1,697 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender16Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final ColumnValueExtractor columnValueExtractor15;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ private final int value15Offset;
+
+ public FixedWidthHashTableValuesAppender16Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 16);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ columnValueExtractor15 = structure.getColumnValueExtractors()[15];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ value15Offset = structure.getValuesOffsets()[15];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue15(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 16);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 10) == 1;
+ }
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffsetBatch[i] + value10Offset, blockBuilder10);
+ }
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 11) == 1;
+ }
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffsetBatch[i] + value11Offset, blockBuilder11);
+ }
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 12) == 1;
+ }
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffsetBatch[i] + value12Offset, blockBuilder12);
+ }
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 13) == 1;
+ }
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffsetBatch[i] + value13Offset, blockBuilder13);
+ }
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 14) == 1;
+ }
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffsetBatch[i] + value14Offset, blockBuilder14);
+ }
+ }
+ }
+
+ private void appendValue15(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 15) == 1;
+ }
+ BlockBuilder blockBuilder15 = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder15.appendNull();
+ }
+ else {
+ columnValueExtractor15.appendValue(buffer, valuesOffsetBatch[i] + value15Offset, blockBuilder15);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue15(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 16);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ if (entries.isNull(position, 10) == 1) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffset + value10Offset, blockBuilder10);
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ if (entries.isNull(position, 11) == 1) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffset + value11Offset, blockBuilder11);
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ if (entries.isNull(position, 12) == 1) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffset + value12Offset, blockBuilder12);
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ if (entries.isNull(position, 13) == 1) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffset + value13Offset, blockBuilder13);
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ if (entries.isNull(position, 14) == 1) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffset + value14Offset, blockBuilder14);
+ }
+ }
+
+ private void appendValue15(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder15 = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ if (entries.isNull(position, 15) == 1) {
+ blockBuilder15.appendNull();
+ }
+ else {
+ columnValueExtractor15.appendValue(buffer, valuesOffset + value15Offset, blockBuilder15);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender17Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender17Channels.java
new file mode 100644
index 000000000000..392ef740e9f0
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender17Channels.java
@@ -0,0 +1,736 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender17Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final ColumnValueExtractor columnValueExtractor15;
+
+ private final ColumnValueExtractor columnValueExtractor16;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ private final int value15Offset;
+
+ private final int value16Offset;
+
+ public FixedWidthHashTableValuesAppender17Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 17);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ columnValueExtractor15 = structure.getColumnValueExtractors()[15];
+ columnValueExtractor16 = structure.getColumnValueExtractors()[16];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ value15Offset = structure.getValuesOffsets()[15];
+ value16Offset = structure.getValuesOffsets()[16];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue15(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue16(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 17);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 10) == 1;
+ }
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffsetBatch[i] + value10Offset, blockBuilder10);
+ }
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 11) == 1;
+ }
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffsetBatch[i] + value11Offset, blockBuilder11);
+ }
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 12) == 1;
+ }
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffsetBatch[i] + value12Offset, blockBuilder12);
+ }
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 13) == 1;
+ }
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffsetBatch[i] + value13Offset, blockBuilder13);
+ }
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 14) == 1;
+ }
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffsetBatch[i] + value14Offset, blockBuilder14);
+ }
+ }
+ }
+
+ private void appendValue15(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 15) == 1;
+ }
+ BlockBuilder blockBuilder15 = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder15.appendNull();
+ }
+ else {
+ columnValueExtractor15.appendValue(buffer, valuesOffsetBatch[i] + value15Offset, blockBuilder15);
+ }
+ }
+ }
+
+ private void appendValue16(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 16) == 1;
+ }
+ BlockBuilder blockBuilder16 = pageBuilder.getBlockBuilder(outputChannelOffset + 16);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder16.appendNull();
+ }
+ else {
+ columnValueExtractor16.appendValue(buffer, valuesOffsetBatch[i] + value16Offset, blockBuilder16);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue15(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue16(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 17);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ if (entries.isNull(position, 10) == 1) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffset + value10Offset, blockBuilder10);
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ if (entries.isNull(position, 11) == 1) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffset + value11Offset, blockBuilder11);
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ if (entries.isNull(position, 12) == 1) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffset + value12Offset, blockBuilder12);
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ if (entries.isNull(position, 13) == 1) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffset + value13Offset, blockBuilder13);
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ if (entries.isNull(position, 14) == 1) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffset + value14Offset, blockBuilder14);
+ }
+ }
+
+ private void appendValue15(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder15 = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ if (entries.isNull(position, 15) == 1) {
+ blockBuilder15.appendNull();
+ }
+ else {
+ columnValueExtractor15.appendValue(buffer, valuesOffset + value15Offset, blockBuilder15);
+ }
+ }
+
+ private void appendValue16(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder16 = pageBuilder.getBlockBuilder(outputChannelOffset + 16);
+ if (entries.isNull(position, 16) == 1) {
+ blockBuilder16.appendNull();
+ }
+ else {
+ columnValueExtractor16.appendValue(buffer, valuesOffset + value16Offset, blockBuilder16);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender18Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender18Channels.java
new file mode 100644
index 000000000000..b9097dcc0c62
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender18Channels.java
@@ -0,0 +1,775 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender18Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final ColumnValueExtractor columnValueExtractor15;
+
+ private final ColumnValueExtractor columnValueExtractor16;
+
+ private final ColumnValueExtractor columnValueExtractor17;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ private final int value15Offset;
+
+ private final int value16Offset;
+
+ private final int value17Offset;
+
+ public FixedWidthHashTableValuesAppender18Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 18);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ columnValueExtractor15 = structure.getColumnValueExtractors()[15];
+ columnValueExtractor16 = structure.getColumnValueExtractors()[16];
+ columnValueExtractor17 = structure.getColumnValueExtractors()[17];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ value15Offset = structure.getValuesOffsets()[15];
+ value16Offset = structure.getValuesOffsets()[16];
+ value17Offset = structure.getValuesOffsets()[17];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue15(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue16(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue17(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 18);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 10) == 1;
+ }
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffsetBatch[i] + value10Offset, blockBuilder10);
+ }
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 11) == 1;
+ }
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffsetBatch[i] + value11Offset, blockBuilder11);
+ }
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 12) == 1;
+ }
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffsetBatch[i] + value12Offset, blockBuilder12);
+ }
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 13) == 1;
+ }
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffsetBatch[i] + value13Offset, blockBuilder13);
+ }
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 14) == 1;
+ }
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffsetBatch[i] + value14Offset, blockBuilder14);
+ }
+ }
+ }
+
+ private void appendValue15(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 15) == 1;
+ }
+ BlockBuilder blockBuilder15 = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder15.appendNull();
+ }
+ else {
+ columnValueExtractor15.appendValue(buffer, valuesOffsetBatch[i] + value15Offset, blockBuilder15);
+ }
+ }
+ }
+
+ private void appendValue16(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 16) == 1;
+ }
+ BlockBuilder blockBuilder16 = pageBuilder.getBlockBuilder(outputChannelOffset + 16);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder16.appendNull();
+ }
+ else {
+ columnValueExtractor16.appendValue(buffer, valuesOffsetBatch[i] + value16Offset, blockBuilder16);
+ }
+ }
+ }
+
+ private void appendValue17(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 17) == 1;
+ }
+ BlockBuilder blockBuilder17 = pageBuilder.getBlockBuilder(outputChannelOffset + 17);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder17.appendNull();
+ }
+ else {
+ columnValueExtractor17.appendValue(buffer, valuesOffsetBatch[i] + value17Offset, blockBuilder17);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue15(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue16(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue17(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 18);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ if (entries.isNull(position, 10) == 1) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffset + value10Offset, blockBuilder10);
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ if (entries.isNull(position, 11) == 1) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffset + value11Offset, blockBuilder11);
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ if (entries.isNull(position, 12) == 1) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffset + value12Offset, blockBuilder12);
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ if (entries.isNull(position, 13) == 1) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffset + value13Offset, blockBuilder13);
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ if (entries.isNull(position, 14) == 1) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffset + value14Offset, blockBuilder14);
+ }
+ }
+
+ private void appendValue15(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder15 = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ if (entries.isNull(position, 15) == 1) {
+ blockBuilder15.appendNull();
+ }
+ else {
+ columnValueExtractor15.appendValue(buffer, valuesOffset + value15Offset, blockBuilder15);
+ }
+ }
+
+ private void appendValue16(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder16 = pageBuilder.getBlockBuilder(outputChannelOffset + 16);
+ if (entries.isNull(position, 16) == 1) {
+ blockBuilder16.appendNull();
+ }
+ else {
+ columnValueExtractor16.appendValue(buffer, valuesOffset + value16Offset, blockBuilder16);
+ }
+ }
+
+ private void appendValue17(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder17 = pageBuilder.getBlockBuilder(outputChannelOffset + 17);
+ if (entries.isNull(position, 17) == 1) {
+ blockBuilder17.appendNull();
+ }
+ else {
+ columnValueExtractor17.appendValue(buffer, valuesOffset + value17Offset, blockBuilder17);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender19Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender19Channels.java
new file mode 100644
index 000000000000..5d7cd48fee7d
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender19Channels.java
@@ -0,0 +1,814 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender19Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final ColumnValueExtractor columnValueExtractor15;
+
+ private final ColumnValueExtractor columnValueExtractor16;
+
+ private final ColumnValueExtractor columnValueExtractor17;
+
+ private final ColumnValueExtractor columnValueExtractor18;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ private final int value15Offset;
+
+ private final int value16Offset;
+
+ private final int value17Offset;
+
+ private final int value18Offset;
+
+ public FixedWidthHashTableValuesAppender19Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 19);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ columnValueExtractor15 = structure.getColumnValueExtractors()[15];
+ columnValueExtractor16 = structure.getColumnValueExtractors()[16];
+ columnValueExtractor17 = structure.getColumnValueExtractors()[17];
+ columnValueExtractor18 = structure.getColumnValueExtractors()[18];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ value15Offset = structure.getValuesOffsets()[15];
+ value16Offset = structure.getValuesOffsets()[16];
+ value17Offset = structure.getValuesOffsets()[17];
+ value18Offset = structure.getValuesOffsets()[18];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue15(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue16(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue17(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue18(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 19);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 10) == 1;
+ }
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffsetBatch[i] + value10Offset, blockBuilder10);
+ }
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 11) == 1;
+ }
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffsetBatch[i] + value11Offset, blockBuilder11);
+ }
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 12) == 1;
+ }
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffsetBatch[i] + value12Offset, blockBuilder12);
+ }
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 13) == 1;
+ }
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffsetBatch[i] + value13Offset, blockBuilder13);
+ }
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 14) == 1;
+ }
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffsetBatch[i] + value14Offset, blockBuilder14);
+ }
+ }
+ }
+
+ private void appendValue15(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 15) == 1;
+ }
+ BlockBuilder blockBuilder15 = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder15.appendNull();
+ }
+ else {
+ columnValueExtractor15.appendValue(buffer, valuesOffsetBatch[i] + value15Offset, blockBuilder15);
+ }
+ }
+ }
+
+ private void appendValue16(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 16) == 1;
+ }
+ BlockBuilder blockBuilder16 = pageBuilder.getBlockBuilder(outputChannelOffset + 16);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder16.appendNull();
+ }
+ else {
+ columnValueExtractor16.appendValue(buffer, valuesOffsetBatch[i] + value16Offset, blockBuilder16);
+ }
+ }
+ }
+
+ private void appendValue17(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 17) == 1;
+ }
+ BlockBuilder blockBuilder17 = pageBuilder.getBlockBuilder(outputChannelOffset + 17);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder17.appendNull();
+ }
+ else {
+ columnValueExtractor17.appendValue(buffer, valuesOffsetBatch[i] + value17Offset, blockBuilder17);
+ }
+ }
+ }
+
+ private void appendValue18(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 18) == 1;
+ }
+ BlockBuilder blockBuilder18 = pageBuilder.getBlockBuilder(outputChannelOffset + 18);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder18.appendNull();
+ }
+ else {
+ columnValueExtractor18.appendValue(buffer, valuesOffsetBatch[i] + value18Offset, blockBuilder18);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue15(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue16(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue17(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue18(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 19);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ if (entries.isNull(position, 10) == 1) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffset + value10Offset, blockBuilder10);
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ if (entries.isNull(position, 11) == 1) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffset + value11Offset, blockBuilder11);
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ if (entries.isNull(position, 12) == 1) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffset + value12Offset, blockBuilder12);
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ if (entries.isNull(position, 13) == 1) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffset + value13Offset, blockBuilder13);
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ if (entries.isNull(position, 14) == 1) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffset + value14Offset, blockBuilder14);
+ }
+ }
+
+ private void appendValue15(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder15 = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ if (entries.isNull(position, 15) == 1) {
+ blockBuilder15.appendNull();
+ }
+ else {
+ columnValueExtractor15.appendValue(buffer, valuesOffset + value15Offset, blockBuilder15);
+ }
+ }
+
+ private void appendValue16(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder16 = pageBuilder.getBlockBuilder(outputChannelOffset + 16);
+ if (entries.isNull(position, 16) == 1) {
+ blockBuilder16.appendNull();
+ }
+ else {
+ columnValueExtractor16.appendValue(buffer, valuesOffset + value16Offset, blockBuilder16);
+ }
+ }
+
+ private void appendValue17(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder17 = pageBuilder.getBlockBuilder(outputChannelOffset + 17);
+ if (entries.isNull(position, 17) == 1) {
+ blockBuilder17.appendNull();
+ }
+ else {
+ columnValueExtractor17.appendValue(buffer, valuesOffset + value17Offset, blockBuilder17);
+ }
+ }
+
+ private void appendValue18(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder18 = pageBuilder.getBlockBuilder(outputChannelOffset + 18);
+ if (entries.isNull(position, 18) == 1) {
+ blockBuilder18.appendNull();
+ }
+ else {
+ columnValueExtractor18.appendValue(buffer, valuesOffset + value18Offset, blockBuilder18);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender1Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender1Channels.java
new file mode 100644
index 000000000000..1b6b68d83455
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender1Channels.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender1Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ public FixedWidthHashTableValuesAppender1Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 1);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender20Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender20Channels.java
new file mode 100644
index 000000000000..5eee4abe9555
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender20Channels.java
@@ -0,0 +1,853 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender20Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final ColumnValueExtractor columnValueExtractor9;
+
+ private final ColumnValueExtractor columnValueExtractor10;
+
+ private final ColumnValueExtractor columnValueExtractor11;
+
+ private final ColumnValueExtractor columnValueExtractor12;
+
+ private final ColumnValueExtractor columnValueExtractor13;
+
+ private final ColumnValueExtractor columnValueExtractor14;
+
+ private final ColumnValueExtractor columnValueExtractor15;
+
+ private final ColumnValueExtractor columnValueExtractor16;
+
+ private final ColumnValueExtractor columnValueExtractor17;
+
+ private final ColumnValueExtractor columnValueExtractor18;
+
+ private final ColumnValueExtractor columnValueExtractor19;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ private final int value9Offset;
+
+ private final int value10Offset;
+
+ private final int value11Offset;
+
+ private final int value12Offset;
+
+ private final int value13Offset;
+
+ private final int value14Offset;
+
+ private final int value15Offset;
+
+ private final int value16Offset;
+
+ private final int value17Offset;
+
+ private final int value18Offset;
+
+ private final int value19Offset;
+
+ public FixedWidthHashTableValuesAppender20Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 20);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ columnValueExtractor9 = structure.getColumnValueExtractors()[9];
+ columnValueExtractor10 = structure.getColumnValueExtractors()[10];
+ columnValueExtractor11 = structure.getColumnValueExtractors()[11];
+ columnValueExtractor12 = structure.getColumnValueExtractors()[12];
+ columnValueExtractor13 = structure.getColumnValueExtractors()[13];
+ columnValueExtractor14 = structure.getColumnValueExtractors()[14];
+ columnValueExtractor15 = structure.getColumnValueExtractors()[15];
+ columnValueExtractor16 = structure.getColumnValueExtractors()[16];
+ columnValueExtractor17 = structure.getColumnValueExtractors()[17];
+ columnValueExtractor18 = structure.getColumnValueExtractors()[18];
+ columnValueExtractor19 = structure.getColumnValueExtractors()[19];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ value9Offset = structure.getValuesOffsets()[9];
+ value10Offset = structure.getValuesOffsets()[10];
+ value11Offset = structure.getValuesOffsets()[11];
+ value12Offset = structure.getValuesOffsets()[12];
+ value13Offset = structure.getValuesOffsets()[13];
+ value14Offset = structure.getValuesOffsets()[14];
+ value15Offset = structure.getValuesOffsets()[15];
+ value16Offset = structure.getValuesOffsets()[16];
+ value17Offset = structure.getValuesOffsets()[17];
+ value18Offset = structure.getValuesOffsets()[18];
+ value19Offset = structure.getValuesOffsets()[19];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue15(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue16(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue17(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue18(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue19(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 20);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 9) == 1;
+ }
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffsetBatch[i] + value9Offset, blockBuilder9);
+ }
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 10) == 1;
+ }
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffsetBatch[i] + value10Offset, blockBuilder10);
+ }
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 11) == 1;
+ }
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffsetBatch[i] + value11Offset, blockBuilder11);
+ }
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 12) == 1;
+ }
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffsetBatch[i] + value12Offset, blockBuilder12);
+ }
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 13) == 1;
+ }
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffsetBatch[i] + value13Offset, blockBuilder13);
+ }
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 14) == 1;
+ }
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffsetBatch[i] + value14Offset, blockBuilder14);
+ }
+ }
+ }
+
+ private void appendValue15(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 15) == 1;
+ }
+ BlockBuilder blockBuilder15 = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder15.appendNull();
+ }
+ else {
+ columnValueExtractor15.appendValue(buffer, valuesOffsetBatch[i] + value15Offset, blockBuilder15);
+ }
+ }
+ }
+
+ private void appendValue16(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 16) == 1;
+ }
+ BlockBuilder blockBuilder16 = pageBuilder.getBlockBuilder(outputChannelOffset + 16);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder16.appendNull();
+ }
+ else {
+ columnValueExtractor16.appendValue(buffer, valuesOffsetBatch[i] + value16Offset, blockBuilder16);
+ }
+ }
+ }
+
+ private void appendValue17(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 17) == 1;
+ }
+ BlockBuilder blockBuilder17 = pageBuilder.getBlockBuilder(outputChannelOffset + 17);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder17.appendNull();
+ }
+ else {
+ columnValueExtractor17.appendValue(buffer, valuesOffsetBatch[i] + value17Offset, blockBuilder17);
+ }
+ }
+ }
+
+ private void appendValue18(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 18) == 1;
+ }
+ BlockBuilder blockBuilder18 = pageBuilder.getBlockBuilder(outputChannelOffset + 18);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder18.appendNull();
+ }
+ else {
+ columnValueExtractor18.appendValue(buffer, valuesOffsetBatch[i] + value18Offset, blockBuilder18);
+ }
+ }
+ }
+
+ private void appendValue19(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 19) == 1;
+ }
+ BlockBuilder blockBuilder19 = pageBuilder.getBlockBuilder(outputChannelOffset + 19);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder19.appendNull();
+ }
+ else {
+ columnValueExtractor19.appendValue(buffer, valuesOffsetBatch[i] + value19Offset, blockBuilder19);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue9(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue10(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue11(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue12(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue13(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue14(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue15(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue16(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue17(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue18(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue19(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 20);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+
+ private void appendValue9(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder9 = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ if (entries.isNull(position, 9) == 1) {
+ blockBuilder9.appendNull();
+ }
+ else {
+ columnValueExtractor9.appendValue(buffer, valuesOffset + value9Offset, blockBuilder9);
+ }
+ }
+
+ private void appendValue10(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder10 = pageBuilder.getBlockBuilder(outputChannelOffset + 10);
+ if (entries.isNull(position, 10) == 1) {
+ blockBuilder10.appendNull();
+ }
+ else {
+ columnValueExtractor10.appendValue(buffer, valuesOffset + value10Offset, blockBuilder10);
+ }
+ }
+
+ private void appendValue11(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder11 = pageBuilder.getBlockBuilder(outputChannelOffset + 11);
+ if (entries.isNull(position, 11) == 1) {
+ blockBuilder11.appendNull();
+ }
+ else {
+ columnValueExtractor11.appendValue(buffer, valuesOffset + value11Offset, blockBuilder11);
+ }
+ }
+
+ private void appendValue12(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder12 = pageBuilder.getBlockBuilder(outputChannelOffset + 12);
+ if (entries.isNull(position, 12) == 1) {
+ blockBuilder12.appendNull();
+ }
+ else {
+ columnValueExtractor12.appendValue(buffer, valuesOffset + value12Offset, blockBuilder12);
+ }
+ }
+
+ private void appendValue13(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder13 = pageBuilder.getBlockBuilder(outputChannelOffset + 13);
+ if (entries.isNull(position, 13) == 1) {
+ blockBuilder13.appendNull();
+ }
+ else {
+ columnValueExtractor13.appendValue(buffer, valuesOffset + value13Offset, blockBuilder13);
+ }
+ }
+
+ private void appendValue14(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder14 = pageBuilder.getBlockBuilder(outputChannelOffset + 14);
+ if (entries.isNull(position, 14) == 1) {
+ blockBuilder14.appendNull();
+ }
+ else {
+ columnValueExtractor14.appendValue(buffer, valuesOffset + value14Offset, blockBuilder14);
+ }
+ }
+
+ private void appendValue15(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder15 = pageBuilder.getBlockBuilder(outputChannelOffset + 15);
+ if (entries.isNull(position, 15) == 1) {
+ blockBuilder15.appendNull();
+ }
+ else {
+ columnValueExtractor15.appendValue(buffer, valuesOffset + value15Offset, blockBuilder15);
+ }
+ }
+
+ private void appendValue16(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder16 = pageBuilder.getBlockBuilder(outputChannelOffset + 16);
+ if (entries.isNull(position, 16) == 1) {
+ blockBuilder16.appendNull();
+ }
+ else {
+ columnValueExtractor16.appendValue(buffer, valuesOffset + value16Offset, blockBuilder16);
+ }
+ }
+
+ private void appendValue17(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder17 = pageBuilder.getBlockBuilder(outputChannelOffset + 17);
+ if (entries.isNull(position, 17) == 1) {
+ blockBuilder17.appendNull();
+ }
+ else {
+ columnValueExtractor17.appendValue(buffer, valuesOffset + value17Offset, blockBuilder17);
+ }
+ }
+
+ private void appendValue18(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder18 = pageBuilder.getBlockBuilder(outputChannelOffset + 18);
+ if (entries.isNull(position, 18) == 1) {
+ blockBuilder18.appendNull();
+ }
+ else {
+ columnValueExtractor18.appendValue(buffer, valuesOffset + value18Offset, blockBuilder18);
+ }
+ }
+
+ private void appendValue19(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder19 = pageBuilder.getBlockBuilder(outputChannelOffset + 19);
+ if (entries.isNull(position, 19) == 1) {
+ blockBuilder19.appendNull();
+ }
+ else {
+ columnValueExtractor19.appendValue(buffer, valuesOffset + value19Offset, blockBuilder19);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender2Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender2Channels.java
new file mode 100644
index 000000000000..2ab51d4892c6
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender2Channels.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender2Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final int value1Offset;
+
+ public FixedWidthHashTableValuesAppender2Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 2);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ value1Offset = structure.getValuesOffsets()[1];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender3Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender3Channels.java
new file mode 100644
index 000000000000..f16a73f7d072
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender3Channels.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender3Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ public FixedWidthHashTableValuesAppender3Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 3);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender4Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender4Channels.java
new file mode 100644
index 000000000000..38b744586642
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender4Channels.java
@@ -0,0 +1,229 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender4Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ public FixedWidthHashTableValuesAppender4Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 4);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender5Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender5Channels.java
new file mode 100644
index 000000000000..0a0307e5ffef
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender5Channels.java
@@ -0,0 +1,268 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender5Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ public FixedWidthHashTableValuesAppender5Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 5);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender6Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender6Channels.java
new file mode 100644
index 000000000000..d5a0b98dd748
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender6Channels.java
@@ -0,0 +1,307 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender6Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ public FixedWidthHashTableValuesAppender6Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 6);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender7Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender7Channels.java
new file mode 100644
index 000000000000..9e89751bb151
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender7Channels.java
@@ -0,0 +1,346 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender7Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ public FixedWidthHashTableValuesAppender7Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 7);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender8Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender8Channels.java
new file mode 100644
index 000000000000..91d929ceb45a
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender8Channels.java
@@ -0,0 +1,385 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender8Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ public FixedWidthHashTableValuesAppender8Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 8);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender9Channels.java b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender9Channels.java
new file mode 100644
index 000000000000..ceeb31cb8c13
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/hash/fixedwidth/gen/FixedWidthHashTableValuesAppender9Channels.java
@@ -0,0 +1,424 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.hash.fixedwidth.gen;
+
+import io.trino.operator.hash.AbstractHashTableValuesAppender;
+import io.trino.operator.hash.ColumnValueExtractor;
+import io.trino.operator.hash.HashTableData;
+import io.trino.operator.hash.fastbb.FastByteBuffer;
+import io.trino.operator.hash.fixedwidth.FixedWidthEntryStructure;
+import io.trino.operator.hash.fixedwidth.FixedWidthGroupByHashTableEntries;
+import io.trino.spi.PageBuilder;
+import io.trino.spi.block.BlockBuilder;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * This class has been generated by the FixedWidthHashTableValuesAppenderGenerator class.
+ * The generation is a one-time event and is not repeated during build
+ */
+public final class FixedWidthHashTableValuesAppender9Channels
+ extends AbstractHashTableValuesAppender
+{
+ private final ColumnValueExtractor columnValueExtractor0;
+
+ private final ColumnValueExtractor columnValueExtractor1;
+
+ private final ColumnValueExtractor columnValueExtractor2;
+
+ private final ColumnValueExtractor columnValueExtractor3;
+
+ private final ColumnValueExtractor columnValueExtractor4;
+
+ private final ColumnValueExtractor columnValueExtractor5;
+
+ private final ColumnValueExtractor columnValueExtractor6;
+
+ private final ColumnValueExtractor columnValueExtractor7;
+
+ private final ColumnValueExtractor columnValueExtractor8;
+
+ private final int value1Offset;
+
+ private final int value2Offset;
+
+ private final int value3Offset;
+
+ private final int value4Offset;
+
+ private final int value5Offset;
+
+ private final int value6Offset;
+
+ private final int value7Offset;
+
+ private final int value8Offset;
+
+ public FixedWidthHashTableValuesAppender9Channels(FixedWidthEntryStructure structure)
+ {
+ requireNonNull(structure, "structure is null");
+ checkArgument(structure.getHashChannelsCount() == 9);
+ columnValueExtractor0 = structure.getColumnValueExtractors()[0];
+ columnValueExtractor1 = structure.getColumnValueExtractors()[1];
+ columnValueExtractor2 = structure.getColumnValueExtractors()[2];
+ columnValueExtractor3 = structure.getColumnValueExtractors()[3];
+ columnValueExtractor4 = structure.getColumnValueExtractors()[4];
+ columnValueExtractor5 = structure.getColumnValueExtractors()[5];
+ columnValueExtractor6 = structure.getColumnValueExtractors()[6];
+ columnValueExtractor7 = structure.getColumnValueExtractors()[7];
+ columnValueExtractor8 = structure.getColumnValueExtractors()[8];
+ value1Offset = structure.getValuesOffsets()[1];
+ value2Offset = structure.getValuesOffsets()[2];
+ value3Offset = structure.getValuesOffsets()[3];
+ value4Offset = structure.getValuesOffsets()[4];
+ value5Offset = structure.getValuesOffsets()[5];
+ value6Offset = structure.getValuesOffsets()[6];
+ value7Offset = structure.getValuesOffsets()[7];
+ value8Offset = structure.getValuesOffsets()[8];
+ }
+
+ @Override
+ protected void appendValuesTo(HashTableData data, int batchSize, int[] groupIdBatch,
+ int[] positionsBatch, int[] valuesOffsetBatch, boolean[] isNullBatch,
+ PageBuilder pageBuilder, int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ for (int i = 0; i < batchSize; i++) {
+ positionsBatch[i] = data.getPosition(groupIdBatch[i]);
+ }
+ for (int i = 0; i < batchSize; i++) {
+ valuesOffsetBatch[i] = entries.getValuesOffset(positionsBatch[i]);
+ }
+ FastByteBuffer buffer = entries.getBuffer();
+ appendValue0(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, batchSize, positionsBatch, valuesOffsetBatch, isNullBatch, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ for (int i = 0; i < batchSize; i++) {
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(positionsBatch[i]));
+ }
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 0) == 1;
+ }
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffsetBatch[i], blockBuilder0);
+ }
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 1) == 1;
+ }
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffsetBatch[i] + value1Offset, blockBuilder1);
+ }
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 2) == 1;
+ }
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffsetBatch[i] + value2Offset, blockBuilder2);
+ }
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 3) == 1;
+ }
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffsetBatch[i] + value3Offset, blockBuilder3);
+ }
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 4) == 1;
+ }
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffsetBatch[i] + value4Offset, blockBuilder4);
+ }
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 5) == 1;
+ }
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffsetBatch[i] + value5Offset, blockBuilder5);
+ }
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 6) == 1;
+ }
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffsetBatch[i] + value6Offset, blockBuilder6);
+ }
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 7) == 1;
+ }
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffsetBatch[i] + value7Offset, blockBuilder7);
+ }
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int batchSize, int[] positionsBatch,
+ int[] valuesOffsetBatch, boolean[] isNullBatch, FastByteBuffer buffer)
+ {
+ for (int i = 0; i < batchSize; i++) {
+ isNullBatch[i] = entries.isNull(positionsBatch[i], 8) == 1;
+ }
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ for (int i = 0; i < batchSize; i++) {
+ if (isNullBatch[i]) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffsetBatch[i] + value8Offset, blockBuilder8);
+ }
+ }
+ }
+
+ @Override
+ public void appendValuesTo(HashTableData data, int groupId, PageBuilder pageBuilder,
+ int outputChannelOffset, boolean outputHash)
+ {
+ FixedWidthGroupByHashTableEntries entries = (FixedWidthGroupByHashTableEntries) data.entries();
+ int position = data.getPosition(groupId);
+ FastByteBuffer buffer = entries.getBuffer();
+ int valuesOffset = entries.getValuesOffset(position);
+ appendValue0(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue1(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue2(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue3(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue4(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue5(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue6(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue7(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ appendValue8(pageBuilder, outputChannelOffset, entries, position, valuesOffset, buffer);
+ if (outputHash) {
+ BlockBuilder hashBlockBuilder = pageBuilder.getBlockBuilder(outputChannelOffset + 9);
+ BIGINT.writeLong(hashBlockBuilder, entries.getHash(position));
+ }
+ }
+
+ private void appendValue0(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder0 = pageBuilder.getBlockBuilder(outputChannelOffset);
+ if (entries.isNull(position, 0) == 1) {
+ blockBuilder0.appendNull();
+ }
+ else {
+ columnValueExtractor0.appendValue(buffer, valuesOffset, blockBuilder0);
+ }
+ }
+
+ private void appendValue1(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder1 = pageBuilder.getBlockBuilder(outputChannelOffset + 1);
+ if (entries.isNull(position, 1) == 1) {
+ blockBuilder1.appendNull();
+ }
+ else {
+ columnValueExtractor1.appendValue(buffer, valuesOffset + value1Offset, blockBuilder1);
+ }
+ }
+
+ private void appendValue2(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder2 = pageBuilder.getBlockBuilder(outputChannelOffset + 2);
+ if (entries.isNull(position, 2) == 1) {
+ blockBuilder2.appendNull();
+ }
+ else {
+ columnValueExtractor2.appendValue(buffer, valuesOffset + value2Offset, blockBuilder2);
+ }
+ }
+
+ private void appendValue3(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder3 = pageBuilder.getBlockBuilder(outputChannelOffset + 3);
+ if (entries.isNull(position, 3) == 1) {
+ blockBuilder3.appendNull();
+ }
+ else {
+ columnValueExtractor3.appendValue(buffer, valuesOffset + value3Offset, blockBuilder3);
+ }
+ }
+
+ private void appendValue4(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder4 = pageBuilder.getBlockBuilder(outputChannelOffset + 4);
+ if (entries.isNull(position, 4) == 1) {
+ blockBuilder4.appendNull();
+ }
+ else {
+ columnValueExtractor4.appendValue(buffer, valuesOffset + value4Offset, blockBuilder4);
+ }
+ }
+
+ private void appendValue5(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder5 = pageBuilder.getBlockBuilder(outputChannelOffset + 5);
+ if (entries.isNull(position, 5) == 1) {
+ blockBuilder5.appendNull();
+ }
+ else {
+ columnValueExtractor5.appendValue(buffer, valuesOffset + value5Offset, blockBuilder5);
+ }
+ }
+
+ private void appendValue6(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder6 = pageBuilder.getBlockBuilder(outputChannelOffset + 6);
+ if (entries.isNull(position, 6) == 1) {
+ blockBuilder6.appendNull();
+ }
+ else {
+ columnValueExtractor6.appendValue(buffer, valuesOffset + value6Offset, blockBuilder6);
+ }
+ }
+
+ private void appendValue7(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder7 = pageBuilder.getBlockBuilder(outputChannelOffset + 7);
+ if (entries.isNull(position, 7) == 1) {
+ blockBuilder7.appendNull();
+ }
+ else {
+ columnValueExtractor7.appendValue(buffer, valuesOffset + value7Offset, blockBuilder7);
+ }
+ }
+
+ private void appendValue8(PageBuilder pageBuilder, int outputChannelOffset,
+ FixedWidthGroupByHashTableEntries entries, int position, int valuesOffset,
+ FastByteBuffer buffer)
+ {
+ BlockBuilder blockBuilder8 = pageBuilder.getBlockBuilder(outputChannelOffset + 8);
+ if (entries.isNull(position, 8) == 1) {
+ blockBuilder8.appendNull();
+ }
+ else {
+ columnValueExtractor8.appendValue(buffer, valuesOffset + value8Offset, blockBuilder8);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java b/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java
index 826780540841..ced07b98e408 100644
--- a/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java
+++ b/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java
@@ -24,6 +24,7 @@
import io.trino.metadata.Split;
import io.trino.operator.Driver;
import io.trino.operator.DriverFactory;
+import io.trino.operator.GroupByHashFactory;
import io.trino.operator.PagesIndex;
import io.trino.operator.PipelineContext;
import io.trino.operator.TaskContext;
@@ -31,7 +32,6 @@
import io.trino.spi.Page;
import io.trino.spi.PageBuilder;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.sql.planner.plan.PlanNodeId;
import io.trino.type.BlockTypeOperators;
import io.trino.type.BlockTypeOperators.BlockPositionEqual;
@@ -73,8 +73,7 @@ public class IndexLoader
private final List keyTypes;
private final List keyEqualOperators;
private final PagesIndex.Factory pagesIndexFactory;
- private final JoinCompiler joinCompiler;
- private final BlockTypeOperators blockTypeOperators;
+ private final GroupByHashFactory groupByHashFactory;
@GuardedBy("this")
private IndexSnapshotLoader indexSnapshotLoader; // Lazily initialized
@@ -95,7 +94,7 @@ public IndexLoader(
DataSize maxIndexMemorySize,
IndexJoinLookupStats stats,
PagesIndex.Factory pagesIndexFactory,
- JoinCompiler joinCompiler,
+ GroupByHashFactory groupByHashFactory,
BlockTypeOperators blockTypeOperators)
{
requireNonNull(lookupSourceInputChannels, "lookupSourceInputChannels is null");
@@ -109,7 +108,7 @@ public IndexLoader(
requireNonNull(maxIndexMemorySize, "maxIndexMemorySize is null");
requireNonNull(stats, "stats is null");
requireNonNull(pagesIndexFactory, "pagesIndexFactory is null");
- requireNonNull(joinCompiler, "joinCompiler is null");
+ requireNonNull(groupByHashFactory, "groupByHashFactory is null");
requireNonNull(blockTypeOperators, "blockTypeOperators is null");
this.lookupSourceInputChannels = ImmutableSet.copyOf(lookupSourceInputChannels);
@@ -121,8 +120,7 @@ public IndexLoader(
this.maxIndexMemorySize = maxIndexMemorySize;
this.stats = stats;
this.pagesIndexFactory = pagesIndexFactory;
- this.joinCompiler = joinCompiler;
- this.blockTypeOperators = blockTypeOperators;
+ this.groupByHashFactory = groupByHashFactory;
this.keyTypes = keyOutputChannels.stream()
.map(outputTypes::get)
@@ -266,8 +264,7 @@ private synchronized void initializeStateIfNecessary()
expectedPositions,
maxIndexMemorySize,
pagesIndexFactory,
- joinCompiler,
- blockTypeOperators);
+ groupByHashFactory);
}
}
@@ -280,8 +277,7 @@ private static class IndexSnapshotLoader
private final Set allInputChannels;
private final List indexTypes;
private final AtomicReference indexSnapshotReference;
- private final JoinCompiler joinCompiler;
- private final BlockTypeOperators blockTypeOperators;
+ private final GroupByHashFactory groupByHashFactory;
private final IndexSnapshotBuilder indexSnapshotBuilder;
@@ -296,15 +292,13 @@ private IndexSnapshotLoader(
int expectedPositions,
DataSize maxIndexMemorySize,
PagesIndex.Factory pagesIndexFactory,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
this.pipelineContext = pipelineContext;
this.indexSnapshotReference = indexSnapshotReference;
this.lookupSourceInputChannels = lookupSourceInputChannels;
this.indexTypes = indexTypes;
- this.joinCompiler = joinCompiler;
- this.blockTypeOperators = blockTypeOperators;
+ this.groupByHashFactory = groupByHashFactory;
List outputTypes = indexBuildDriverFactoryProvider.getOutputTypes();
this.indexSnapshotBuilder = new IndexSnapshotBuilder(
@@ -332,7 +326,7 @@ public long getCacheSizeInBytes()
public boolean load(List requests)
{
// Generate a RecordSet that only presents index keys that have not been cached and are deduped based on lookupSourceInputChannels
- UnloadedIndexKeyRecordSet recordSetForLookupSource = new UnloadedIndexKeyRecordSet(pipelineContext.getSession(), indexSnapshotReference.get(), lookupSourceInputChannels, indexTypes, requests, joinCompiler, blockTypeOperators);
+ UnloadedIndexKeyRecordSet recordSetForLookupSource = new UnloadedIndexKeyRecordSet(pipelineContext.getSession(), indexSnapshotReference.get(), lookupSourceInputChannels, indexTypes, requests, groupByHashFactory);
// Drive index lookup to produce the output (landing in indexSnapshotBuilder)
try (Driver driver = driverFactory.createDriver(pipelineContext.addDriverContext())) {
@@ -353,7 +347,7 @@ public boolean load(List requests)
// Generate a RecordSet that presents unique index keys that have not been cached
UnloadedIndexKeyRecordSet indexKeysRecordSet = (lookupSourceInputChannels.equals(allInputChannels))
? recordSetForLookupSource
- : new UnloadedIndexKeyRecordSet(pipelineContext.getSession(), indexSnapshotReference.get(), allInputChannels, indexTypes, requests, joinCompiler, blockTypeOperators);
+ : new UnloadedIndexKeyRecordSet(pipelineContext.getSession(), indexSnapshotReference.get(), allInputChannels, indexTypes, requests, groupByHashFactory);
// Create lookup source with new data
IndexSnapshot newValue = indexSnapshotBuilder.createIndexSnapshot(indexKeysRecordSet);
diff --git a/core/trino-main/src/main/java/io/trino/operator/index/IndexLookupSourceFactory.java b/core/trino-main/src/main/java/io/trino/operator/index/IndexLookupSourceFactory.java
index df2554789a36..7f1280124f7b 100644
--- a/core/trino-main/src/main/java/io/trino/operator/index/IndexLookupSourceFactory.java
+++ b/core/trino-main/src/main/java/io/trino/operator/index/IndexLookupSourceFactory.java
@@ -18,6 +18,7 @@
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import io.airlift.units.DataSize;
+import io.trino.operator.GroupByHashFactory;
import io.trino.operator.PagesIndex;
import io.trino.operator.TaskContext;
import io.trino.operator.join.LookupSourceFactory;
@@ -25,7 +26,6 @@
import io.trino.operator.join.OuterPositionIterator;
import io.trino.operator.join.StaticLookupSourceProvider;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
import io.trino.type.BlockTypeOperators;
import java.util.List;
@@ -56,17 +56,17 @@ public IndexLookupSourceFactory(
IndexJoinLookupStats stats,
boolean shareIndexLoading,
PagesIndex.Factory pagesIndexFactory,
- JoinCompiler joinCompiler,
+ GroupByHashFactory groupByHashFactory,
BlockTypeOperators blockTypeOperators)
{
this.outputTypes = ImmutableList.copyOf(requireNonNull(outputTypes, "outputTypes is null"));
if (shareIndexLoading) {
- IndexLoader shared = new IndexLoader(lookupSourceInputChannels, keyOutputChannels, keyOutputHashChannel, outputTypes, indexBuildDriverFactoryProvider, 10_000, maxIndexMemorySize, stats, pagesIndexFactory, joinCompiler, blockTypeOperators);
+ IndexLoader shared = new IndexLoader(lookupSourceInputChannels, keyOutputChannels, keyOutputHashChannel, outputTypes, indexBuildDriverFactoryProvider, 10_000, maxIndexMemorySize, stats, pagesIndexFactory, groupByHashFactory, blockTypeOperators);
this.indexLoaderSupplier = () -> shared;
}
else {
- this.indexLoaderSupplier = () -> new IndexLoader(lookupSourceInputChannels, keyOutputChannels, keyOutputHashChannel, outputTypes, indexBuildDriverFactoryProvider, 10_000, maxIndexMemorySize, stats, pagesIndexFactory, joinCompiler, blockTypeOperators);
+ this.indexLoaderSupplier = () -> new IndexLoader(lookupSourceInputChannels, keyOutputChannels, keyOutputHashChannel, outputTypes, indexBuildDriverFactoryProvider, 10_000, maxIndexMemorySize, stats, pagesIndexFactory, groupByHashFactory, blockTypeOperators);
}
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/index/UnloadedIndexKeyRecordSet.java b/core/trino-main/src/main/java/io/trino/operator/index/UnloadedIndexKeyRecordSet.java
index 245ce62970e2..fc5d0f3ddd47 100644
--- a/core/trino-main/src/main/java/io/trino/operator/index/UnloadedIndexKeyRecordSet.java
+++ b/core/trino-main/src/main/java/io/trino/operator/index/UnloadedIndexKeyRecordSet.java
@@ -18,6 +18,7 @@
import io.airlift.slice.Slice;
import io.trino.Session;
import io.trino.operator.GroupByHash;
+import io.trino.operator.GroupByHashFactory;
import io.trino.operator.GroupByIdBlock;
import io.trino.operator.Work;
import io.trino.spi.Page;
@@ -25,8 +26,6 @@
import io.trino.spi.connector.RecordCursor;
import io.trino.spi.connector.RecordSet;
import io.trino.spi.type.Type;
-import io.trino.sql.gen.JoinCompiler;
-import io.trino.type.BlockTypeOperators;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntList;
import it.unimi.dsi.fastutil.ints.IntListIterator;
@@ -40,7 +39,6 @@
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Verify.verify;
-import static io.trino.operator.GroupByHash.createGroupByHash;
import static io.trino.operator.UpdateMemory.NOOP;
import static io.trino.operator.index.IndexSnapshot.UNLOADED_INDEX_KEY;
import static java.util.Objects.requireNonNull;
@@ -57,8 +55,7 @@ public UnloadedIndexKeyRecordSet(
Set channelsForDistinct,
List types,
List requests,
- JoinCompiler joinCompiler,
- BlockTypeOperators blockTypeOperators)
+ GroupByHashFactory groupByHashFactory)
{
requireNonNull(existingSnapshot, "existingSnapshot is null");
this.types = ImmutableList.copyOf(requireNonNull(types, "types is null"));
@@ -73,7 +70,7 @@ public UnloadedIndexKeyRecordSet(
}
ImmutableList.Builder builder = ImmutableList.builder();
- GroupByHash groupByHash = createGroupByHash(session, distinctChannelTypes, normalizedDistinctChannels, Optional.empty(), 10_000, joinCompiler, blockTypeOperators, NOOP);
+ GroupByHash groupByHash = groupByHashFactory.createGroupByHash(session, distinctChannelTypes, normalizedDistinctChannels, Optional.empty(), 10_000, NOOP);
for (UpdateRequest request : requests) {
Page page = request.getPage();
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/AdaptivePositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/AdaptivePositionsAppender.java
new file mode 100644
index 000000000000..52ce1252ec34
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/AdaptivePositionsAppender.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilderStatus;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import io.trino.type.BlockTypeOperators.BlockPositionEqual;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import javax.annotation.Nullable;
+
+import static java.util.Objects.requireNonNull;
+
+public class AdaptivePositionsAppender
+ implements BlockTypeAwarePositionsAppender
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(AdaptivePositionsAppender.class).instanceSize();
+ private static final int EMPTY_STATE_MARKER = 0;
+ private static final int FLAT_STATE_MARKER = -1;
+
+ private final int expectedPositions;
+ private final BlockPositionEqual equalOperator;
+ private final BlockTypeAwarePositionsAppender flat;
+
+ @Nullable
+ private Block rleValue;
+
+ // FLAT_STATE_MARKER means flat state, EMPTY_STATE_MARKER means initial empty state, positive means rle state and the current rle position count.
+ private int rlePositionCount = EMPTY_STATE_MARKER;
+
+ public AdaptivePositionsAppender(BlockPositionEqual equalOperator, BlockTypeAwarePositionsAppender flat, int expectedPositions)
+ {
+ this.flat = requireNonNull(flat, "appender is null");
+ this.expectedPositions = expectedPositions;
+ this.equalOperator = equalOperator;
+ }
+
+ @Override
+ public void append(IntArrayList positions, Block source)
+ {
+ switchToFlat();
+ flat.append(positions, source);
+ }
+
+ @Override
+ public void appendDictionary(IntArrayList positions, DictionaryBlock source)
+ {
+ switchToFlat();
+ flat.appendDictionary(positions, source);
+ }
+
+ @Override
+ public void appendRle(RunLengthEncodedBlock source)
+ {
+ if (isEmpty()) {
+ // initial empty state, switch to rle state
+ rleValue = source.getValue();
+ rlePositionCount = source.getPositionCount();
+ }
+ else if (rleValue != null) {
+ // we are in the rle state
+ if (equalOperator.equalNullSafe(rleValue, 0, source.getValue(), 0)) {
+ // the values match. we can just add positions.
+ this.rlePositionCount += source.getPositionCount();
+ return;
+ }
+ // RLE values do not match. switch to flat state
+ switchToFlat();
+ flat.appendRle(source);
+ }
+ else {
+ // flat state
+ flat.appendRle(source);
+ }
+ }
+
+ @Override
+ public void appendRow(Block source, int position)
+ {
+ switchToFlat();
+ flat.appendRow(source, position);
+ }
+
+ @Override
+ public Block build()
+ {
+ if (rleValue != null) {
+ return new RunLengthEncodedBlock(rleValue, rlePositionCount);
+ }
+
+ return flat.build();
+ }
+
+ @Override
+ public BlockTypeAwarePositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus)
+ {
+ return new AdaptivePositionsAppender(equalOperator, (BlockTypeAwarePositionsAppender) this.flat.newStateLike(blockBuilderStatus), expectedPositions);
+ }
+
+ @Override
+ public long getRetainedSizeInBytes()
+ {
+ long retainedRleSize = rleValue != null ? rleValue.getRetainedSizeInBytes() : 0;
+ long currentRetainedSize = flat != null ? flat.getRetainedSizeInBytes() : 0;
+ return INSTANCE_SIZE + retainedRleSize + currentRetainedSize;
+ }
+
+ private void switchToFlat()
+ {
+ if (rleValue != null) {
+ // we are in the rle state, flatten all RLE blocks
+ flat.appendRle(new RunLengthEncodedBlock(rleValue, rlePositionCount));
+ rleValue = null;
+ }
+ rlePositionCount = FLAT_STATE_MARKER;
+ }
+
+ private boolean isEmpty()
+ {
+ return rlePositionCount == EMPTY_STATE_MARKER;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/BlockTypeAwarePositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/BlockTypeAwarePositionsAppender.java
new file mode 100644
index 000000000000..fc1c9b8fd9fb
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/BlockTypeAwarePositionsAppender.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.block.Block;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+
+interface BlockTypeAwarePositionsAppender
+ extends PositionsAppender
+{
+ /**
+ * Appends value from the {@code rleBlock} to this appender {@link RunLengthEncodedBlock#getPositionCount()} times.
+ * The result is the same as with using {@link PositionsAppender#append(IntArrayList, Block)} with
+ * positions list [0...{@link RunLengthEncodedBlock#getPositionCount()} -1]
+ * but with possible performance optimizations for {@link RunLengthEncodedBlock}.
+ */
+ void appendRle(RunLengthEncodedBlock rleBlock);
+
+ /**
+ * Appends given {@code positions} from the {@code source} to this appender.
+ * The result is the same as with using {@link PositionsAppender#append(IntArrayList, Block)}
+ * but with possible performance optimizations for {@link DictionaryBlock}.
+ */
+ void appendDictionary(IntArrayList positions, DictionaryBlock source);
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/BlockTypeDispatchingPositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/BlockTypeDispatchingPositionsAppender.java
new file mode 100644
index 000000000000..359c22099048
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/BlockTypeDispatchingPositionsAppender.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilderStatus;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import javax.annotation.Nullable;
+
+import static java.util.Objects.requireNonNull;
+
+public class BlockTypeDispatchingPositionsAppender
+ implements BlockTypeAwarePositionsAppender
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(BlockTypeDispatchingPositionsAppender.class).instanceSize();
+
+ private final BlockTypeAwarePositionsAppender delegate;
+
+ public BlockTypeDispatchingPositionsAppender(BlockTypeAwarePositionsAppender delegate)
+ {
+ this.delegate = requireNonNull(delegate, "delegate is null");
+ }
+
+ @Override
+ public void append(IntArrayList positions, Block source)
+ {
+ if (positions.isEmpty()) {
+ return;
+ }
+
+ if (source instanceof RunLengthEncodedBlock) {
+ delegate.appendRle(flattenRle((RunLengthEncodedBlock) source, positions.size()));
+ }
+ else if (source instanceof DictionaryBlock) {
+ appendDictionary(positions, (DictionaryBlock) source);
+ }
+ else {
+ delegate.append(positions, source);
+ }
+ }
+
+ private BlockView flattenDictionary(DictionaryBlock source, IntArrayList positions)
+ {
+ Block dictionary = source.getDictionary();
+ if (!(dictionary instanceof RunLengthEncodedBlock || dictionary instanceof DictionaryBlock)) {
+ return new BlockView(source, positions);
+ }
+
+ while (dictionary instanceof RunLengthEncodedBlock || dictionary instanceof DictionaryBlock) {
+ if (dictionary instanceof RunLengthEncodedBlock) {
+ // since at some level, dictionary contains only a single value, it can be flattened to rle
+ RunLengthEncodedBlock rleDictionary = flattenRle((RunLengthEncodedBlock) dictionary, positions.size());
+ return new BlockView(rleDictionary, positions);
+ }
+ else {
+ // dictionary is a nested dictionary. we need to remap the ids
+ DictionaryBlock dictionaryValue = (DictionaryBlock) dictionary;
+ int[] newPositions = new int[positions.size()];
+ for (int i = 0; i < newPositions.length; i++) {
+ newPositions[i] = source.getId(positions.getInt(i));
+ }
+ positions = IntArrayList.wrap(newPositions);
+ dictionary = dictionaryValue.getDictionary();
+ source = dictionaryValue;
+ }
+ }
+ return new BlockView(source, positions);
+ }
+
+ private RunLengthEncodedBlock flattenRle(RunLengthEncodedBlock source, int positionCount)
+ {
+ Block value = source.getValue();
+ if (!(value instanceof RunLengthEncodedBlock || value instanceof DictionaryBlock)) {
+ if (source.getPositionCount() == positionCount) {
+ return source;
+ }
+ return new RunLengthEncodedBlock(source.getValue(), positionCount);
+ }
+
+ int position = 0;
+ while (value instanceof RunLengthEncodedBlock || value instanceof DictionaryBlock) {
+ if (value instanceof RunLengthEncodedBlock) {
+ value = ((RunLengthEncodedBlock) value).getValue();
+ position = 0;
+ }
+ else {
+ DictionaryBlock dictionaryValue = (DictionaryBlock) value;
+ position = dictionaryValue.getId(position);
+ value = dictionaryValue.getDictionary();
+ }
+ }
+
+ if (value.getPositionCount() > 1 || position != 0) {
+ value = value.getSingleValueBlock(position);
+ }
+ return new RunLengthEncodedBlock(value, positionCount);
+ }
+
+ @Override
+ public void appendRle(RunLengthEncodedBlock source)
+ {
+ delegate.appendRle(flattenRle(source, source.getPositionCount()));
+ }
+
+ @Override
+ public void appendDictionary(IntArrayList positions, DictionaryBlock source)
+ {
+ BlockView flatDictionary = flattenDictionary(source, positions);
+ if (flatDictionary.getBlock() instanceof DictionaryBlock) {
+ delegate.appendDictionary(flatDictionary.getPositions(), (DictionaryBlock) flatDictionary.getBlock());
+ }
+ else {
+ delegate.appendRle((RunLengthEncodedBlock) flatDictionary.getBlock());
+ }
+ }
+
+ @Override
+ public Block build()
+ {
+ return delegate.build();
+ }
+
+ @Override
+ public BlockTypeAwarePositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus)
+ {
+ return new BlockTypeDispatchingPositionsAppender((BlockTypeAwarePositionsAppender) delegate.newStateLike(blockBuilderStatus));
+ }
+
+ @Override
+ public void appendRow(Block source, int position)
+ {
+ delegate.appendRow(source, position);
+ }
+
+ @Override
+ public long getRetainedSizeInBytes()
+ {
+ return INSTANCE_SIZE + delegate.getRetainedSizeInBytes();
+ }
+
+ private static class BlockView
+ {
+ private final Block block;
+ private final IntArrayList positions;
+
+ private BlockView(Block block, IntArrayList positions)
+ {
+ this.block = requireNonNull(block, "block is null");
+ this.positions = requireNonNull(positions, "positions is null");
+ }
+
+ public Block getBlock()
+ {
+ return block;
+ }
+
+ public IntArrayList getPositions()
+ {
+ return positions;
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/BytePositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/BytePositionsAppender.java
new file mode 100644
index 000000000000..dd33569448a2
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/BytePositionsAppender.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilderStatus;
+import io.trino.spi.block.BlockUtil;
+import io.trino.spi.block.ByteArrayBlock;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.Optional;
+
+import static io.airlift.slice.SizeOf.sizeOf;
+import static io.trino.spi.block.BlockUtil.calculateBlockResetSize;
+import static java.lang.Math.max;
+
+public class BytePositionsAppender
+ implements BlockTypeAwarePositionsAppender
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(BytePositionsAppender.class).instanceSize();
+ private static final Block NULL_VALUE_BLOCK = new ByteArrayBlock(1, Optional.of(new boolean[] {true}), new byte[1]);
+
+ @Nullable
+ private final BlockBuilderStatus blockBuilderStatus;
+ private boolean initialized;
+ private final int initialEntryCount;
+
+ private int positionCount;
+ private boolean hasNullValue;
+ private boolean hasNonNullValue;
+
+ // it is assumed that these arrays are the same length
+ private boolean[] valueIsNull = new boolean[0];
+ private byte[] values = new byte[0];
+
+ private long retainedSizeInBytes;
+
+ public BytePositionsAppender(@Nullable BlockBuilderStatus blockBuilderStatus, int expectedEntries)
+ {
+ this.blockBuilderStatus = blockBuilderStatus;
+ this.initialEntryCount = max(expectedEntries, 1);
+
+ updateDataSize();
+ }
+
+ @Override
+ public void append(IntArrayList positions, Block block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ boolean isNull = block.isNull(position);
+ int positionIndex = positionCount + i;
+ if (isNull) {
+ valueIsNull[positionIndex] = true;
+ hasNullValue = true;
+ }
+ else {
+ values[positionIndex] = block.getByte(position, 0);
+ hasNonNullValue = true;
+ }
+ }
+ this.positionCount += newPositionCount;
+ }
+ else {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ values[positionCount + i] = block.getByte(position, 0);
+ }
+ positionCount += newPositionCount;
+ this.hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(ByteArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendDictionary(IntArrayList positions, DictionaryBlock block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ if (block.isNull(position)) {
+ valueIsNull[positionCount] = true;
+ hasNullValue = true;
+ }
+ else {
+ values[positionCount] = block.getByte(position, 0);
+ hasNonNullValue = true;
+ }
+ positionCount++;
+ }
+ }
+ else {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ values[positionCount] = block.getByte(position, 0);
+ positionCount++;
+ }
+ hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(ByteArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendRle(RunLengthEncodedBlock block)
+ {
+ int rlePositionCount = block.getPositionCount();
+ int sourcePosition = 0;
+ ensureCapacity(positionCount + rlePositionCount);
+ if (block.isNull(sourcePosition)) {
+ Arrays.fill(valueIsNull, positionCount, positionCount + rlePositionCount, true);
+ hasNullValue = true;
+ }
+ else {
+ byte value = block.getByte(sourcePosition, 0);
+ Arrays.fill(values, positionCount, positionCount + rlePositionCount, value);
+ hasNonNullValue = true;
+ }
+ positionCount += rlePositionCount;
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(ByteArrayBlock.SIZE_IN_BYTES_PER_POSITION * rlePositionCount);
+ }
+ }
+
+ @Override
+ public void appendRow(Block source, int position)
+ {
+ if (source.isNull(position)) {
+ appendNull();
+ }
+ else {
+ writeByte(source.getByte(position, 0));
+ }
+ }
+
+ private void appendNull()
+ {
+ ensureCapacity(positionCount + 1);
+
+ valueIsNull[positionCount] = true;
+
+ hasNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(ByteArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ private void writeByte(byte value)
+ {
+ ensureCapacity(positionCount + 1);
+
+ values[positionCount] = value;
+
+ hasNonNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(ByteArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ @Override
+ public Block build()
+ {
+ if (!hasNonNullValue) {
+ return new RunLengthEncodedBlock(NULL_VALUE_BLOCK, positionCount);
+ }
+ return new ByteArrayBlock(positionCount, hasNullValue ? Optional.of(valueIsNull) : Optional.empty(), values);
+ }
+
+ @Override
+ public BlockTypeAwarePositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus)
+ {
+ return new BytePositionsAppender(blockBuilderStatus, calculateBlockResetSize(positionCount));
+ }
+
+ @Override
+ public long getRetainedSizeInBytes()
+ {
+ return retainedSizeInBytes;
+ }
+
+ private void ensureCapacity(int capacity)
+ {
+ if (values.length >= capacity) {
+ return;
+ }
+
+ int newSize;
+ if (initialized) {
+ newSize = BlockUtil.calculateNewArraySize(values.length);
+ }
+ else {
+ newSize = initialEntryCount;
+ initialized = true;
+ }
+ newSize = Math.max(newSize, capacity);
+
+ valueIsNull = Arrays.copyOf(valueIsNull, newSize);
+ values = Arrays.copyOf(values, newSize);
+ updateDataSize();
+ }
+
+ private void updateDataSize()
+ {
+ retainedSizeInBytes = INSTANCE_SIZE + sizeOf(valueIsNull) + sizeOf(values);
+ if (blockBuilderStatus != null) {
+ retainedSizeInBytes += BlockBuilderStatus.INSTANCE_SIZE;
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/Int128PositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/Int128PositionsAppender.java
new file mode 100644
index 000000000000..e1e36afc8fdc
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/Int128PositionsAppender.java
@@ -0,0 +1,263 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilderStatus;
+import io.trino.spi.block.BlockUtil;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.Int128ArrayBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.Optional;
+
+import static io.airlift.slice.SizeOf.SIZE_OF_LONG;
+import static io.airlift.slice.SizeOf.sizeOf;
+import static io.trino.spi.block.BlockUtil.calculateBlockResetSize;
+import static java.lang.Math.max;
+
+public class Int128PositionsAppender
+ implements BlockTypeAwarePositionsAppender
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(Int128PositionsAppender.class).instanceSize();
+ private static final Block NULL_VALUE_BLOCK = new Int128ArrayBlock(1, Optional.of(new boolean[] {true}), new long[2]);
+
+ @Nullable
+ private final BlockBuilderStatus blockBuilderStatus;
+ private boolean initialized;
+ private final int initialEntryCount;
+
+ private int positionCount;
+ private boolean hasNullValue;
+ private boolean hasNonNullValue;
+
+ // it is assumed that these arrays are the same length
+ private boolean[] valueIsNull = new boolean[0];
+ private long[] values = new long[0];
+
+ private long retainedSizeInBytes;
+
+ public Int128PositionsAppender(@Nullable BlockBuilderStatus blockBuilderStatus, int expectedEntries)
+ {
+ this.blockBuilderStatus = blockBuilderStatus;
+ this.initialEntryCount = max(expectedEntries, 1);
+
+ updateDataSize();
+ }
+
+ @Override
+ public void append(IntArrayList positions, Block block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+
+ if (block.mayHaveNull()) {
+ int positionIndex = positionCount * 2;
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ boolean isNull = block.isNull(position);
+
+ if (isNull) {
+ valueIsNull[positionCount + i] = true;
+ hasNullValue = true;
+ }
+ else {
+ values[positionIndex] = block.getLong(position, 0);
+ values[positionIndex + 1] = block.getLong(position, SIZE_OF_LONG);
+ hasNonNullValue = true;
+ }
+ positionIndex += 2;
+ }
+ this.positionCount += newPositionCount;
+ }
+ else {
+ int positionIndex = positionCount * 2;
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ values[positionIndex] = block.getLong(position, 0);
+ values[positionIndex + 1] = block.getLong(position, SIZE_OF_LONG);
+ positionIndex += 2;
+ }
+ positionCount += newPositionCount;
+ this.hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(Int128ArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendDictionary(IntArrayList positions, DictionaryBlock block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+ if (block.mayHaveNull()) {
+ int positionIndex = positionCount * 2;
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ if (block.isNull(position)) {
+ valueIsNull[positionCount] = true;
+ hasNullValue = true;
+ }
+ else {
+ values[positionIndex] = block.getLong(position, 0);
+ values[positionIndex + 1] = block.getLong(position, SIZE_OF_LONG);
+ hasNonNullValue = true;
+ }
+ positionIndex += 2;
+ positionCount++;
+ }
+ }
+ else {
+ int positionIndex = positionCount * 2;
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ values[positionIndex] = block.getLong(position, 0);
+ values[positionIndex + 1] = block.getLong(position, SIZE_OF_LONG);
+ positionCount++;
+ positionIndex += 2;
+ }
+ hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(Int128ArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendRle(RunLengthEncodedBlock block)
+ {
+ int rlePositionCount = block.getPositionCount();
+ int sourcePosition = 0;
+ ensureCapacity(positionCount + rlePositionCount);
+ if (block.isNull(sourcePosition)) {
+ Arrays.fill(valueIsNull, positionCount, positionCount + rlePositionCount, true);
+ hasNullValue = true;
+ }
+ else {
+ long valueHigh = block.getLong(sourcePosition, 0);
+ long valueLow = block.getLong(sourcePosition, SIZE_OF_LONG);
+ int positionIndex = positionCount * 2;
+ for (int i = 0; i < rlePositionCount; i++) {
+ values[positionIndex] = valueHigh;
+ values[positionIndex + 1] = valueLow;
+ positionIndex += 2;
+ }
+ hasNonNullValue = true;
+ }
+ positionCount += rlePositionCount;
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(Int128ArrayBlock.SIZE_IN_BYTES_PER_POSITION * rlePositionCount);
+ }
+ }
+
+ @Override
+ public void appendRow(Block source, int position)
+ {
+ if (source.isNull(position)) {
+ appendNull();
+ }
+ else {
+ writeInt128(source.getLong(position, 0), source.getLong(position, SIZE_OF_LONG));
+ }
+ }
+
+ private void appendNull()
+ {
+ ensureCapacity(positionCount + 1);
+
+ valueIsNull[positionCount] = true;
+
+ hasNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(Int128ArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ private void writeInt128(long high, long low)
+ {
+ ensureCapacity(positionCount + 1);
+
+ int positionIndex = positionCount * 2;
+ this.values[positionIndex] = high;
+ this.values[positionIndex + 1] = low;
+
+ hasNonNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(Int128ArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ @Override
+ public Block build()
+ {
+ if (!hasNonNullValue) {
+ return new RunLengthEncodedBlock(NULL_VALUE_BLOCK, positionCount);
+ }
+ return new Int128ArrayBlock(positionCount, hasNullValue ? Optional.of(valueIsNull) : Optional.empty(), values);
+ }
+
+ @Override
+ public BlockTypeAwarePositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus)
+ {
+ return new Int128PositionsAppender(blockBuilderStatus, calculateBlockResetSize(positionCount));
+ }
+
+ @Override
+ public long getRetainedSizeInBytes()
+ {
+ return retainedSizeInBytes;
+ }
+
+ private void ensureCapacity(int capacity)
+ {
+ if (valueIsNull.length >= capacity) {
+ return;
+ }
+
+ int newSize;
+ if (initialized) {
+ newSize = BlockUtil.calculateNewArraySize(valueIsNull.length);
+ }
+ else {
+ newSize = initialEntryCount;
+ initialized = true;
+ }
+ newSize = Math.max(newSize, capacity);
+
+ valueIsNull = Arrays.copyOf(valueIsNull, newSize);
+ values = Arrays.copyOf(values, newSize * 2);
+ updateDataSize();
+ }
+
+ private void updateDataSize()
+ {
+ retainedSizeInBytes = INSTANCE_SIZE + sizeOf(valueIsNull) + sizeOf(values);
+ if (blockBuilderStatus != null) {
+ retainedSizeInBytes += BlockBuilderStatus.INSTANCE_SIZE;
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/Int96PositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/Int96PositionsAppender.java
new file mode 100644
index 000000000000..d1371e85ce30
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/Int96PositionsAppender.java
@@ -0,0 +1,254 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilderStatus;
+import io.trino.spi.block.BlockUtil;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.Int96ArrayBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.Optional;
+
+import static io.airlift.slice.SizeOf.SIZE_OF_LONG;
+import static io.airlift.slice.SizeOf.sizeOf;
+import static io.trino.spi.block.BlockUtil.calculateBlockResetSize;
+import static java.lang.Math.max;
+
+public class Int96PositionsAppender
+ implements BlockTypeAwarePositionsAppender
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(Int96PositionsAppender.class).instanceSize();
+ private static final Block NULL_VALUE_BLOCK = new Int96ArrayBlock(1, Optional.of(new boolean[] {true}), new long[1], new int[1]);
+
+ @Nullable
+ private final BlockBuilderStatus blockBuilderStatus;
+ private boolean initialized;
+ private final int initialEntryCount;
+
+ private int positionCount;
+ private boolean hasNullValue;
+ private boolean hasNonNullValue;
+
+ // it is assumed that these arrays are the same length
+ private boolean[] valueIsNull = new boolean[0];
+ private long[] high = new long[0];
+ private int[] low = new int[0];
+
+ private long retainedSizeInBytes;
+
+ public Int96PositionsAppender(@Nullable BlockBuilderStatus blockBuilderStatus, int expectedEntries)
+ {
+ this.blockBuilderStatus = blockBuilderStatus;
+ this.initialEntryCount = max(expectedEntries, 1);
+
+ updateDataSize();
+ }
+
+ @Override
+ public void append(IntArrayList positions, Block block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ boolean isNull = block.isNull(position);
+ int positionIndex = positionCount + i;
+ if (isNull) {
+ valueIsNull[positionIndex] = true;
+ hasNullValue = true;
+ }
+ else {
+ high[positionIndex] = block.getLong(position, 0);
+ low[positionIndex] = block.getInt(position, SIZE_OF_LONG);
+ hasNonNullValue = true;
+ }
+ }
+ this.positionCount += newPositionCount;
+ }
+ else {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ high[positionCount + i] = block.getLong(position, 0);
+ low[positionCount + i] = block.getInt(position, SIZE_OF_LONG);
+ }
+ positionCount += newPositionCount;
+ this.hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(Int96ArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendDictionary(IntArrayList positions, DictionaryBlock block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ if (block.isNull(position)) {
+ valueIsNull[positionCount] = true;
+ hasNullValue = true;
+ }
+ else {
+ high[positionCount] = block.getLong(position, 0);
+ low[positionCount] = block.getInt(position, SIZE_OF_LONG);
+ hasNonNullValue = true;
+ }
+ positionCount++;
+ }
+ }
+ else {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ high[positionCount] = block.getLong(position, 0);
+ low[positionCount] = block.getInt(position, SIZE_OF_LONG);
+ positionCount++;
+ }
+ hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(Int96ArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendRle(RunLengthEncodedBlock block)
+ {
+ int rlePositionCount = block.getPositionCount();
+ int sourcePosition = 0;
+ ensureCapacity(positionCount + rlePositionCount);
+ if (block.isNull(sourcePosition)) {
+ Arrays.fill(valueIsNull, positionCount, positionCount + rlePositionCount, true);
+ hasNullValue = true;
+ }
+ else {
+ long valueHigh = block.getLong(sourcePosition, 0);
+ int valueLow = block.getInt(sourcePosition, SIZE_OF_LONG);
+ for (int i = 0; i < rlePositionCount; i++) {
+ high[positionCount + i] = valueHigh;
+ low[positionCount + i] = valueLow;
+ }
+ hasNonNullValue = true;
+ }
+ positionCount += rlePositionCount;
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(Int96ArrayBlock.SIZE_IN_BYTES_PER_POSITION * rlePositionCount);
+ }
+ }
+
+ @Override
+ public void appendRow(Block source, int position)
+ {
+ if (source.isNull(position)) {
+ appendNull();
+ }
+ else {
+ writeInt96(source.getLong(position, 0), source.getInt(position, SIZE_OF_LONG));
+ }
+ }
+
+ private void appendNull()
+ {
+ ensureCapacity(positionCount + 1);
+
+ valueIsNull[positionCount] = true;
+
+ hasNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(Int96ArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ private void writeInt96(long high, int low)
+ {
+ ensureCapacity(positionCount + 1);
+
+ this.high[positionCount] = high;
+ this.low[positionCount] = low;
+
+ hasNonNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(Int96ArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ @Override
+ public Block build()
+ {
+ if (!hasNonNullValue) {
+ return new RunLengthEncodedBlock(NULL_VALUE_BLOCK, positionCount);
+ }
+ return new Int96ArrayBlock(positionCount, hasNullValue ? Optional.of(valueIsNull) : Optional.empty(), high, low);
+ }
+
+ @Override
+ public BlockTypeAwarePositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus)
+ {
+ return new Int96PositionsAppender(blockBuilderStatus, calculateBlockResetSize(positionCount));
+ }
+
+ @Override
+ public long getRetainedSizeInBytes()
+ {
+ return retainedSizeInBytes;
+ }
+
+ private void ensureCapacity(int capacity)
+ {
+ if (valueIsNull.length >= capacity) {
+ return;
+ }
+
+ int newSize;
+ if (initialized) {
+ newSize = BlockUtil.calculateNewArraySize(valueIsNull.length);
+ }
+ else {
+ newSize = initialEntryCount;
+ initialized = true;
+ }
+ newSize = Math.max(newSize, capacity);
+
+ valueIsNull = Arrays.copyOf(valueIsNull, newSize);
+ high = Arrays.copyOf(high, newSize);
+ low = Arrays.copyOf(low, newSize);
+ updateDataSize();
+ }
+
+ private void updateDataSize()
+ {
+ retainedSizeInBytes = INSTANCE_SIZE + sizeOf(valueIsNull) + sizeOf(high) + sizeOf(low);
+ if (blockBuilderStatus != null) {
+ retainedSizeInBytes += BlockBuilderStatus.INSTANCE_SIZE;
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/IntPositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/IntPositionsAppender.java
new file mode 100644
index 000000000000..f83b6c9dad46
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/IntPositionsAppender.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilderStatus;
+import io.trino.spi.block.BlockUtil;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.IntArrayBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.Optional;
+
+import static io.airlift.slice.SizeOf.sizeOf;
+import static io.trino.spi.block.BlockUtil.calculateBlockResetSize;
+import static java.lang.Math.max;
+
+public class IntPositionsAppender
+ implements BlockTypeAwarePositionsAppender
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(IntPositionsAppender.class).instanceSize();
+ private static final Block NULL_VALUE_BLOCK = new IntArrayBlock(1, Optional.of(new boolean[] {true}), new int[1]);
+
+ @Nullable
+ private final BlockBuilderStatus blockBuilderStatus;
+ private boolean initialized;
+ private final int initialEntryCount;
+
+ private int positionCount;
+ private boolean hasNullValue;
+ private boolean hasNonNullValue;
+
+ // it is assumed that these arrays are the same length
+ private boolean[] valueIsNull = new boolean[0];
+ private int[] values = new int[0];
+
+ private long retainedSizeInBytes;
+
+ public IntPositionsAppender(@Nullable BlockBuilderStatus blockBuilderStatus, int expectedEntries)
+ {
+ this.blockBuilderStatus = blockBuilderStatus;
+ this.initialEntryCount = max(expectedEntries, 1);
+
+ updateDataSize();
+ }
+
+ @Override
+ public void append(IntArrayList positions, Block block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ boolean isNull = block.isNull(position);
+ int positionIndex = positionCount + i;
+ if (isNull) {
+ valueIsNull[positionIndex] = true;
+ hasNullValue = true;
+ }
+ else {
+ values[positionIndex] = block.getInt(position, 0);
+ hasNonNullValue = true;
+ }
+ }
+ this.positionCount += newPositionCount;
+ }
+ else {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ values[positionCount + i] = block.getInt(position, 0);
+ }
+ positionCount += newPositionCount;
+ this.hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(IntArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendDictionary(IntArrayList positions, DictionaryBlock block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ if (block.isNull(position)) {
+ valueIsNull[positionCount] = true;
+ hasNullValue = true;
+ }
+ else {
+ values[positionCount] = block.getInt(position, 0);
+ hasNonNullValue = true;
+ }
+ positionCount++;
+ }
+ }
+ else {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ values[positionCount] = block.getInt(position, 0);
+ positionCount++;
+ }
+ hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(IntArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendRle(RunLengthEncodedBlock block)
+ {
+ int rlePositionCount = block.getPositionCount();
+ int sourcePosition = 0;
+ ensureCapacity(positionCount + rlePositionCount);
+ if (block.isNull(sourcePosition)) {
+ Arrays.fill(valueIsNull, positionCount, positionCount + rlePositionCount, true);
+ hasNullValue = true;
+ }
+ else {
+ int value = block.getInt(sourcePosition, 0);
+ Arrays.fill(values, positionCount, positionCount + rlePositionCount, value);
+ hasNonNullValue = true;
+ }
+ positionCount += rlePositionCount;
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(IntArrayBlock.SIZE_IN_BYTES_PER_POSITION * rlePositionCount);
+ }
+ }
+
+ @Override
+ public void appendRow(Block source, int position)
+ {
+ if (source.isNull(position)) {
+ appendNull();
+ }
+ else {
+ writeInt(source.getInt(position, 0));
+ }
+ }
+
+ private void appendNull()
+ {
+ ensureCapacity(positionCount + 1);
+
+ valueIsNull[positionCount] = true;
+
+ hasNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(IntArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ private void writeInt(int value)
+ {
+ ensureCapacity(positionCount + 1);
+
+ values[positionCount] = value;
+
+ hasNonNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(IntArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ @Override
+ public Block build()
+ {
+ if (!hasNonNullValue) {
+ return new RunLengthEncodedBlock(NULL_VALUE_BLOCK, positionCount);
+ }
+ return new IntArrayBlock(positionCount, hasNullValue ? Optional.of(valueIsNull) : Optional.empty(), values);
+ }
+
+ @Override
+ public BlockTypeAwarePositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus)
+ {
+ return new IntPositionsAppender(blockBuilderStatus, calculateBlockResetSize(positionCount));
+ }
+
+ @Override
+ public long getRetainedSizeInBytes()
+ {
+ return retainedSizeInBytes;
+ }
+
+ private void ensureCapacity(int capacity)
+ {
+ if (values.length >= capacity) {
+ return;
+ }
+
+ int newSize;
+ if (initialized) {
+ newSize = BlockUtil.calculateNewArraySize(values.length);
+ }
+ else {
+ newSize = initialEntryCount;
+ initialized = true;
+ }
+ newSize = Math.max(newSize, capacity);
+
+ valueIsNull = Arrays.copyOf(valueIsNull, newSize);
+ values = Arrays.copyOf(values, newSize);
+ updateDataSize();
+ }
+
+ private void updateDataSize()
+ {
+ retainedSizeInBytes = INSTANCE_SIZE + sizeOf(valueIsNull) + sizeOf(values);
+ if (blockBuilderStatus != null) {
+ retainedSizeInBytes += BlockBuilderStatus.INSTANCE_SIZE;
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/LongPositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/LongPositionsAppender.java
new file mode 100644
index 000000000000..4ed583a0f1c5
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/LongPositionsAppender.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilderStatus;
+import io.trino.spi.block.BlockUtil;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.LongArrayBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.Optional;
+
+import static io.airlift.slice.SizeOf.sizeOf;
+import static io.trino.spi.block.BlockUtil.calculateBlockResetSize;
+import static java.lang.Math.max;
+
+public class LongPositionsAppender
+ implements BlockTypeAwarePositionsAppender
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(LongPositionsAppender.class).instanceSize();
+ private static final Block NULL_VALUE_BLOCK = new LongArrayBlock(1, Optional.of(new boolean[] {true}), new long[1]);
+
+ @Nullable
+ private final BlockBuilderStatus blockBuilderStatus;
+ private boolean initialized;
+ private final int initialEntryCount;
+
+ private int positionCount;
+ private boolean hasNullValue;
+ private boolean hasNonNullValue;
+
+ // it is assumed that these arrays are the same length
+ private boolean[] valueIsNull = new boolean[0];
+ private long[] values = new long[0];
+
+ private long retainedSizeInBytes;
+
+ public LongPositionsAppender(@Nullable BlockBuilderStatus blockBuilderStatus, int expectedEntries)
+ {
+ this.blockBuilderStatus = blockBuilderStatus;
+ this.initialEntryCount = max(expectedEntries, 1);
+
+ updateDataSize();
+ }
+
+ @Override
+ public void append(IntArrayList positions, Block block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ int positionIndex = positionCount + i;
+ boolean isNull = block.isNull(position);
+ if (isNull) {
+ valueIsNull[positionIndex] = true;
+ hasNullValue = true;
+ }
+ else {
+ values[positionIndex] = block.getLong(position, 0);
+ hasNonNullValue = true;
+ }
+ }
+ this.positionCount += newPositionCount;
+ }
+ else {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ values[positionCount + i] = block.getLong(position, 0);
+ }
+ positionCount += newPositionCount;
+ this.hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(LongArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendDictionary(IntArrayList positions, DictionaryBlock block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ if (block.isNull(position)) {
+ valueIsNull[positionCount] = true;
+ hasNullValue = true;
+ }
+ else {
+ values[positionCount] = block.getLong(position, 0);
+ hasNonNullValue = true;
+ }
+ positionCount++;
+ }
+ }
+ else {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ values[positionCount] = block.getLong(position, 0);
+ positionCount++;
+ }
+ hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(LongArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendRle(RunLengthEncodedBlock block)
+ {
+ int rlePositionCount = block.getPositionCount();
+ int sourcePosition = 0;
+ ensureCapacity(positionCount + rlePositionCount);
+ if (block.isNull(sourcePosition)) {
+ Arrays.fill(valueIsNull, positionCount, positionCount + rlePositionCount, true);
+ hasNullValue = true;
+ }
+ else {
+ long value = block.getLong(sourcePosition, 0);
+ Arrays.fill(values, positionCount, positionCount + rlePositionCount, value);
+ hasNonNullValue = true;
+ }
+ positionCount += rlePositionCount;
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(LongArrayBlock.SIZE_IN_BYTES_PER_POSITION * rlePositionCount);
+ }
+ }
+
+ @Override
+ public void appendRow(Block source, int position)
+ {
+ if (source.isNull(position)) {
+ appendNull();
+ }
+ else {
+ writeLong(source.getLong(position, 0));
+ }
+ }
+
+ private void appendNull()
+ {
+ ensureCapacity(positionCount + 1);
+
+ valueIsNull[positionCount] = true;
+
+ hasNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(LongArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ private void writeLong(long value)
+ {
+ ensureCapacity(positionCount + 1);
+
+ values[positionCount] = value;
+
+ hasNonNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(LongArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ @Override
+ public Block build()
+ {
+ if (!hasNonNullValue) {
+ return new RunLengthEncodedBlock(NULL_VALUE_BLOCK, positionCount);
+ }
+ return new LongArrayBlock(positionCount, hasNullValue ? Optional.of(valueIsNull) : Optional.empty(), values);
+ }
+
+ @Override
+ public BlockTypeAwarePositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus)
+ {
+ return new LongPositionsAppender(blockBuilderStatus, calculateBlockResetSize(positionCount));
+ }
+
+ @Override
+ public long getRetainedSizeInBytes()
+ {
+ return retainedSizeInBytes;
+ }
+
+ private void ensureCapacity(int capacity)
+ {
+ if (values.length >= capacity) {
+ return;
+ }
+
+ int newSize;
+ if (initialized) {
+ newSize = BlockUtil.calculateNewArraySize(values.length);
+ }
+ else {
+ newSize = initialEntryCount;
+ initialized = true;
+ }
+ newSize = Math.max(newSize, capacity);
+
+ valueIsNull = Arrays.copyOf(valueIsNull, newSize);
+ values = Arrays.copyOf(values, newSize);
+ updateDataSize();
+ }
+
+ private void updateDataSize()
+ {
+ retainedSizeInBytes = INSTANCE_SIZE + sizeOf(valueIsNull) + sizeOf(values);
+ if (blockBuilderStatus != null) {
+ retainedSizeInBytes += BlockBuilderStatus.INSTANCE_SIZE;
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java b/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java
index a62072256d36..7091cf47ead9 100644
--- a/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java
+++ b/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java
@@ -24,9 +24,7 @@
import io.trino.operator.OperatorContext;
import io.trino.operator.PartitionFunction;
import io.trino.spi.Page;
-import io.trino.spi.PageBuilder;
import io.trino.spi.block.Block;
-import io.trino.spi.block.BlockBuilder;
import io.trino.spi.block.DictionaryBlock;
import io.trino.spi.block.RunLengthEncodedBlock;
import io.trino.spi.predicate.NullableValue;
@@ -65,14 +63,12 @@ public class PagePartitioner
@Nullable
private final Block[] partitionConstantBlocks; // when null, no constants are present. Only non-null elements are constants
private final PagesSerde serde;
- private final PageBuilder[] pageBuilders;
+ private final PositionsAppenderPageBuilder[] pageBuilders;
private final boolean replicatesAnyRow;
private final int nullChannel; // when >= 0, send the position to every partition if this channel is null
private final AtomicLong rowsAdded = new AtomicLong();
private final AtomicLong pagesAdded = new AtomicLong();
private final OperatorContext operatorContext;
- private final PositionsAppenderFactory positionsAppenderFactory;
- private final PositionsAppender[] positionsAppenders;
private boolean hasAnyRowBeenReplicated;
@@ -91,7 +87,7 @@ public PagePartitioner(
{
this.partitionFunction = requireNonNull(partitionFunction, "partitionFunction is null");
this.partitionChannels = Ints.toArray(requireNonNull(partitionChannels, "partitionChannels is null"));
- this.positionsAppenderFactory = requireNonNull(positionsAppenderFactory, "positionsAppenderFactory is null");
+ requireNonNull(positionsAppenderFactory, "positionsAppenderFactory is null");
Block[] partitionConstantBlocks = requireNonNull(partitionConstants, "partitionConstants is null").stream()
.map(constant -> constant.map(NullableValue::asBlock).orElse(null))
.toArray(Block[]::new);
@@ -120,11 +116,10 @@ public PagePartitioner(
int pageSize = toIntExact(min(DEFAULT_MAX_PAGE_SIZE_IN_BYTES, maxMemory.toBytes() / partitionCount));
pageSize = max(1, pageSize);
- this.pageBuilders = new PageBuilder[partitionCount];
+ this.pageBuilders = new PositionsAppenderPageBuilder[partitionCount];
for (int i = 0; i < partitionCount; i++) {
- pageBuilders[i] = PageBuilder.withMaxPageSize(pageSize, sourceTypes);
+ pageBuilders[i] = PositionsAppenderPageBuilder.withMaxPageSize(pageSize, sourceTypes, positionsAppenderFactory);
}
- positionsAppenders = new PositionsAppender[sourceTypes.size()];
}
public ListenableFuture isFull()
@@ -137,7 +132,7 @@ public long getSizeInBytes()
// We use a foreach loop instead of streams
// as it has much better performance.
long sizeInBytes = 0;
- for (PageBuilder pageBuilder : pageBuilders) {
+ for (PositionsAppenderPageBuilder pageBuilder : pageBuilders) {
sizeInBytes += pageBuilder.getSizeInBytes();
}
return sizeInBytes;
@@ -149,7 +144,7 @@ public long getSizeInBytes()
public long getRetainedSizeInBytes()
{
long sizeInBytes = 0;
- for (PageBuilder pageBuilder : pageBuilders) {
+ for (PositionsAppenderPageBuilder pageBuilder : pageBuilders) {
sizeInBytes += pageBuilder.getRetainedSizeInBytes();
}
return sizeInBytes;
@@ -194,11 +189,12 @@ public void partitionPageByRow(Page page)
return;
}
+ int channelCount = sourceTypes.length;
int position;
// Handle "any row" replication outside of the inner loop processing
if (replicatesAnyRow && !hasAnyRowBeenReplicated) {
- for (PageBuilder pageBuilder : pageBuilders) {
- appendRow(pageBuilder, page, 0);
+ for (PositionsAppenderPageBuilder pageBuilder : pageBuilders) {
+ appendRow(pageBuilder, page, 0, channelCount);
}
hasAnyRowBeenReplicated = true;
position = 1;
@@ -213,33 +209,33 @@ public void partitionPageByRow(Page page)
Block nullsBlock = page.getBlock(nullChannel);
for (; position < page.getPositionCount(); position++) {
if (nullsBlock.isNull(position)) {
- for (PageBuilder pageBuilder : pageBuilders) {
- appendRow(pageBuilder, page, position);
+ for (PositionsAppenderPageBuilder pageBuilder : pageBuilders) {
+ appendRow(pageBuilder, page, position, channelCount);
}
}
else {
int partition = partitionFunction.getPartition(partitionFunctionArgs, position);
- appendRow(pageBuilders[partition], page, position);
+ appendRow(pageBuilders[partition], page, position, channelCount);
}
}
}
else {
for (; position < page.getPositionCount(); position++) {
int partition = partitionFunction.getPartition(partitionFunctionArgs, position);
- appendRow(pageBuilders[partition], page, position);
+ appendRow(pageBuilders[partition], page, position, channelCount);
}
}
flush(false);
}
- private void appendRow(PageBuilder pageBuilder, Page page, int position)
+ private void appendRow(PositionsAppenderPageBuilder pageBuilder, Page page, int position, int channelCount)
{
pageBuilder.declarePosition();
- for (int channel = 0; channel < sourceTypes.length; channel++) {
- Type type = sourceTypes[channel];
- type.appendTo(page.getBlock(channel), position, pageBuilder.getBlockBuilder(channel));
+ for (int channel = 0; channel < channelCount; channel++) {
+ PositionsAppender target = pageBuilder.getAppender(channel);
+ target.appendRow(page.getBlock(channel), position);
}
}
@@ -247,12 +243,10 @@ public void partitionPageByColumn(Page page)
{
IntArrayList[] partitionedPositions = partitionPositions(page);
- PositionsAppender[] positionsAppenders = getAppenders(page);
-
for (int i = 0; i < partitionFunction.getPartitionCount(); i++) {
IntArrayList partitionPositions = partitionedPositions[i];
if (!partitionPositions.isEmpty()) {
- appendToOutputPartition(pageBuilders[i], page, partitionPositions, positionsAppenders);
+ appendToOutputPartition(pageBuilders[i], page, partitionPositions);
partitionPositions.clear();
}
}
@@ -260,14 +254,6 @@ public void partitionPageByColumn(Page page)
flush(false);
}
- private PositionsAppender[] getAppenders(Page page)
- {
- for (int i = 0; i < positionsAppenders.length; i++) {
- positionsAppenders[i] = positionsAppenderFactory.create(sourceTypes[i], page.getBlock(i).getClass());
- }
- return positionsAppenders;
- }
-
private IntArrayList[] partitionPositions(Page page)
{
verify(page.getPositionCount() > 0, "position count is 0");
@@ -301,14 +287,14 @@ else if (partitionFunctionArgs.getChannelCount() == 1 && isDictionaryProcessingF
return partitionPositions;
}
- private void appendToOutputPartition(PageBuilder outputPartition, Page page, IntArrayList positions, PositionsAppender[] positionsAppenders)
+ private void appendToOutputPartition(PositionsAppenderPageBuilder outputPartition, Page page, IntArrayList positions)
{
outputPartition.declarePositions(positions.size());
- for (int channel = 0; channel < positionsAppenders.length; channel++) {
+ for (int channel = 0; channel < sourceTypes.length; channel++) {
Block partitionBlock = page.getBlock(channel);
- BlockBuilder target = outputPartition.getBlockBuilder(channel);
- positionsAppenders[channel].appendTo(positions, partitionBlock, target);
+ PositionsAppender target = outputPartition.getAppender(channel);
+ target.append(positions, partitionBlock);
}
}
@@ -448,9 +434,15 @@ private IntArrayList[] partitionNullablePositions(Page page, int position, IntAr
private IntArrayList[] partitionNotNullPositions(Page page, int startingPosition, IntArrayList[] partitionPositions, IntUnaryOperator partitionFunction)
{
- for (int position = startingPosition; position < page.getPositionCount(); position++) {
+ int positionCount = page.getPositionCount();
+ int[] partitionsPerPosition = new int[positionCount];
+ for (int position = startingPosition; position < positionCount; position++) {
int partition = partitionFunction.applyAsInt(position);
- partitionPositions[partition].add(position);
+ partitionsPerPosition[position] = partition;
+ }
+
+ for (int position = startingPosition; position < positionCount; position++) {
+ partitionPositions[partitionsPerPosition[position]].add(position);
}
return partitionPositions;
@@ -481,7 +473,7 @@ public void flush(boolean force)
try (PagesSerde.PagesSerdeContext context = serde.newContext()) {
// add all full pages to output buffer
for (int partition = 0; partition < pageBuilders.length; partition++) {
- PageBuilder partitionPageBuilder = pageBuilders[partition];
+ PositionsAppenderPageBuilder partitionPageBuilder = pageBuilders[partition];
if (!partitionPageBuilder.isEmpty() && (force || partitionPageBuilder.isFull())) {
Page pagePartition = partitionPageBuilder.build();
partitionPageBuilder.reset();
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppender.java
index 0b5aa8414f8b..5c9b830bea0e 100644
--- a/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppender.java
+++ b/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppender.java
@@ -11,37 +11,27 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package io.trino.operator.output;
import io.trino.spi.block.Block;
-import io.trino.spi.block.BlockBuilder;
-import io.trino.spi.type.Type;
+import io.trino.spi.block.BlockBuilderStatus;
import it.unimi.dsi.fastutil.ints.IntArrayList;
-import static java.util.Objects.requireNonNull;
+import javax.annotation.Nullable;
public interface PositionsAppender
{
- void appendTo(IntArrayList positions, Block source, BlockBuilder target);
+ void append(IntArrayList positions, Block source);
+
+ void appendRow(Block source, int position);
- class TypedPositionsAppender
- implements PositionsAppender
- {
- private final Type type;
+ Block build();
- public TypedPositionsAppender(Type type)
- {
- this.type = requireNonNull(type, "type is null");
- }
+ /**
+ * Creates new empty {@link PositionsAppender} of the same type as {@code this}
+ * pre-sized to the current position count and size in bytes.
+ */
+ PositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus);
- @Override
- public void appendTo(IntArrayList positions, Block source, BlockBuilder target)
- {
- int[] positionArray = positions.elements();
- for (int i = 0; i < positions.size(); i++) {
- type.appendTo(source, positionArray[i], target);
- }
- }
- }
+ long getRetainedSizeInBytes();
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderFactory.java b/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderFactory.java
index 41021d98b70e..382339179516 100644
--- a/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderFactory.java
+++ b/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderFactory.java
@@ -13,341 +13,72 @@
*/
package io.trino.operator.output;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import io.airlift.bytecode.DynamicClassLoader;
-import io.trino.collect.cache.NonEvictableLoadingCache;
-import io.trino.operator.output.PositionsAppender.TypedPositionsAppender;
-import io.trino.spi.block.Block;
-import io.trino.spi.block.BlockBuilder;
+import io.trino.spi.block.BlockBuilderStatus;
import io.trino.spi.block.Int128ArrayBlock;
import io.trino.spi.block.Int96ArrayBlock;
import io.trino.spi.type.FixedWidthType;
import io.trino.spi.type.Type;
import io.trino.spi.type.VariableWidthType;
-import io.trino.sql.gen.IsolatedClass;
-import it.unimi.dsi.fastutil.ints.IntArrayList;
+import io.trino.type.BlockTypeOperators;
+import io.trino.type.BlockTypeOperators.BlockPositionEqual;
-import java.util.Objects;
-import java.util.Optional;
+import javax.annotation.Nullable;
-import static io.airlift.slice.SizeOf.SIZE_OF_LONG;
-import static io.trino.collect.cache.SafeCaches.buildNonEvictableCache;
import static java.util.Objects.requireNonNull;
-/**
- * Isolates the {@code PositionsAppender} class per type and block tuples.
- * Type specific {@code PositionsAppender} implementations manually inline {@code Type#appendTo} method inside the loop
- * to avoid virtual(mega-morphic) calls and force jit to inline the {@code Block} and {@code BlockBuilder} methods.
- * Ideally, {@code TypedPositionsAppender} could work instead of type specific {@code PositionsAppender}s,
- * but in practice jit falls back to virtual calls in some cases (e.g. {@link Block#isNull}).
- */
public class PositionsAppenderFactory
{
- private final NonEvictableLoadingCache cache;
+ private final BlockTypeOperators blockTypeOperators;
- public PositionsAppenderFactory()
+ public PositionsAppenderFactory(BlockTypeOperators blockTypeOperators)
{
- this.cache = buildNonEvictableCache(
- CacheBuilder.newBuilder().maximumSize(1000),
- CacheLoader.from(key -> createAppender(key.type)));
+ this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
}
- public PositionsAppender create(Type type, Class extends Block> blockClass)
+ public PositionsAppender create(Type type, @Nullable BlockBuilderStatus blockBuilderStatus, int expectedPositions)
{
- return cache.getUnchecked(new CacheKey(type, blockClass));
+ return new BlockTypeDispatchingPositionsAppender(
+ new AdaptivePositionsAppender(
+ getEqualOperator(type),
+ createDedicatedAppenderFor(type, blockBuilderStatus, expectedPositions),
+ expectedPositions));
}
- private PositionsAppender createAppender(Type type)
+ private BlockPositionEqual getEqualOperator(Type type)
{
- return Optional.ofNullable(findDedicatedAppenderClassFor(type))
- .map(this::isolateAppender)
- .orElseGet(() -> isolateTypeAppender(type));
+ if (type.isComparable()) {
+ return blockTypeOperators.getEqualOperator(type);
+ }
+ else {
+ // if type is not comparable, we are not going to be able to support different RLE values
+ return (left, leftPosition, right, rightPosition) -> false;
+ }
}
- private Class extends PositionsAppender> findDedicatedAppenderClassFor(Type type)
+ private BlockTypeAwarePositionsAppender createDedicatedAppenderFor(Type type, @Nullable BlockBuilderStatus blockBuilderStatus, int expectedPositions)
{
if (type instanceof FixedWidthType) {
switch (((FixedWidthType) type).getFixedSize()) {
case Byte.BYTES:
- return BytePositionsAppender.class;
+ return new io.trino.operator.output.BytePositionsAppender(blockBuilderStatus, expectedPositions);
case Short.BYTES:
- return SmallintPositionsAppender.class;
+ return new ShortPositionsAppender(blockBuilderStatus, expectedPositions);
case Integer.BYTES:
- return IntPositionsAppender.class;
+ return new io.trino.operator.output.IntPositionsAppender(blockBuilderStatus, expectedPositions);
case Long.BYTES:
- return LongPositionsAppender.class;
+ return new io.trino.operator.output.LongPositionsAppender(blockBuilderStatus, expectedPositions);
case Int96ArrayBlock.INT96_BYTES:
- return Int96PositionsAppender.class;
+ return new io.trino.operator.output.Int96PositionsAppender(blockBuilderStatus, expectedPositions);
case Int128ArrayBlock.INT128_BYTES:
- return Int128PositionsAppender.class;
+ return new io.trino.operator.output.Int128PositionsAppender(blockBuilderStatus, expectedPositions);
default:
// size not supported directly, fallback to the generic appender
}
}
else if (type instanceof VariableWidthType) {
- return SlicePositionsAppender.class;
- }
-
- return null;
- }
-
- private PositionsAppender isolateTypeAppender(Type type)
- {
- Class extends PositionsAppender> isolatedAppenderClass = isolateAppenderClass(TypedPositionsAppender.class);
- try {
- return isolatedAppenderClass.getConstructor(Type.class).newInstance(type);
- }
- catch (ReflectiveOperationException e) {
- throw new RuntimeException(e);
- }
- }
-
- private PositionsAppender isolateAppender(Class extends PositionsAppender> appenderClass)
- {
- Class extends PositionsAppender> isolatedAppenderClass = isolateAppenderClass(appenderClass);
- try {
- return isolatedAppenderClass.getConstructor().newInstance();
- }
- catch (ReflectiveOperationException e) {
- throw new RuntimeException(e);
- }
- }
-
- private Class extends PositionsAppender> isolateAppenderClass(Class extends PositionsAppender> appenderClass)
- {
- DynamicClassLoader dynamicClassLoader = new DynamicClassLoader(PositionsAppender.class.getClassLoader());
-
- Class extends PositionsAppender> isolatedBatchPositionsTransferClass = IsolatedClass.isolateClass(
- dynamicClassLoader,
- PositionsAppender.class,
- appenderClass);
- return isolatedBatchPositionsTransferClass;
- }
-
- public static class LongPositionsAppender
- implements PositionsAppender
- {
- @Override
- public void appendTo(IntArrayList positions, Block block, BlockBuilder blockBuilder)
- {
- int[] positionArray = positions.elements();
- if (block.mayHaveNull()) {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- if (block.isNull(position)) {
- blockBuilder.appendNull();
- }
- else {
- blockBuilder.writeLong(block.getLong(position, 0)).closeEntry();
- }
- }
- }
- else {
- for (int i = 0; i < positions.size(); i++) {
- blockBuilder.writeLong(block.getLong(positionArray[i], 0)).closeEntry();
- }
- }
- }
- }
-
- public static class IntPositionsAppender
- implements PositionsAppender
- {
- @Override
- public void appendTo(IntArrayList positions, Block block, BlockBuilder blockBuilder)
- {
- int[] positionArray = positions.elements();
- if (block.mayHaveNull()) {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- if (block.isNull(position)) {
- blockBuilder.appendNull();
- }
- else {
- blockBuilder.writeInt(block.getInt(position, 0)).closeEntry();
- }
- }
- }
- else {
- for (int i = 0; i < positions.size(); i++) {
- blockBuilder.writeInt(block.getInt(positionArray[i], 0)).closeEntry();
- }
- }
- }
- }
-
- public static class BytePositionsAppender
- implements PositionsAppender
- {
- @Override
- public void appendTo(IntArrayList positions, Block block, BlockBuilder blockBuilder)
- {
- int[] positionArray = positions.elements();
- if (block.mayHaveNull()) {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- if (block.isNull(position)) {
- blockBuilder.appendNull();
- }
- else {
- blockBuilder.writeByte(block.getByte(position, 0)).closeEntry();
- }
- }
- }
- else {
- for (int i = 0; i < positions.size(); i++) {
- blockBuilder.writeByte(block.getByte(positionArray[i], 0)).closeEntry();
- }
- }
- }
- }
-
- public static class SlicePositionsAppender
- implements PositionsAppender
- {
- @Override
- public void appendTo(IntArrayList positions, Block block, BlockBuilder blockBuilder)
- {
- int[] positionArray = positions.elements();
- if (block.mayHaveNull()) {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- if (block.isNull(position)) {
- blockBuilder.appendNull();
- }
- else {
- block.writeBytesTo(position, 0, block.getSliceLength(position), blockBuilder);
- blockBuilder.closeEntry();
- }
- }
- }
- else {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- block.writeBytesTo(position, 0, block.getSliceLength(position), blockBuilder);
- blockBuilder.closeEntry();
- }
- }
- }
- }
-
- public static class SmallintPositionsAppender
- implements PositionsAppender
- {
- @Override
- public void appendTo(IntArrayList positions, Block block, BlockBuilder blockBuilder)
- {
- int[] positionArray = positions.elements();
- if (block.mayHaveNull()) {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- if (block.isNull(position)) {
- blockBuilder.appendNull();
- }
- else {
- blockBuilder.writeShort(block.getShort(position, 0)).closeEntry();
- }
- }
- }
- else {
- for (int i = 0; i < positions.size(); i++) {
- blockBuilder.writeShort(block.getShort(positionArray[i], 0)).closeEntry();
- }
- }
+ return new io.trino.operator.output.SlicePositionsAppender(blockBuilderStatus, expectedPositions);
}
- }
- public static class Int96PositionsAppender
- implements PositionsAppender
- {
- @Override
- public void appendTo(IntArrayList positions, Block block, BlockBuilder blockBuilder)
- {
- int[] positionArray = positions.elements();
- if (block.mayHaveNull()) {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- if (block.isNull(position)) {
- blockBuilder.appendNull();
- }
- else {
- blockBuilder.writeLong(block.getLong(position, 0));
- blockBuilder.writeInt(block.getInt(position, SIZE_OF_LONG));
- blockBuilder.closeEntry();
- }
- }
- }
- else {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- blockBuilder.writeLong(block.getLong(position, 0));
- blockBuilder.writeInt(block.getInt(position, SIZE_OF_LONG));
- blockBuilder.closeEntry();
- }
- }
- }
- }
-
- public static class Int128PositionsAppender
- implements PositionsAppender
- {
- @Override
- public void appendTo(IntArrayList positions, Block block, BlockBuilder blockBuilder)
- {
- int[] positionArray = positions.elements();
- if (block.mayHaveNull()) {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- if (block.isNull(position)) {
- blockBuilder.appendNull();
- }
- else {
- blockBuilder.writeLong(block.getLong(position, 0));
- blockBuilder.writeLong(block.getLong(position, SIZE_OF_LONG));
- blockBuilder.closeEntry();
- }
- }
- }
- else {
- for (int i = 0; i < positions.size(); i++) {
- int position = positionArray[i];
- blockBuilder.writeLong(block.getLong(position, 0));
- blockBuilder.writeLong(block.getLong(position, SIZE_OF_LONG));
- blockBuilder.closeEntry();
- }
- }
- }
- }
-
- private static class CacheKey
- {
- private final Type type;
- private final Class extends Block> blockClass;
-
- private CacheKey(Type type, Class extends Block> blockClass)
- {
- this.type = requireNonNull(type, "type is null");
- this.blockClass = requireNonNull(blockClass, "blockClass is null");
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- CacheKey cacheKey = (CacheKey) o;
- return type.equals(cacheKey.type) && blockClass.equals(cacheKey.blockClass);
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(type, blockClass);
- }
+ return new TypedPositionsAppender(type, blockBuilderStatus, expectedPositions);
}
}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderPageBuilder.java b/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderPageBuilder.java
new file mode 100644
index 000000000000..1018a5a28e13
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderPageBuilder.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.Page;
+import io.trino.spi.block.Block;
+import io.trino.spi.block.PageBuilderStatus;
+import io.trino.spi.type.Type;
+
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkState;
+import static io.trino.spi.PageBuilder.DEFAULT_INITIAL_EXPECTED_ENTRIES;
+import static java.util.Objects.requireNonNull;
+
+public class PositionsAppenderPageBuilder
+{
+ private final PositionsAppender[] positionsAppenders;
+ private final List extends Type> types;
+ private final PositionsAppenderFactory positionsAppenderFactory;
+ private PageBuilderStatus pageBuilderStatus;
+ private int declaredPositions;
+
+ private PositionsAppenderPageBuilder(
+ int initialExpectedEntries,
+ int maxPageBytes,
+ List extends Type> types,
+ PositionsAppenderFactory positionsAppenderFactory)
+ {
+ this.types = requireNonNull(types, "types is null");
+ this.positionsAppenderFactory = requireNonNull(positionsAppenderFactory, "positionsAppenderFactory is null");
+
+ pageBuilderStatus = new PageBuilderStatus(maxPageBytes);
+ positionsAppenders = new PositionsAppender[types.size()];
+ createAppenders(initialExpectedEntries);
+ }
+
+ public static PositionsAppenderPageBuilder withMaxPageSize(int maxPageBytes, List sourceTypes, PositionsAppenderFactory positionsAppenderFactory)
+ {
+ return new PositionsAppenderPageBuilder(DEFAULT_INITIAL_EXPECTED_ENTRIES, maxPageBytes, sourceTypes, positionsAppenderFactory);
+ }
+
+ public long getSizeInBytes()
+ {
+ return pageBuilderStatus.getSizeInBytes();
+ }
+
+ public long getRetainedSizeInBytes()
+ {
+ // We use a foreach loop instead of streams
+ // as it has much better performance.
+ long retainedSizeInBytes = 0;
+ for (PositionsAppender positionsAppender : positionsAppenders) {
+ retainedSizeInBytes += positionsAppender.getRetainedSizeInBytes();
+ }
+ return retainedSizeInBytes;
+ }
+
+ public void declarePosition()
+ {
+ declaredPositions++;
+ }
+
+ public void declarePositions(int positions)
+ {
+ declaredPositions += positions;
+ }
+
+ public boolean isFull()
+ {
+ return declaredPositions == Integer.MAX_VALUE || pageBuilderStatus.isFull();
+ }
+
+ public boolean isEmpty()
+ {
+ return declaredPositions == 0;
+ }
+
+ public PositionsAppender getAppender(int channel)
+ {
+ return positionsAppenders[channel];
+ }
+
+ public Page build()
+ {
+ if (positionsAppenders.length == 0) {
+ return new Page(declaredPositions);
+ }
+
+ Block[] blocks = new Block[positionsAppenders.length];
+ for (int i = 0; i < blocks.length; i++) {
+ blocks[i] = positionsAppenders[i].build();
+ checkState(blocks[i].getPositionCount() == declaredPositions, "Declared positions (%s) does not match block %s's number of entries (%s)", declaredPositions, i, blocks[i].getPositionCount());
+ }
+
+ return Page.wrapBlocksWithoutCopy(declaredPositions, blocks);
+ }
+
+ public void reset()
+ {
+ if (isEmpty()) {
+ return;
+ }
+ pageBuilderStatus = new PageBuilderStatus(pageBuilderStatus.getMaxPageSizeInBytes());
+ for (int i = 0; i < positionsAppenders.length; i++) {
+ positionsAppenders[i] = positionsAppenders[i].newStateLike(pageBuilderStatus.createBlockBuilderStatus());
+ }
+
+ declaredPositions = 0;
+ }
+
+ private void createAppenders(int expectedPositions)
+ {
+ for (int i = 0; i < positionsAppenders.length; i++) {
+ positionsAppenders[i] = positionsAppenderFactory.create(types.get(i), pageBuilderStatus.createBlockBuilderStatus(), expectedPositions);
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/ShortPositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/ShortPositionsAppender.java
new file mode 100644
index 000000000000..01e1bad531c1
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/ShortPositionsAppender.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilderStatus;
+import io.trino.spi.block.BlockUtil;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import io.trino.spi.block.ShortArrayBlock;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.Optional;
+
+import static io.airlift.slice.SizeOf.sizeOf;
+import static io.trino.spi.block.BlockUtil.calculateBlockResetSize;
+import static java.lang.Math.max;
+
+public class ShortPositionsAppender
+ implements BlockTypeAwarePositionsAppender
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(ShortPositionsAppender.class).instanceSize();
+ private static final Block NULL_VALUE_BLOCK = new ShortArrayBlock(1, Optional.of(new boolean[] {true}), new short[1]);
+
+ @Nullable
+ private final BlockBuilderStatus blockBuilderStatus;
+ private boolean initialized;
+ private final int initialEntryCount;
+
+ private int positionCount;
+ private boolean hasNullValue;
+ private boolean hasNonNullValue;
+
+ // it is assumed that these arrays are the same length
+ private boolean[] valueIsNull = new boolean[0];
+ private short[] values = new short[0];
+
+ private long retainedSizeInBytes;
+
+ public ShortPositionsAppender(@Nullable BlockBuilderStatus blockBuilderStatus, int expectedEntries)
+ {
+ this.blockBuilderStatus = blockBuilderStatus;
+ this.initialEntryCount = max(expectedEntries, 1);
+
+ updateDataSize();
+ }
+
+ @Override
+ public void append(IntArrayList positions, Block block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ boolean isNull = block.isNull(position);
+ int positionIndex = positionCount + i;
+ if (isNull) {
+ valueIsNull[positionIndex] = true;
+ hasNullValue = true;
+ }
+ else {
+ values[positionIndex] = block.getShort(position, 0);
+ hasNonNullValue = true;
+ }
+ }
+ this.positionCount += newPositionCount;
+ }
+ else {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ values[positionCount + i] = block.getShort(position, 0);
+ }
+ positionCount += newPositionCount;
+ this.hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(ShortArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendDictionary(IntArrayList positions, DictionaryBlock block)
+ {
+ int[] positionArray = positions.elements();
+ int newPositionCount = positions.size();
+ ensureCapacity(positionCount + newPositionCount);
+ if (block.mayHaveNull()) {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ if (block.isNull(position)) {
+ valueIsNull[positionCount] = true;
+ hasNullValue = true;
+ }
+ else {
+ values[positionCount] = block.getShort(position, 0);
+ hasNonNullValue = true;
+ }
+ positionCount++;
+ }
+ }
+ else {
+ for (int i = 0; i < newPositionCount; i++) {
+ int position = positionArray[i];
+ values[positionCount] = block.getShort(position, 0);
+ positionCount++;
+ }
+ hasNonNullValue = true;
+ }
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(ShortArrayBlock.SIZE_IN_BYTES_PER_POSITION * newPositionCount);
+ }
+ }
+
+ @Override
+ public void appendRle(RunLengthEncodedBlock block)
+ {
+ int rlePositionCount = block.getPositionCount();
+ int sourcePosition = 0;
+ ensureCapacity(positionCount + rlePositionCount);
+ if (block.isNull(sourcePosition)) {
+ Arrays.fill(valueIsNull, positionCount, positionCount + rlePositionCount, true);
+ hasNullValue = true;
+ }
+ else {
+ short value = block.getShort(sourcePosition, 0);
+ Arrays.fill(values, positionCount, positionCount + rlePositionCount, value);
+ hasNonNullValue = true;
+ }
+ positionCount += rlePositionCount;
+
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(ShortArrayBlock.SIZE_IN_BYTES_PER_POSITION * rlePositionCount);
+ }
+ }
+
+ @Override
+ public void appendRow(Block source, int position)
+ {
+ if (source.isNull(position)) {
+ appendNull();
+ }
+ else {
+ writeShort(source.getShort(position, 0));
+ }
+ }
+
+ private void appendNull()
+ {
+ ensureCapacity(positionCount + 1);
+
+ valueIsNull[positionCount] = true;
+
+ hasNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(ShortArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ private void writeShort(short value)
+ {
+ ensureCapacity(positionCount + 1);
+
+ values[positionCount] = value;
+
+ hasNonNullValue = true;
+ positionCount++;
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes(ShortArrayBlock.SIZE_IN_BYTES_PER_POSITION);
+ }
+ }
+
+ @Override
+ public Block build()
+ {
+ if (!hasNonNullValue) {
+ return new RunLengthEncodedBlock(NULL_VALUE_BLOCK, positionCount);
+ }
+ return new ShortArrayBlock(positionCount, hasNullValue ? Optional.of(valueIsNull) : Optional.empty(), values);
+ }
+
+ @Override
+ public BlockTypeAwarePositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus)
+ {
+ return new ShortPositionsAppender(blockBuilderStatus, calculateBlockResetSize(positionCount));
+ }
+
+ @Override
+ public long getRetainedSizeInBytes()
+ {
+ return retainedSizeInBytes;
+ }
+
+ private void ensureCapacity(int capacity)
+ {
+ if (values.length >= capacity) {
+ return;
+ }
+
+ int newSize;
+ if (initialized) {
+ newSize = BlockUtil.calculateNewArraySize(values.length);
+ }
+ else {
+ newSize = initialEntryCount;
+ initialized = true;
+ }
+ newSize = Math.max(newSize, capacity);
+
+ valueIsNull = Arrays.copyOf(valueIsNull, newSize);
+ values = Arrays.copyOf(values, newSize);
+ updateDataSize();
+ }
+
+ private void updateDataSize()
+ {
+ retainedSizeInBytes = INSTANCE_SIZE + sizeOf(valueIsNull) + sizeOf(values);
+ if (blockBuilderStatus != null) {
+ retainedSizeInBytes += BlockBuilderStatus.INSTANCE_SIZE;
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/SlicePositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/SlicePositionsAppender.java
new file mode 100644
index 000000000000..b2ab375bb00d
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/SlicePositionsAppender.java
@@ -0,0 +1,417 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.airlift.slice.Slice;
+import io.airlift.slice.Slices;
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilder;
+import io.trino.spi.block.BlockBuilderStatus;
+import io.trino.spi.block.BlockUtil;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.PageBuilderStatus;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import io.trino.spi.block.VariableWidthBlock;
+import io.trino.spi.block.VariableWidthBlockBuilder;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.Optional;
+
+import static io.airlift.slice.SizeOf.SIZE_OF_BYTE;
+import static io.airlift.slice.SizeOf.SIZE_OF_INT;
+import static io.airlift.slice.SizeOf.sizeOf;
+import static io.airlift.slice.Slices.EMPTY_SLICE;
+import static io.trino.spi.block.BlockUtil.MAX_ARRAY_SIZE;
+import static io.trino.spi.block.BlockUtil.calculateBlockResetBytes;
+import static io.trino.spi.block.BlockUtil.calculateBlockResetSize;
+import static io.trino.spi.type.AbstractVariableWidthType.EXPECTED_BYTES_PER_ENTRY;
+import static java.lang.Math.min;
+import static sun.misc.Unsafe.ARRAY_BYTE_BASE_OFFSET;
+
+public class SlicePositionsAppender
+ implements BlockTypeAwarePositionsAppender
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(SlicePositionsAppender.class).instanceSize();
+ private static final Block NULL_VALUE_BLOCK = new VariableWidthBlock(1, EMPTY_SLICE, new int[] {0, 0}, Optional.of(new boolean[] {true}));
+
+ @Nullable
+ private final BlockBuilderStatus blockBuilderStatus;
+
+ private boolean initialized;
+ private final int initialEntryCount;
+ private int initialBytesSize;
+
+ private byte[] bytes = new byte[0];
+ private int currentOffset;
+
+ private boolean hasNullValue;
+ private boolean hasNonNullValue;
+ // it is assumed that the offsets array is one position longer than the valueIsNull array
+ private boolean[] valueIsNull = new boolean[0];
+ private int[] offsets = new int[1];
+
+ private int positionCount;
+
+ private long arraysRetainedSizeInBytes;
+ private final BlockBuilder fakeBlockBuilder;
+
+ public SlicePositionsAppender(@Nullable BlockBuilderStatus blockBuilderStatus, int expectedPositions)
+ {
+ this(blockBuilderStatus, expectedPositions, getExpectedBytes(blockBuilderStatus, expectedPositions));
+ }
+
+ public SlicePositionsAppender(@Nullable BlockBuilderStatus blockBuilderStatus, int expectedEntries, int expectedBytes)
+ {
+ this.blockBuilderStatus = blockBuilderStatus;
+
+ initialEntryCount = expectedEntries;
+ initialBytesSize = min(expectedBytes, MAX_ARRAY_SIZE);
+
+ updateArraysDataSize();
+ fakeBlockBuilder = new VariableWidthBlockBuilder(null, 0, 0)
+ {
+ @Override
+ public BlockBuilder writeBytes(Slice source, int sourceIndex, int length)
+ {
+ SlicePositionsAppender.this.writeBytes(source, sourceIndex, length);
+ return this;
+ }
+ };
+ }
+
+ @Override
+ public void append(IntArrayList positions, Block block)
+ {
+ ensurePositionCapacity(positionCount + positions.size());
+ int[] positionArray = positions.elements();
+ int newByteCount = 0;
+ int notNullCount = 0;
+ int[] lengths = new int[positions.size()];
+ int[] offsets = new int[positions.size()];
+ VariableWidthBlock variableWidthBlock = (VariableWidthBlock) block;
+ Slice rawSlice = variableWidthBlock.getRawSlice(0);
+
+ if (block.mayHaveNull()) {
+ int[] targetOffsets = new int[positions.size() + 1];
+ targetOffsets[0] = this.offsets[positionCount];
+ for (int i = 0; i < positions.size(); i++) {
+ int position = positionArray[i];
+ if (variableWidthBlock.isNull(position)) {
+ this.offsets[positionCount + i + 1] = this.offsets[positionCount + i];
+ valueIsNull[positionCount + i] = true;
+ }
+ else {
+ int length = variableWidthBlock.getSliceLength(position);
+ int offset = variableWidthBlock.getPositionOffset(position);
+ lengths[notNullCount] = length;
+ offsets[notNullCount] = offset;
+ targetOffsets[notNullCount + 1] = targetOffsets[notNullCount] + length;
+ newByteCount += length;
+ notNullCount++;
+ this.offsets[positionCount + i + 1] = this.offsets[positionCount + i] + length;
+ }
+ }
+ int nullCount = positions.size() - notNullCount;
+ hasNullValue |= nullCount > 0;
+ hasNonNullValue |= notNullCount > 0;
+ positionCount += nullCount;
+ copyBytes(rawSlice, lengths, offsets, notNullCount, targetOffsets, 0, newByteCount);
+ }
+ else {
+ newByteCount = extractLengths(positions, positionArray, variableWidthBlock, lengths, offsets);
+ notNullCount = positions.size();
+ hasNonNullValue |= notNullCount > 0;
+ copyBytes(rawSlice, lengths, offsets, notNullCount, this.offsets, positionCount, newByteCount);
+ }
+ // update BlockBuilderStatus for null values
+ updateBlockBuilderStatus(positions.size() - notNullCount, 0);
+ }
+
+ private void copyBytes(Slice rawSlice, int[] lengths, int[] sourceOffsets, int count, int[] targetOffsets, int targetOffsetsIndex, int newByteCount)
+ {
+ ensureBytesCapacity(currentOffset + newByteCount);
+ Object base = rawSlice.getBase();
+
+ if (base instanceof byte[] && rawSlice.getAddress() == ARRAY_BYTE_BASE_OFFSET) {
+ for (int i = 0; i < count; i++) {
+ int length = lengths[i];
+ int offset = sourceOffsets[i];
+ System.arraycopy(base, offset, bytes, targetOffsets[targetOffsetsIndex + i], length);
+ }
+ }
+ else {
+ for (int i = 0; i < count; i++) {
+ int length = lengths[i];
+ int offset = sourceOffsets[i];
+ rawSlice.getBytes(offset, bytes, targetOffsets[targetOffsetsIndex + i], length);
+ }
+ }
+ positionCount += count;
+ currentOffset += newByteCount;
+ updateBlockBuilderStatus(count, newByteCount);
+ }
+
+ private int extractLengths(IntArrayList positions, int[] positionArray, VariableWidthBlock variableWidthBlock, int[] lengths, int[] offsets)
+ {
+ int newByteCount = 0;
+ for (int i = 0; i < positions.size(); i++) {
+ int position = positionArray[i];
+ int length = variableWidthBlock.getSliceLength(position);
+ int offset = variableWidthBlock.getPositionOffset(position);
+ lengths[i] = length;
+ offsets[i] = offset;
+ newByteCount += length;
+ this.offsets[positionCount + i + 1] = this.offsets[positionCount + i] + length;
+ }
+ return newByteCount;
+ }
+
+ @Override
+ public void appendDictionary(IntArrayList positions, DictionaryBlock block)
+ {
+ ensurePositionCapacity(positionCount + positions.size());
+ int[] positionArray = positions.elements();
+ int newByteCount = 0;
+ int notNullCount = 0;
+ int[] lengths = new int[positions.size()];
+ int[] offsets = new int[positions.size()];
+ VariableWidthBlock variableWidthBlock = (VariableWidthBlock) block.getDictionary();
+ Slice rawSlice = variableWidthBlock.getRawSlice(0);
+
+ if (block.mayHaveNull()) {
+ int[] targetOffsets = new int[positions.size() + 1];
+ targetOffsets[0] = this.offsets[positionCount];
+ for (int i = 0; i < positions.size(); i++) {
+ int position = block.getId(positionArray[i]);
+ if (variableWidthBlock.isNull(position)) {
+ this.offsets[positionCount + i + 1] = this.offsets[positionCount + i];
+ valueIsNull[positionCount + i] = true;
+ }
+ else {
+ int length = variableWidthBlock.getSliceLength(position);
+ int offset = variableWidthBlock.getPositionOffset(position);
+ lengths[notNullCount] = length;
+ offsets[notNullCount] = offset;
+ targetOffsets[notNullCount + 1] = targetOffsets[notNullCount] + length;
+ newByteCount += length;
+ notNullCount++;
+ this.offsets[positionCount + i + 1] = this.offsets[positionCount + i] + length;
+ }
+ }
+ int nullCount = positions.size() - notNullCount;
+ hasNullValue |= nullCount > 0;
+ hasNonNullValue |= notNullCount > 0;
+ positionCount += nullCount;
+ copyBytes(rawSlice, lengths, offsets, notNullCount, targetOffsets, 0, newByteCount);
+ }
+ else {
+ for (int i = 0; i < positions.size(); i++) {
+ int position = block.getId(positionArray[i]);
+ int length = variableWidthBlock.getSliceLength(position);
+ int offset = variableWidthBlock.getPositionOffset(position);
+ lengths[i] = length;
+ offsets[i] = offset;
+ newByteCount += length;
+ this.offsets[positionCount + i + 1] = this.offsets[positionCount + i] + length;
+ }
+ notNullCount = positions.size();
+ hasNonNullValue |= notNullCount > 0;
+ copyBytes(rawSlice, lengths, offsets, notNullCount, this.offsets, positionCount, newByteCount);
+ }
+ // update BlockBuilderStatus for null values
+ updateBlockBuilderStatus(positions.size() - notNullCount, 0);
+ }
+
+ @Override
+ public void appendRle(RunLengthEncodedBlock block)
+ {
+ int rlePositionCount = block.getPositionCount();
+ int sourcePosition = 0;
+ ensurePositionCapacity(positionCount + rlePositionCount);
+ if (block.isNull(sourcePosition)) {
+ int offset = this.offsets[positionCount];
+ Arrays.fill(valueIsNull, positionCount, positionCount + rlePositionCount, true);
+ Arrays.fill(offsets, positionCount + 1, positionCount + rlePositionCount + 1, offset);
+ positionCount += rlePositionCount;
+
+ hasNullValue = true;
+ updateBlockBuilderStatus(rlePositionCount, 0);
+ }
+ else {
+ VariableWidthBlock variableWidthBlock = (VariableWidthBlock) block.getValue();
+ Slice rawSlice = variableWidthBlock.getRawSlice(0);
+
+ int length = variableWidthBlock.getSliceLength(sourcePosition);
+ int offset = variableWidthBlock.getPositionOffset(sourcePosition);
+
+ int startOffset = offsets[positionCount];
+ hasNonNullValue = true;
+ duplicateBytes(rawSlice, offset, length, rlePositionCount, startOffset);
+ }
+ }
+
+ /**
+ * Copy {@code length} bytes from {@code rawSlice}, starting at offset {@code sourceOffset} to {@code count} consecutive positions in the {@link #bytes} array.
+ */
+ private void duplicateBytes(Slice rawSlice, int sourceOffset, int length, int count, int startOffset)
+ {
+ int newByteCount = count * length;
+ ensureBytesCapacity(currentOffset + newByteCount);
+ Object base = rawSlice.getBase();
+
+ if (base instanceof byte[] && rawSlice.getAddress() == ARRAY_BYTE_BASE_OFFSET) {
+ for (int i = 0; i < count; i++) {
+ System.arraycopy(base, sourceOffset, bytes, startOffset + i * length, length);
+ this.offsets[positionCount + i + 1] = startOffset + ((i + 1) * length);
+ }
+ }
+ else {
+ for (int i = 0; i < count; i++) {
+ rawSlice.getBytes(sourceOffset, bytes, startOffset + i * length, length);
+ this.offsets[positionCount + i + 1] = startOffset + ((i + 1) * length);
+ }
+ }
+ positionCount += count;
+ currentOffset += newByteCount;
+ updateBlockBuilderStatus(count, newByteCount);
+ }
+
+ @Override
+ public void appendRow(Block source, int position)
+ {
+ if (source.isNull(position)) {
+ appendNull();
+ }
+ else {
+ source.writeBytesTo(position, 0, source.getSliceLength(position), fakeBlockBuilder);
+ }
+ }
+
+ private void appendNull()
+ {
+ ensurePositionCapacity(positionCount + 1);
+ hasNullValue = true;
+ entryAdded(0, true);
+ }
+
+ private void writeBytes(Slice source, int sourceIndex, int length)
+ {
+ ensureCapacity(positionCount + 1, currentOffset + length);
+ source.getBytes(sourceIndex, bytes, currentOffset, length);
+ hasNonNullValue = true;
+ entryAdded(length, false);
+ }
+
+ private void entryAdded(int bytesWritten, boolean isNull)
+ {
+ valueIsNull[positionCount] = isNull;
+ offsets[positionCount + 1] = currentOffset + bytesWritten;
+ currentOffset += bytesWritten;
+ positionCount++;
+
+ updateBlockBuilderStatus(1, bytesWritten);
+ }
+
+ @Override
+ public Block build()
+ {
+ if (!hasNonNullValue) {
+ return new RunLengthEncodedBlock(NULL_VALUE_BLOCK, positionCount);
+ }
+ return new VariableWidthBlock(positionCount, Slices.wrappedBuffer(bytes, 0, currentOffset), offsets, hasNullValue ? Optional.of(valueIsNull) : Optional.empty());
+ }
+
+ @Override
+ public BlockTypeAwarePositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus)
+ {
+ int currentSizeInBytes = positionCount == 0 ? positionCount : (offsets[positionCount] - offsets[0]);
+ return new SlicePositionsAppender(blockBuilderStatus, calculateBlockResetSize(positionCount), calculateBlockResetBytes(currentSizeInBytes));
+ }
+
+ private static int getExpectedBytes(BlockBuilderStatus blockBuilderStatus, int expectedPositions)
+ {
+ int maxBlockSizeInBytes;
+ if (blockBuilderStatus == null) {
+ maxBlockSizeInBytes = PageBuilderStatus.DEFAULT_MAX_PAGE_SIZE_IN_BYTES;
+ }
+ else {
+ maxBlockSizeInBytes = blockBuilderStatus.getMaxPageSizeInBytes();
+ }
+
+ // it is guaranteed Math.min will not overflow; safe to cast
+ return (int) min((long) expectedPositions * EXPECTED_BYTES_PER_ENTRY, maxBlockSizeInBytes);
+ }
+
+ private void updateBlockBuilderStatus(int count, int bytesWritten)
+ {
+ if (blockBuilderStatus != null) {
+ blockBuilderStatus.addBytes((SIZE_OF_BYTE + SIZE_OF_INT) * count + bytesWritten);
+ }
+ }
+
+ @Override
+ public long getRetainedSizeInBytes()
+ {
+ long size = INSTANCE_SIZE + arraysRetainedSizeInBytes;
+ if (blockBuilderStatus != null) {
+ size += BlockBuilderStatus.INSTANCE_SIZE;
+ }
+ return size;
+ }
+
+ private void ensureCapacity(int capacity, int bytesCapacity)
+ {
+ ensurePositionCapacity(capacity);
+ ensureBytesCapacity(bytesCapacity);
+ }
+
+ private void ensureBytesCapacity(int bytesCapacity)
+ {
+ if (bytes.length < bytesCapacity) {
+ int newBytesLength = Math.max(bytes.length, initialBytesSize);
+ if (bytesCapacity > newBytesLength) {
+ newBytesLength = Math.max(bytesCapacity, BlockUtil.calculateNewArraySize(newBytesLength));
+ }
+ bytes = Arrays.copyOf(bytes, newBytesLength);
+ }
+ }
+
+ private void ensurePositionCapacity(int capacity)
+ {
+ if (valueIsNull.length < capacity) {
+ int newSize;
+ if (initialized) {
+ newSize = BlockUtil.calculateNewArraySize(valueIsNull.length);
+ }
+ else {
+ newSize = initialEntryCount;
+ initialized = true;
+ }
+ newSize = Math.max(newSize, capacity);
+
+ valueIsNull = Arrays.copyOf(valueIsNull, newSize);
+ offsets = Arrays.copyOf(offsets, newSize + 1);
+ updateArraysDataSize();
+ }
+ }
+
+ private void updateArraysDataSize()
+ {
+ arraysRetainedSizeInBytes = sizeOf(valueIsNull) + sizeOf(offsets) + sizeOf(bytes);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/output/TypedPositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/TypedPositionsAppender.java
new file mode 100644
index 000000000000..4af26526339c
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/operator/output/TypedPositionsAppender.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.operator.output;
+
+import io.trino.spi.block.Block;
+import io.trino.spi.block.BlockBuilder;
+import io.trino.spi.block.BlockBuilderStatus;
+import io.trino.spi.block.DictionaryBlock;
+import io.trino.spi.block.RunLengthEncodedBlock;
+import io.trino.spi.type.Type;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import javax.annotation.Nullable;
+
+import static java.util.Objects.requireNonNull;
+
+public class TypedPositionsAppender
+ implements BlockTypeAwarePositionsAppender
+{
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(TypedPositionsAppender.class).instanceSize();
+
+ protected final Type type;
+ protected final BlockBuilder blockBuilder;
+
+ TypedPositionsAppender(Type type, @Nullable BlockBuilderStatus blockBuilderStatus, int expectedPositions)
+ {
+ this(
+ requireNonNull(type, "type is null"),
+ type.createBlockBuilder(blockBuilderStatus, expectedPositions));
+ }
+
+ TypedPositionsAppender(Type type, BlockBuilder blockBuilder)
+ {
+ this.type = requireNonNull(type, "type is null");
+ this.blockBuilder = requireNonNull(blockBuilder, "blockBuilder is null");
+ }
+
+ @Override
+ public void append(IntArrayList positions, Block source)
+ {
+ int[] positionArray = positions.elements();
+ for (int i = 0; i < positions.size(); i++) {
+ type.appendTo(source, positionArray[i], blockBuilder);
+ }
+ }
+
+ @Override
+ public void appendRle(RunLengthEncodedBlock block)
+ {
+ for (int i = 0; i < block.getPositionCount(); i++) {
+ type.appendTo(block, 0, blockBuilder);
+ }
+ }
+
+ @Override
+ public void appendDictionary(IntArrayList positions, DictionaryBlock source)
+ {
+ int[] positionArray = positions.elements();
+ for (int i = 0; i < positions.size(); i++) {
+ type.appendTo(source, positionArray[i], blockBuilder);
+ }
+ }
+
+ @Override
+ public void appendRow(Block source, int position)
+ {
+ type.appendTo(source, position, blockBuilder);
+ }
+
+ @Override
+ public Block build()
+ {
+ return blockBuilder.build();
+ }
+
+ @Override
+ public BlockTypeAwarePositionsAppender newStateLike(@Nullable BlockBuilderStatus blockBuilderStatus)
+ {
+ return new TypedPositionsAppender(type, blockBuilder.newBlockBuilderLike(blockBuilderStatus));
+ }
+
+ @Override
+ public long getRetainedSizeInBytes()
+ {
+ return INSTANCE_SIZE + blockBuilder.getRetainedSizeInBytes();
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java
index 17d7c7d773b2..1a3981ced504 100644
--- a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java
+++ b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java
@@ -94,11 +94,13 @@
import io.trino.operator.DirectExchangeClientFactory;
import io.trino.operator.DirectExchangeClientSupplier;
import io.trino.operator.ForExchange;
+import io.trino.operator.GroupByHashFactory;
import io.trino.operator.GroupByHashPageIndexerFactory;
import io.trino.operator.OperatorFactories;
import io.trino.operator.PagesIndex;
import io.trino.operator.PagesIndexPageSorter;
import io.trino.operator.TrinoOperatorFactories;
+import io.trino.operator.hash.IsolatedHashTableFactory;
import io.trino.operator.index.IndexJoinLookupStats;
import io.trino.server.ExpressionSerialization.ExpressionDeserializer;
import io.trino.server.ExpressionSerialization.ExpressionSerializer;
@@ -333,6 +335,8 @@ protected void setup(Binder binder)
binder.bind(JoinFilterFunctionCompiler.class).in(Scopes.SINGLETON);
newExporter(binder).export(JoinFilterFunctionCompiler.class).withGeneratedName();
binder.bind(JoinCompiler.class).in(Scopes.SINGLETON);
+ binder.bind(IsolatedHashTableFactory.class).in(Scopes.SINGLETON);
+ binder.bind(GroupByHashFactory.class).in(Scopes.SINGLETON);
newExporter(binder).export(JoinCompiler.class).withGeneratedName();
binder.bind(OrderingCompiler.class).in(Scopes.SINGLETON);
newExporter(binder).export(OrderingCompiler.class).withGeneratedName();
diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java
index c9cec05ec5af..4b50dd723f80 100644
--- a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java
+++ b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java
@@ -62,6 +62,7 @@
import io.trino.operator.ExchangeOperator.ExchangeOperatorFactory;
import io.trino.operator.ExplainAnalyzeOperator.ExplainAnalyzeOperatorFactory;
import io.trino.operator.FilterAndProjectOperator;
+import io.trino.operator.GroupByHashFactory;
import io.trino.operator.GroupIdOperator;
import io.trino.operator.HashAggregationOperator.HashAggregationOperatorFactory;
import io.trino.operator.HashSemiJoinOperator;
@@ -291,6 +292,7 @@
import static io.trino.SystemSessionProperties.getAggregationOperatorUnspillMemoryLimit;
import static io.trino.SystemSessionProperties.getFilterAndProjectMinOutputPageRowCount;
import static io.trino.SystemSessionProperties.getFilterAndProjectMinOutputPageSize;
+import static io.trino.SystemSessionProperties.getMaxPartialAggregationMemoryUsage;
import static io.trino.SystemSessionProperties.getTaskConcurrency;
import static io.trino.SystemSessionProperties.getTaskWriterCount;
import static io.trino.SystemSessionProperties.isAdaptivePartialAggregationEnabled;
@@ -394,13 +396,14 @@ public class LocalExecutionPlanner
private final PartitioningSpillerFactory partitioningSpillerFactory;
private final PagesIndex.Factory pagesIndexFactory;
private final JoinCompiler joinCompiler;
+ private final GroupByHashFactory groupByHashFactory;
private final OperatorFactories operatorFactories;
private final OrderingCompiler orderingCompiler;
private final DynamicFilterConfig dynamicFilterConfig;
private final BlockTypeOperators blockTypeOperators;
private final TableExecuteContextManager tableExecuteContextManager;
private final ExchangeManagerRegistry exchangeManagerRegistry;
- private final PositionsAppenderFactory positionsAppenderFactory = new PositionsAppenderFactory();
+ private final PositionsAppenderFactory positionsAppenderFactory;
private final NonEvictableCache accumulatorFactoryCache = buildNonEvictableCache(CacheBuilder.newBuilder()
.maximumSize(1000)
@@ -429,6 +432,7 @@ public LocalExecutionPlanner(
PartitioningSpillerFactory partitioningSpillerFactory,
PagesIndex.Factory pagesIndexFactory,
JoinCompiler joinCompiler,
+ GroupByHashFactory groupByHashFactory,
OperatorFactories operatorFactories,
OrderingCompiler orderingCompiler,
DynamicFilterConfig dynamicFilterConfig,
@@ -458,12 +462,14 @@ public LocalExecutionPlanner(
this.maxLocalExchangeBufferSize = taskManagerConfig.getMaxLocalExchangeBufferSize();
this.pagesIndexFactory = requireNonNull(pagesIndexFactory, "pagesIndexFactory is null");
this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
+ this.groupByHashFactory = requireNonNull(groupByHashFactory, "groupByHashFactory is null");
this.operatorFactories = requireNonNull(operatorFactories, "operatorFactories is null");
this.orderingCompiler = requireNonNull(orderingCompiler, "orderingCompiler is null");
this.dynamicFilterConfig = requireNonNull(dynamicFilterConfig, "dynamicFilterConfig is null");
this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
this.tableExecuteContextManager = requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null");
this.exchangeManagerRegistry = requireNonNull(exchangeManagerRegistry, "exchangeManagerRegistry is null");
+ this.positionsAppenderFactory = new PositionsAppenderFactory(blockTypeOperators);
}
public LocalExecutionPlan plan(
@@ -994,8 +1000,7 @@ public PhysicalOperation visitRowNumber(RowNumberNode node, LocalExecutionPlanCo
node.getMaxRowCountPerPartition(),
hashChannel,
10_000,
- joinCompiler,
- blockTypeOperators);
+ groupByHashFactory);
return new PhysicalOperation(operatorFactory, outputMappings.buildOrThrow(), context, source);
}
@@ -1050,7 +1055,7 @@ public PhysicalOperation visitTopNRanking(TopNRankingNode node, LocalExecutionPl
hashChannel,
1000,
maxPartialTopNMemorySize,
- joinCompiler,
+ groupByHashFactory,
plannerContext.getTypeOperators(),
blockTypeOperators);
@@ -1735,8 +1740,7 @@ public PhysicalOperation visitDistinctLimit(DistinctLimitNode node, LocalExecuti
distinctChannels,
node.getLimit(),
hashChannel,
- joinCompiler,
- blockTypeOperators);
+ groupByHashFactory);
return new PhysicalOperation(operatorFactory, makeLayout(node), context, source);
}
@@ -1812,7 +1816,7 @@ public PhysicalOperation visitMarkDistinct(MarkDistinctNode node, LocalExecution
List channels = getChannelsForSymbols(node.getDistinctSymbols(), source.getLayout());
Optional hashChannel = node.getHashSymbol().map(channelGetter(source));
- MarkDistinctOperatorFactory operator = new MarkDistinctOperatorFactory(context.getNextOperatorId(), node.getId(), source.getTypes(), channels, hashChannel, joinCompiler, blockTypeOperators);
+ MarkDistinctOperatorFactory operator = new MarkDistinctOperatorFactory(context.getNextOperatorId(), node.getId(), source.getTypes(), channels, hashChannel, groupByHashFactory);
return new PhysicalOperation(operator, makeLayout(node), context, source);
}
@@ -2298,7 +2302,7 @@ public PhysicalOperation visitIndexJoin(IndexJoinNode node, LocalExecutionPlanCo
indexJoinLookupStats,
SystemSessionProperties.isShareIndexLoading(session),
pagesIndexFactory,
- joinCompiler,
+ groupByHashFactory,
blockTypeOperators);
verify(probeSource.getPipelineExecutionStrategy() == UNGROUPED_EXECUTION);
@@ -3103,8 +3107,7 @@ public PhysicalOperation visitSemiJoin(SemiJoinNode node, LocalExecutionPlanCont
buildChannel,
buildHashChannel,
10_000,
- joinCompiler,
- blockTypeOperators);
+ groupByHashFactory);
SetSupplier setProvider = setBuilderOperatorFactory.getSetProvider();
context.addDriverFactory(
buildContext.isInputDriver(),
@@ -3172,6 +3175,7 @@ public PhysicalOperation visitTableWriter(TableWriterNode node, LocalExecutionPl
PARTIAL,
Optional.empty(),
Optional.empty(),
+ Optional.empty(),
source,
false,
false,
@@ -3254,6 +3258,7 @@ public PhysicalOperation visitTableFinish(TableFinishNode node, LocalExecutionPl
FINAL,
Optional.empty(),
Optional.empty(),
+ Optional.empty(),
source,
false,
false,
@@ -3578,7 +3583,8 @@ private List getSymbolTypes(List symbols, TypeProvider types)
private AggregatorFactory buildAggregatorFactory(
PhysicalOperation source,
Aggregation aggregation,
- Step step)
+ Step step,
+ OptionalInt useRawInputChannel)
{
List argumentChannels = new ArrayList<>();
for (Expression argument : aggregation.getArguments()) {
@@ -3604,8 +3610,7 @@ private AggregatorFactory buildAggregatorFactory(
argumentChannels.stream()
.map(channel -> source.getTypes().get(channel))
.collect(toImmutableList()),
- joinCompiler,
- blockTypeOperators,
+ groupByHashFactory,
session);
}
@@ -3668,6 +3673,7 @@ private AggregatorFactory buildAggregatorFactory(
intermediateType,
finalType,
argumentChannels,
+ useRawInputChannel,
maskChannel,
!aggregation.isDistinct() && aggregation.getOrderingScheme().isEmpty(),
lambdaProviders);
@@ -3755,7 +3761,7 @@ private AggregationOperatorFactory createAggregationOperatorFactory(
for (Map.Entry entry : aggregations.entrySet()) {
Symbol symbol = entry.getKey();
Aggregation aggregation = entry.getValue();
- aggregatorFactories.add(buildAggregatorFactory(source, aggregation, step));
+ aggregatorFactories.add(buildAggregatorFactory(source, aggregation, step, OptionalInt.empty()));
outputMappings.put(symbol, outputChannel); // one aggregation per channel
outputChannel++;
}
@@ -3778,6 +3784,7 @@ private PhysicalOperation planGroupByAggregation(
node.getStep(),
node.getHashSymbol(),
node.getGroupIdSymbol(),
+ node.getUseRawInputSymbol(),
source,
node.hasDefaultOutput(),
spillEnabled,
@@ -3787,7 +3794,7 @@ private PhysicalOperation planGroupByAggregation(
0,
mappings,
10_000,
- Optional.of(maxPartialAggregationMemorySize));
+ Optional.of(getMaxPartialAggregationMemoryUsage(context.taskContext.getSession())));
return new PhysicalOperation(operatorFactory, mappings.buildOrThrow(), context, source);
}
@@ -3799,6 +3806,7 @@ private OperatorFactory createHashAggregationOperatorFactory(
Step step,
Optional hashSymbol,
Optional groupIdSymbol,
+ Optional useRawInputSymbol,
PhysicalOperation source,
boolean hasDefaultOutput,
boolean spillEnabled,
@@ -3812,11 +3820,15 @@ private OperatorFactory createHashAggregationOperatorFactory(
{
List aggregationOutputSymbols = new ArrayList<>();
List aggregatorFactories = new ArrayList<>();
+ OptionalInt useRawInputChannel = useRawInputSymbol
+ .filter(symbol -> !step.isInputRaw())
+ .map(symbol -> OptionalInt.of(source.getLayout().get(symbol)))
+ .orElseGet(OptionalInt::empty);
for (Map.Entry entry : aggregations.entrySet()) {
Symbol symbol = entry.getKey();
Aggregation aggregation = entry.getValue();
- aggregatorFactories.add(buildAggregatorFactory(source, aggregation, step));
+ aggregatorFactories.add(buildAggregatorFactory(source, aggregation, step, useRawInputChannel));
aggregationOutputSymbols.add(symbol);
}
@@ -3841,6 +3853,32 @@ private OperatorFactory createHashAggregationOperatorFactory(
outputMappings.put(symbol, channel);
channel++;
}
+ Map