-
Notifications
You must be signed in to change notification settings - Fork 8.9k
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
HADOOP-18180. Replace use of twitter util-core with java futures #4115
Changes from 1 commit
d48eaa7
02be95b
a0dd7fe
bcab50d
339f47d
c5b3c4b
5f9b643
915b1d3
27138bd
57a5ccb
66562cc
ca715d4
c01c7e0
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 |
---|---|---|
|
@@ -21,13 +21,12 @@ | |
|
||
import java.io.IOException; | ||
import java.nio.ByteBuffer; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.Future; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
import java.util.function.Supplier; | ||
|
||
import com.twitter.util.Await; | ||
import com.twitter.util.ExceptionalFunction0; | ||
import com.twitter.util.Future; | ||
import com.twitter.util.FuturePool; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
|
@@ -39,7 +38,7 @@ public abstract class CachingBlockManager extends BlockManager { | |
private static final Logger LOG = LoggerFactory.getLogger(CachingBlockManager.class); | ||
|
||
// Asynchronous tasks are performed in this pool. | ||
private final FuturePool futurePool; | ||
private final ExecutorServiceFuturePool futurePool; | ||
|
||
// Pool of shared ByteBuffer instances. | ||
private BufferPool bufferPool; | ||
|
@@ -78,7 +77,7 @@ public abstract class CachingBlockManager extends BlockManager { | |
* @throws IllegalArgumentException if bufferPoolSize is zero or negative. | ||
*/ | ||
public CachingBlockManager( | ||
FuturePool futurePool, | ||
ExecutorServiceFuturePool futurePool, | ||
BlockData blockData, | ||
int bufferPoolSize) { | ||
super(blockData); | ||
|
@@ -344,7 +343,7 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... | |
/** | ||
* Read task that is submitted to the future pool. | ||
*/ | ||
private static class PrefetchTask extends ExceptionalFunction0<Void> { | ||
private static class PrefetchTask implements Supplier<Void> { | ||
private final BufferData data; | ||
private final CachingBlockManager blockManager; | ||
|
||
|
@@ -354,7 +353,7 @@ private static class PrefetchTask extends ExceptionalFunction0<Void> { | |
} | ||
|
||
@Override | ||
public Void applyE() { | ||
public Void get() { | ||
try { | ||
this.blockManager.prefetch(data); | ||
} catch (Exception e) { | ||
|
@@ -412,7 +411,9 @@ public void requestCaching(BufferData data) { | |
if (state == BufferData.State.PREFETCHING) { | ||
blockFuture = data.getActionFuture(); | ||
} else { | ||
blockFuture = Future.value(null); | ||
CompletableFuture<Void> cf = new CompletableFuture<>(); | ||
cf.complete(null); | ||
blockFuture = cf; | ||
} | ||
|
||
CachePutTask task = new CachePutTask(data, blockFuture, this); | ||
|
@@ -433,13 +434,13 @@ private void addToCacheAndRelease(BufferData data, Future<Void> blockFuture) { | |
} | ||
|
||
try { | ||
Await.result(blockFuture); | ||
blockFuture.get(); //TODO consider calling get(long timeout, TimeUnit unit) instead | ||
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. if there's a todo item here, it'll need fixing or at least understanding of why a fix isn't needed 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. would it be possible to agree a timeout value - maybe 1 hour? - otherwise, this could block indefinitely 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. this is also the same problem that would occur with the previous Await.result call 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. I updated this with a 1 hour timeout |
||
if (data.stateEqualsOneOf(BufferData.State.DONE)) { | ||
// There was an error during prefetch. | ||
return; | ||
} | ||
} catch (Exception e) { | ||
String message = String.format("error waitng on blockFuture: %s", data); | ||
String message = String.format("error waiting on blockFuture: %s", data); | ||
pjfanning marked this conversation as resolved.
Show resolved
Hide resolved
|
||
LOG.error(message, e); | ||
data.setDone(); | ||
return; | ||
|
@@ -500,7 +501,7 @@ protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException { | |
this.cache.put(blockNumber, buffer); | ||
} | ||
|
||
private static class CachePutTask extends ExceptionalFunction0<Void> { | ||
private static class CachePutTask implements Supplier<Void> { | ||
private final BufferData data; | ||
|
||
// Block being asynchronously fetched. | ||
|
@@ -519,7 +520,7 @@ private static class CachePutTask extends ExceptionalFunction0<Void> { | |
} | ||
|
||
@Override | ||
public Void applyE() { | ||
public Void get() { | ||
this.blockManager.addToCacheAndRelease(this.data, this.blockFuture); | ||
return null; | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,47 @@ | ||
/* | ||
* 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.hadoop.fs.common; | ||
|
||
import java.util.Locale; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Future; | ||
import java.util.function.Supplier; | ||
|
||
public class ExecutorServiceFuturePool { | ||
steveloughran marked this conversation as resolved.
Show resolved
Hide resolved
|
||
private ExecutorService executor; | ||
|
||
public ExecutorServiceFuturePool(ExecutorService executor) { | ||
this.executor = executor; | ||
} | ||
|
||
/** | ||
* @param f function to run in future on executor pool | ||
* @return future | ||
* @throws java.util.concurrent.RejectedExecutionException can be thrown | ||
* @throws NullPointerException if f param is null | ||
*/ | ||
public Future<Void> apply(final Supplier<Void> f) { | ||
return executor.submit(f::get); | ||
} | ||
|
||
public String toString() { | ||
return String.format(Locale.ROOT,"ExecutorServiceFuturePool(executor=%s)", executor); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we need to think what we want to do here.
created https://issues.apache.org/jira/browse/HADOOP-18185