- 
                Notifications
    You must be signed in to change notification settings 
- Fork 3.4k
Bound and measure BigQuery's Arrow buffer allocations #24777
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
| @@ -0,0 +1,82 @@ | ||
| /* | ||
| * 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.plugin.bigquery; | ||
|  | ||
| import io.airlift.stats.CounterStat; | ||
| import org.apache.arrow.memory.AllocationListener; | ||
| import org.apache.arrow.memory.AllocationOutcome; | ||
| import org.weakref.jmx.Managed; | ||
| import org.weakref.jmx.Nested; | ||
|  | ||
| public class BigQueryArrowAllocatorStats | ||
| implements AllocationListener | ||
| { | ||
| private final CounterStat preAllocatedMemory = new CounterStat(); | ||
| private final CounterStat allocatedMemory = new CounterStat(); | ||
| private final CounterStat releasedMemory = new CounterStat(); | ||
| private final CounterStat failedMemory = new CounterStat(); | ||
|  | ||
| @Managed | ||
| @Nested | ||
| public CounterStat getAllocatedMemory() | ||
| { | ||
| return allocatedMemory; | ||
| } | ||
|  | ||
| @Managed | ||
| @Nested | ||
| public CounterStat getReleasedMemory() | ||
| { | ||
| return releasedMemory; | ||
| } | ||
|  | ||
| @Managed | ||
| @Nested | ||
| public CounterStat getPreAllocatedMemory() | ||
| { | ||
| return preAllocatedMemory; | ||
| } | ||
|  | ||
| @Managed | ||
| @Nested | ||
| public CounterStat getFailedMemory() | ||
| { | ||
| return failedMemory; | ||
| } | ||
|  | ||
| @Override | ||
| public void onAllocation(long size) | ||
| { | ||
| allocatedMemory.update(size); | ||
| } | ||
|  | ||
| @Override | ||
| public void onRelease(long size) | ||
| { | ||
| releasedMemory.update(size); | ||
| } | ||
|  | ||
| @Override | ||
| public void onPreAllocation(long size) | ||
| { | ||
| preAllocatedMemory.update(size); | ||
| } | ||
|  | ||
| @Override | ||
| public boolean onFailedAllocation(long size, AllocationOutcome outcome) | ||
| { | ||
| failedMemory.update(size); | ||
| return false; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
| @@ -0,0 +1,125 @@ | ||
| /* | ||
| * 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.plugin.bigquery; | ||
|  | ||
| import com.google.inject.Inject; | ||
| import io.airlift.log.Logger; | ||
| import org.apache.arrow.memory.AllocationListener; | ||
| import org.apache.arrow.memory.AllocationOutcome; | ||
| import org.apache.arrow.memory.BufferAllocator; | ||
| import org.apache.arrow.memory.RootAllocator; | ||
| import org.weakref.jmx.Flatten; | ||
| import org.weakref.jmx.Managed; | ||
|  | ||
| import static java.util.Objects.requireNonNull; | ||
|  | ||
| public class BigQueryArrowBufferAllocator | ||
| { | ||
| private static final Logger log = Logger.get(BigQueryArrowBufferAllocator.class); | ||
|  | ||
| private final long maximumAllocation; | ||
| private final BigQueryArrowAllocatorStats stats; | ||
| private final BufferAllocator rootAllocator; | ||
|  | ||
| @Inject | ||
| public BigQueryArrowBufferAllocator(BigQueryArrowConfig config, BigQueryArrowAllocatorStats stats) | ||
| { | ||
| this.maximumAllocation = requireNonNull(config, "config is null").getMaxAllocation().toBytes(); | ||
| There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We avoid using requireNonNull for config classes. #13940 | ||
| this.stats = requireNonNull(stats, "stats is null"); | ||
| this.rootAllocator = new RootAllocator(stats, maximumAllocation); | ||
| } | ||
|  | ||
| public BufferAllocator newChildAllocator(BigQuerySplit split) | ||
| { | ||
| return rootAllocator.newChildAllocator( | ||
| split.streamName(), | ||
| new RetryingAllocationListener(split.streamName(), stats), | ||
| split.dataSize().orElse(0), | ||
| maximumAllocation); | ||
| } | ||
|  | ||
| @Managed | ||
| @Flatten | ||
| public BigQueryArrowAllocatorStats getStats() | ||
| { | ||
| return stats; | ||
| } | ||
|  | ||
| @Managed | ||
| public long getCurrentAllocatedMemory() | ||
| { | ||
| return rootAllocator.getAllocatedMemory(); | ||
| } | ||
|  | ||
| @Managed | ||
| public long getPeakAllocatedMemory() | ||
| { | ||
| return rootAllocator.getPeakMemoryAllocation(); | ||
| } | ||
|  | ||
| @Managed | ||
| public long getCurrentMemoryHeadroom() | ||
| { | ||
| return rootAllocator.getHeadroom(); | ||
| } | ||
|  | ||
| @Managed | ||
| public long getChildAllocatorsCount() | ||
| { | ||
| return rootAllocator.getChildAllocators().size(); | ||
| } | ||
|  | ||
| private static class RetryingAllocationListener | ||
| implements AllocationListener | ||
| { | ||
| private final String name; | ||
| private final BigQueryArrowAllocatorStats stats; | ||
|  | ||
| private RetryingAllocationListener(String name, BigQueryArrowAllocatorStats stats) | ||
| { | ||
| this.name = requireNonNull(name, "name is null"); | ||
| this.stats = requireNonNull(stats, "stats is null"); | ||
| } | ||
|  | ||
| @Override | ||
| public boolean onFailedAllocation(long size, AllocationOutcome outcome) | ||
| { | ||
| stats.onFailedAllocation(size, outcome); | ||
|  | ||
| log.warn("Failed to allocate %d bytes for allocator '%s' due to %s", size, name, outcome.getStatus()); | ||
| outcome.getDetails().ifPresent(details -> { | ||
| log.warn("Allocation failure details: %s", details.toString()); | ||
| }); | ||
| return false; | ||
| } | ||
|  | ||
| @Override | ||
| public void onPreAllocation(long size) | ||
| { | ||
| stats.onPreAllocation(size); | ||
| } | ||
|  | ||
| @Override | ||
| public void onAllocation(long size) | ||
| { | ||
| stats.onAllocation(size); | ||
| } | ||
|  | ||
| @Override | ||
| public void onRelease(long size) | ||
| { | ||
| stats.onRelease(size); | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
| @@ -0,0 +1,36 @@ | ||
| /* | ||
| * 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.plugin.bigquery; | ||
|  | ||
| import io.airlift.configuration.Config; | ||
| import io.airlift.configuration.ConfigDescription; | ||
| import io.airlift.units.DataSize; | ||
|  | ||
| public class BigQueryArrowConfig | ||
| { | ||
| private DataSize maxAllocation = DataSize.ofBytes(Integer.MAX_VALUE); | ||
|  | ||
| public DataSize getMaxAllocation() | ||
| { | ||
| return maxAllocation; | ||
| } | ||
|  | ||
| @ConfigDescription("Maximum memory allocation allowed for Arrow library") | ||
| @Config("bigquery.arrow-serialization.max-allocation") | ||
| There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Dont we want to document this @wendigo ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We can but it's not obligatory. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seems too deep in the weeds .. lets leave it out until we find someone actually uses this for some benefit | ||
| public BigQueryArrowConfig setMaxAllocation(DataSize maxAllocation) | ||
| { | ||
| this.maxAllocation = maxAllocation; | ||
| return this; | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.