Skip to content
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

TEZ-4479: Eagerly Init/Load FileSystem In Tez Task Containers #274

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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 @@ -1131,6 +1131,14 @@ public TezConfiguration(boolean loadDefaults) {

public static final int TASK_HEARTBEAT_TIMEOUT_CHECK_MS_DEFAULT = 30 * 1000;

/**
* String value. Comma seperated list of FileSystem paths which needs to be eagerly initialized.
* For example s3://bucket/,file://,hdfs://localhost:8020/
*/
@ConfigurationScope(Scope.VERTEX)
@ConfigurationProperty
public static final String TEZ_TASK_EAGER_INIT_FS_PATHS = TEZ_TASK_PREFIX + "eager.init.fs.paths";

/**
* Whether to scale down memory requested by each component if the total
* exceeds the available JVM memory
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import java.security.PrivilegedExceptionAction;
import java.text.SimpleDateFormat;
import java.util.Calendar;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
Expand All @@ -40,6 +41,7 @@

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.Credentials;
Expand Down Expand Up @@ -130,6 +132,7 @@ public class TezChild {
private final HadoopShim hadoopShim;
private final TezExecutors sharedExecutor;
private ThreadLocalMap mdcContext;
private static ExecutorService eagerInitFsPool;

public TezChild(Configuration conf, String host, int port, String containerIdentifier,
String tokenIdentifier, int appAttemptNumber, String workingDir, String[] localDirs,
Expand Down Expand Up @@ -503,6 +506,33 @@ public static TezChild newTezChild(Configuration conf, String host, int port, St
hadoopShim);
}

private static void eagerInitFileSystemPaths(Configuration conf) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It will be good to measure the time spent on FS init (even for cloudstores) and share the details, before trying out this patch.

Reason: This get inited in TezChild, but for running container, it closes the FileSystem explicitly via "FileSystem.closeAllForUGI(childUGI);". Refer "public ContainerExecutionResult run()" method.

Even if this gets early inited, it will not have major impact in container reuse scenario. It will be good to measure and find out the timing spent in FS init.

Collection<String> eagerInitPaths = conf.getTrimmedStringCollection(
TezConfiguration.TEZ_TASK_EAGER_INIT_FS_PATHS);
if (eagerInitFsPool == null && !eagerInitPaths.isEmpty()) {
eagerInitFsPool = Executors.newCachedThreadPool(new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("Eager-Init-Fs-Thread-%d")
.build());
}
for (String path : eagerInitPaths) {
eagerInitFsPool.execute(new Runnable() {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

before rushing to create lots of fs instances in parallel, look at HADOOP-17313 and why we actually added semaphores to stop apps like tez creating too many at the same time. this code may cause overload problems, or the fs semaphore will hold you back for safety.

best to look at why its taking so long; if s3a bucket existence checks aren't involved, then it'll be whatever auth mechanism is plugged in. same for abfs

@Override
public void run() {
try {
long startTime = System.currentTimeMillis();
FileSystem fs = new Path(path).getFileSystem(conf);
long duration = System.currentTimeMillis() - startTime;
LOG.info("Eagerly initiated FileSystem at path {} in {} ms", path, duration);
} catch (Exception e) {
// swallow the exception since this doesn't block the core functionality
LOG.error("Unable to eager init FileSystem at the path {}", path, e);
}
}
});
}
}

public static void main(String[] args) throws IOException, InterruptedException, TezException {
TezClassLoader.setupTezClassLoader();
final Configuration defaultConf = new Configuration();
Expand Down Expand Up @@ -530,6 +560,8 @@ public static void main(String[] args) throws IOException, InterruptedException,
DAGProtos.ConfigurationProto confProto =
TezUtilsInternal.readUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name()));
TezUtilsInternal.addUserSpecifiedTezConfiguration(defaultConf, confProto.getConfKeyValuesList());
// eagerly load configured filesystem before it is actually required
eagerInitFileSystemPaths(defaultConf);
UserGroupInformation.setConfiguration(defaultConf);
Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();

Expand Down