[
https://issues.apache.org/jira/browse/BEAM-14129?focusedWorklogId=745168&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-745168
]
ASF GitHub Bot logged work on BEAM-14129:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 21/Mar/22 15:18
Start Date: 21/Mar/22 15:18
Worklog Time Spent: 10m
Work Description: dpcollins-google commented on a change in pull request
#17125:
URL: https://github.com/apache/beam/pull/17125#discussion_r831226411
##########
File path:
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiterImpl.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.beam.sdk.io.gcp.pubsublite.internal;
+
+import static
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+
+import javax.annotation.concurrent.GuardedBy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MemoryLimiterImpl implements MemoryLimiter {
+ private static final Logger LOG =
LoggerFactory.getLogger(MemoryLimiterImpl.class);
+ private final long minBlockSize;
+ private final long maxAvailable;
+
+ @GuardedBy("this")
+ private long available;
+
+ public MemoryLimiterImpl(long minBlockSize, long maxAvailable) {
+ this.minBlockSize = minBlockSize;
+ this.maxAvailable = available;
+ this.available = maxAvailable;
+ }
+
+ @Override
+ public synchronized Block claim(long toAcquire) {
Review comment:
The MemoryLimiter logic should over time reduce the amount of memory
allocated to subscribers if near or over the max limit.
For example, assume the server is out of memory (available == 0) and one
subscriber holds the max amount (100 MiB). There are some amount of servers
holding the min memory (4 MiB). When the subscriber holding the max releases
its block, the limiter has 100 MiB available, but won't give out more than half
unless it would be below the min. So it will give back only 50 MiB, and retain
the other 50 for itself. Then assume one of the subscribers only allocated 4
MiB comes back and requests 100 (the max) again. It will return its 4 MiB, then
request from the pool of 54 MiB available, which the limiter will then give it
26 MiB (half) instead, while having 26 MiB remaining. Over time, this will
equalize all subscribers (assuming the backend is keeping up with them so they
always request the max limit)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 745168)
Time Spent: 4h (was: 3h 50m)
> Fix issues with Pub/Sub Lite IO at high volumes
> -----------------------------------------------
>
> Key: BEAM-14129
> URL: https://issues.apache.org/jira/browse/BEAM-14129
> Project: Beam
> Issue Type: Task
> Components: io-java-gcp
> Reporter: Daniel Collins
> Assignee: Daniel Collins
> Priority: P1
> Fix For: 2.38.0
>
> Time Spent: 4h
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.20.1#820001)