-
Notifications
You must be signed in to change notification settings - Fork 231
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
perf(server): speed up delayed fetch (#633)
Signed-off-by: Ning Yu <[email protected]>
- Loading branch information
1 parent
0396ffd
commit 3fcd307
Showing
6 changed files
with
257 additions
and
122 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,91 @@ | ||
/* | ||
* 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 kafka.server; | ||
|
||
import java.util.concurrent.Semaphore; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.concurrent.locks.Lock; | ||
import java.util.concurrent.locks.ReentrantLock; | ||
|
||
/** | ||
* A fair limiter whose {@link #acquire} method is fair, i.e. the waiting threads are served in the order of arrival. | ||
*/ | ||
public class FairLimiter implements Limiter { | ||
private final int maxPermits; | ||
/** | ||
* The lock used to protect @{link #acquireLocked} | ||
*/ | ||
private final Lock lock = new ReentrantLock(true); | ||
private final Semaphore permits; | ||
|
||
public FairLimiter(int size) { | ||
maxPermits = size; | ||
permits = new Semaphore(size); | ||
} | ||
|
||
@Override | ||
public Handler acquire(int permit) throws InterruptedException { | ||
lock.lock(); | ||
try { | ||
permits.acquire(permit); | ||
return new FairHandler(permit); | ||
} finally { | ||
lock.unlock(); | ||
} | ||
} | ||
|
||
@Override | ||
public Handler acquire(int permit, long timeoutMs) throws InterruptedException { | ||
long start = System.nanoTime(); | ||
if (lock.tryLock(timeoutMs, TimeUnit.MILLISECONDS)) { | ||
try { | ||
// calculate the time left for {@code acquireLocked} | ||
long elapsed = System.nanoTime() - start; | ||
long left = TimeUnit.MILLISECONDS.toNanos(timeoutMs) - elapsed; | ||
// note: {@code left} may be negative here, but it's OK for acquireLocked | ||
return acquireLocked(permit, left); | ||
} finally { | ||
lock.unlock(); | ||
} | ||
} else { | ||
// tryLock timeout, return null | ||
return null; | ||
} | ||
} | ||
|
||
private Handler acquireLocked(int permit, long timeoutNs) throws InterruptedException { | ||
if (permit > maxPermits) { | ||
permit = maxPermits; | ||
} | ||
boolean acquired = permits.tryAcquire(permit, timeoutNs, TimeUnit.NANOSECONDS); | ||
return acquired ? new FairHandler(permit) : null; | ||
} | ||
|
||
public class FairHandler implements Handler { | ||
private final int permit; | ||
|
||
public FairHandler(int permit) { | ||
this.permit = permit; | ||
} | ||
|
||
@Override | ||
public void close() { | ||
permits.release(permit); | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,49 @@ | ||
/* | ||
* 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 kafka.server; | ||
|
||
/** | ||
* A limiter that limits the number of permits that can be acquired at a time. | ||
*/ | ||
public interface Limiter { | ||
|
||
/** | ||
* Acquire permits, if not enough, block until enough. | ||
* | ||
* @param permit the number of permits to acquire, should not be negative | ||
* @return a handler to release the permits, never null. The handler should be closed after use. | ||
* @throws InterruptedException if interrupted while waiting | ||
*/ | ||
Handler acquire(int permit) throws InterruptedException; | ||
|
||
/** | ||
* Acquire permits, if not enough, block until enough or timeout. | ||
* | ||
* @param permit the number of permits to acquire, should not be negative | ||
* @param timeoutMs the maximum time to wait for the permits, in milliseconds. A non-positive value means not to wait. | ||
* @return a handler to release the permits or null if timeout. If not null, the handler should be closed after use. | ||
* @throws InterruptedException if interrupted while waiting | ||
*/ | ||
Handler acquire(int permit, long timeoutMs) throws InterruptedException; | ||
|
||
/** | ||
* A handler to release acquired permits. | ||
*/ | ||
interface Handler extends AutoCloseable { | ||
} | ||
} |
This file was deleted.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,42 @@ | ||
/* | ||
* 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 kafka.server; | ||
|
||
/** | ||
* A limiter that does nothing. | ||
*/ | ||
public class NoopLimiter implements Limiter { | ||
|
||
public static final NoopLimiter INSTANCE = new NoopLimiter(); | ||
|
||
@Override | ||
public Handler acquire(int permit) throws InterruptedException { | ||
return new NoopHandler(); | ||
} | ||
|
||
@Override | ||
public Handler acquire(int permit, long timeoutMs) throws InterruptedException { | ||
return new NoopHandler(); | ||
} | ||
|
||
public static class NoopHandler implements Handler { | ||
@Override | ||
public void close() { | ||
} | ||
} | ||
} |
Oops, something went wrong.