Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -205,9 +205,10 @@ private PlanNodeStatsEstimate estimateLogicalAnd(List<Expression> terms)
**/
private List<PlanNodeStatsEstimate> estimateCorrelatedExpressions(List<Expression> terms, double filterConjunctionIndependenceFactor)
{
ImmutableList.Builder<PlanNodeStatsEstimate> estimatesBuilder = ImmutableList.builder();
List<List<Expression>> extractedCorrelatedGroups = extractCorrelatedGroups(terms, filterConjunctionIndependenceFactor);
ImmutableList.Builder<PlanNodeStatsEstimate> estimatesBuilder = ImmutableList.builderWithExpectedSize(extractedCorrelatedGroups.size());
boolean hasUnestimatedTerm = false;
for (List<Expression> correlatedExpressions : extractCorrelatedGroups(terms, filterConjunctionIndependenceFactor)) {
for (List<Expression> correlatedExpressions : extractedCorrelatedGroups) {
PlanNodeStatsEstimate combinedEstimate = PlanNodeStatsEstimate.unknown();
for (Expression expression : correlatedExpressions) {
PlanNodeStatsEstimate estimate;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -254,8 +254,9 @@ private QueryMetadata createQueryMetadata(QueryInfo queryInfo)

private QueryStatistics createQueryStatistics(QueryInfo queryInfo)
{
ImmutableList.Builder<String> operatorSummaries = ImmutableList.builder();
for (OperatorStats summary : queryInfo.getQueryStats().getOperatorSummaries()) {
List<OperatorStats> operatorStats = queryInfo.getQueryStats().getOperatorSummaries();
ImmutableList.Builder<String> operatorSummaries = ImmutableList.builderWithExpectedSize(operatorStats.size());
for (OperatorStats summary : operatorStats) {
operatorSummaries.add(operatorStatsCodec.toJson(summary));
}

Expand Down Expand Up @@ -350,7 +351,7 @@ private static QueryIOMetadata getQueryIOMetadata(QueryInfo queryInfo)
{
Multimap<FragmentNode, OperatorStats> planNodeStats = extractPlanNodeStats(queryInfo);

ImmutableList.Builder<QueryInputMetadata> inputs = ImmutableList.builder();
ImmutableList.Builder<QueryInputMetadata> inputs = ImmutableList.builderWithExpectedSize(queryInfo.getInputs().size());
for (Input input : queryInfo.getInputs()) {
// Note: input table can be mapped to multiple operators
Collection<OperatorStats> inputTableOperatorStats = planNodeStats.get(new FragmentNode(input.getFragmentId(), input.getPlanNodeId()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ private static Failure toException(ExecutionFailureInfo executionFailureInfo)
for (ExecutionFailureInfo suppressed : executionFailureInfo.getSuppressed()) {
failure.addSuppressed(toException(suppressed));
}
ImmutableList.Builder<StackTraceElement> stackTraceBuilder = ImmutableList.builder();
ImmutableList.Builder<StackTraceElement> stackTraceBuilder = ImmutableList.builderWithExpectedSize(executionFailureInfo.getStack().size());
for (String stack : executionFailureInfo.getStack()) {
stackTraceBuilder.add(toStackTraceElement(stack));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ public OutputBufferInfo getInfo()
Collection<ClientBuffer> buffers = this.buffers.values();

int totalBufferedPages = masterBuffer.getBufferedPages();
ImmutableList.Builder<BufferInfo> infos = ImmutableList.builder();
ImmutableList.Builder<BufferInfo> infos = ImmutableList.builderWithExpectedSize(buffers.size());
for (ClientBuffer buffer : buffers) {
BufferInfo bufferInfo = buffer.getInfo();
infos.add(bufferInfo);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ public PartitionedOutputBuffer(
requireNonNull(notificationExecutor, "notificationExecutor is null"));
this.onPagesReleased = PagesReleasedListener.forOutputBufferMemoryManager(memoryManager);

ImmutableList.Builder<ClientBuffer> partitions = ImmutableList.builder();
ImmutableList.Builder<ClientBuffer> partitions = ImmutableList.builderWithExpectedSize(outputBuffers.getBuffers().keySet().size());
for (OutputBufferId bufferId : outputBuffers.getBuffers().keySet()) {
ClientBuffer partition = new ClientBuffer(taskInstanceId, bufferId, onPagesReleased);
partitions.add(partition);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ public MultilevelSplitQueue(double levelTimeMultiplier)
{
this.levelMinPriority = new AtomicLong[LEVEL_THRESHOLD_SECONDS.length];
this.levelWaitingSplits = new ArrayList<>(LEVEL_THRESHOLD_SECONDS.length);
ImmutableList.Builder<CounterStat> counters = ImmutableList.builder();
ImmutableList.Builder<CounterStat> counters = ImmutableList.builderWithExpectedSize(LEVEL_THRESHOLD_SECONDS.length);

for (int i = 0; i < LEVEL_THRESHOLD_SECONDS.length; i++) {
levelScheduledTime[i] = new AtomicLong();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,7 @@ public synchronized List<PrioritizedSplitRunner> destroy()
{
destroyed = true;

ImmutableList.Builder<PrioritizedSplitRunner> builder = ImmutableList.builder();
ImmutableList.Builder<PrioritizedSplitRunner> builder = ImmutableList.builderWithExpectedSize(runningIntermediateSplits.size() + runningLeafSplits.size() + queuedLeafSplits.size());
builder.addAll(runningIntermediateSplits);
builder.addAll(runningLeafSplits);
builder.addAll(queuedLeafSplits);
Expand Down
2 changes: 1 addition & 1 deletion core/trino-main/src/test/java/io/trino/RowPageBuilder.java
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ public static RowPageBuilder rowPageBuilder(Iterable<Type> types)
RowPageBuilder(Iterable<Type> types)
{
this.types = ImmutableList.copyOf(requireNonNull(types, "types is null"));
ImmutableList.Builder<BlockBuilder> builders = ImmutableList.builder();
ImmutableList.Builder<BlockBuilder> builders = ImmutableList.builderWithExpectedSize(this.types.size());
Comment thread
wendigo marked this conversation as resolved.
Outdated
for (Type type : types) {
builders.add(type.createBlockBuilder(null, 1));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -194,7 +194,7 @@ private <T> void assertBlockPositions(Block block, Supplier<BlockBuilder> newBlo
protected List<Block> splitBlock(Block block, int count)
{
double sizePerSplit = block.getPositionCount() * 1.0 / count;
ImmutableList.Builder<Block> result = ImmutableList.builder();
ImmutableList.Builder<Block> result = ImmutableList.builderWithExpectedSize(count);
for (int i = 0; i < count; i++) {
int startPosition = toIntExact(Math.round(sizePerSplit * i));
int endPosition = toIntExact(Math.round(sizePerSplit * (i + 1)));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,7 @@ public IntermediateTask(TaskExecutor taskExecutor, TaskSpecification specificati
@Override
public void schedule(TaskExecutor taskExecutor, int numSplits)
{
ImmutableList.Builder<SimulationSplit> splits = ImmutableList.builder();
ImmutableList.Builder<SimulationSplit> splits = ImmutableList.builderWithExpectedSize(numSplits);
for (int i = 0; i < numSplits; i++) {
splits.add(splitSpecification.instantiate(this));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -511,10 +511,12 @@ protected JdbcOutputTableHandle createTable(ConnectorSession session, ConnectorT
String remoteTargetTableName = identifierMapping.toRemoteTableName(identity, connection, remoteSchema, targetTableName);
String catalog = connection.getCatalog();

ImmutableList.Builder<String> columnNames = ImmutableList.builder();
ImmutableList.Builder<Type> columnTypes = ImmutableList.builder();
ImmutableList.Builder<String> columnList = ImmutableList.builder();
for (ColumnMetadata column : tableMetadata.getColumns()) {
List<ColumnMetadata> columns = tableMetadata.getColumns();
ImmutableList.Builder<String> columnNames = ImmutableList.builderWithExpectedSize(columns.size());
ImmutableList.Builder<Type> columnTypes = ImmutableList.builderWithExpectedSize(columns.size());
ImmutableList.Builder<String> columnList = ImmutableList.builderWithExpectedSize(columns.size());

for (ColumnMetadata column : columns) {
String columnName = identifierMapping.toRemoteColumnName(connection, column.getName());
columnNames.add(columnName);
columnTypes.add(column.getType());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ public JdbcRecordSet(JdbcClient jdbcClient, ExecutorService executor, ConnectorS

this.table = requireNonNull(table, "table is null");
this.columnHandles = requireNonNull(columnHandles, "columnHandles is null");
ImmutableList.Builder<Type> types = ImmutableList.builder();
ImmutableList.Builder<Type> types = ImmutableList.builderWithExpectedSize(columnHandles.size());
for (JdbcColumnHandle column : columnHandles) {
types.add(column.getColumnType());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorS
jdbcTable.getColumns()
.ifPresent(tableColumns -> verify(ImmutableSet.copyOf(tableColumns).containsAll(columns)));

ImmutableList.Builder<JdbcColumnHandle> handles = ImmutableList.builder();
ImmutableList.Builder<JdbcColumnHandle> handles = ImmutableList.builderWithExpectedSize(columns.size());
for (ColumnHandle handle : columns) {
handles.add((JdbcColumnHandle) handle);
}
Expand Down