Skip to content

[BEAM-1491]Identify HADOOP_CONF_DIR(or YARN_CONF_DIR) environment variables#2819

Closed
wypb wants to merge 1 commit intoapache:masterfrom
wypb:BEAM-1491
Closed

[BEAM-1491]Identify HADOOP_CONF_DIR(or YARN_CONF_DIR) environment variables#2819
wypb wants to merge 1 commit intoapache:masterfrom
wypb:BEAM-1491

Conversation

@wypb
Copy link
Contributor

@wypb wypb commented May 2, 2017

Be sure to do all of the following to help us incorporate your contribution
quickly and easily:

  • Make sure the PR title is formatted like:
    [BEAM-<Jira issue #>] Description of pull request
  • Make sure tests pass via mvn clean verify. (Even better, enable
    Travis-CI on your fork and ensure the whole test matrix passes).
  • Replace <Jira issue #> in the title with the actual Jira issue
    number, if there is one.
  • If this contribution is large, please file an Apache
    Individual Contributor License Agreement.

@wypb
Copy link
Contributor Author

wypb commented May 2, 2017

Hi @lukecwik, can you please review my PR? thank you.

@coveralls
Copy link

Coverage Status

Coverage remained the same at 69.852% when pulling fea5822 on 397090770:BEAM-1491 into ae72456 on apache:master.

Copy link
Member

@lukecwik lukecwik left a comment

Choose a reason for hiding this comment

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

Please also investigate the test failures that Jenkins has reported
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/10512/testReport/


// We just need to load both core-site.xml and hdfs-site.xml to determine the
// default fs path and the hdfs configuration
if (new File(hadoopConfPath + "/core-site.xml").exists()) {
Copy link
Member

@lukecwik lukecwik May 2, 2017

Choose a reason for hiding this comment

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

Please use the two argument constructor for File/Path here and below so we aren't assuming how path resolution works ("/" is common but not for every file system)

new File(hadoopConfPath, "core-site.xml");
new Path(hadoopConfPath, "core-site.xml");

if (new File(hadoopConfPath + "/core-site.xml").exists()) {
conf.addResource(new Path(hadoopConfPath + "/core-site.xml"));

if (LOG.isDebugEnabled()) {
Copy link
Member

Choose a reason for hiding this comment

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

Please use parameterized messages

if (LOG.isDebugEnabled()) {
  LOG.debug("Log " + x);
}

becomes

LOG.debug("Log {}", x);

See:
https://www.slf4j.org/faq.html#logging_performance

Configuration conf = new Configuration(false);
List<String> hadoopEnvList = Lists.newArrayList("HADOOP_CONF_DIR", "YARN_CONF_DIR");
for (String env : hadoopEnvList) {
String hadoopConfPath = System.getenv(env);
Copy link
Member

@lukecwik lukecwik May 2, 2017

Choose a reason for hiding this comment

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

Add unit tests.

Note that System.getenv isn't mockable so the best bet is to make a method on ConfigurationLocator like:

@VisibleForTesting
Map<String, String> getEnvironment() {
  return System.getenv(key);
}

and spy it in your unit tests.

See GcpOptions and GcpOptionsTest for an example of how this kind of interaction can be tested

}
}
}
return Lists.<Configuration>newArrayList(conf);
Copy link
Member

Choose a reason for hiding this comment

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

We shouldn't be returning a configuration if we didn't load one from one of the paths.

// Find default configuration when HADOOP_CONF_DIR or YARN_CONF_DIR is set.
Configuration conf = new Configuration(false);
List<String> hadoopEnvList = Lists.newArrayList("HADOOP_CONF_DIR", "YARN_CONF_DIR");
for (String env : hadoopEnvList) {
Copy link
Member

Choose a reason for hiding this comment

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

If we find a configuration in HADOOP_CONF_DIR and YARN_CONF_DIR, we should be returning them both separately and not having the YARN_CONF_DIR overwriting the properties found in HADOOP_CONF_DIR.

Also, ensure that we only load one configuration if both HADOOP_CONF_DIR and YARN_CONF_DIR point to the same location.

@lukecwik
Copy link
Member

lukecwik commented May 2, 2017

Sorry for the churn since the SDK is going through several changes towards the first stable release.

@wypb wypb closed this May 4, 2017
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants