Skip to content

tuningConfig.jobProperties not passed to hadoop #5135

@gvsmirnov

Description

@gvsmirnov

While upgrading from 0.9.1 to 0.10.1, we noticed that the segment reindexing tasks are failing with the following exception:

Caused by: java.lang.IllegalArgumentException: AWS Access Key ID and Secret Access Key must be specified as the username or password (respectively) of a s3n URL, or by setting the fs.s3n.awsAccessKeyId or fs.s3n.awsSecretAccessKey properties (respectively).
	at org.apache.hadoop.fs.s3.S3Credentials.initialize(S3Credentials.java:70) ~[?:?]
	at org.apache.hadoop.fs.s3native.Jets3tNativeFileSystemStore.initialize(Jets3tNativeFileSystemStore.java:80) ~[?:?]
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ~[?:1.8.0_131]
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) ~[?:1.8.0_131]
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_131]
	at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_131]
	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:191) ~[?:?]
	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102) ~[?:?]
	at org.apache.hadoop.fs.s3native.$Proxy209.initialize(Unknown Source) ~[?:?]
	at org.apache.hadoop.fs.s3native.NativeS3FileSystem.initialize(NativeS3FileSystem.java:334) ~[?:?]
	at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2669) ~[?:?]
	at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:94) ~[?:?]
	at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2703) ~[?:?]
	at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2685) ~[?:?]
	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:373) ~[?:?]
	at org.apache.hadoop.fs.Path.getFileSystem(Path.java:295) ~[?:?]
	at io.druid.indexer.hadoop.DatasourceInputFormat$3$1.listStatus(DatasourceInputFormat.java:173) ~[?:?]
	at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:315) ~[?:?]
	at io.druid.indexer.hadoop.DatasourceInputFormat.lambda$getLocations$1(DatasourceInputFormat.java:213) ~[?:?]
	at java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:267) ~[?:1.8.0_131]
	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1374) ~[?:1.8.0_131]
	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481) ~[?:1.8.0_131]
	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471) ~[?:1.8.0_131]
	at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) ~[?:1.8.0_131]
	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:1.8.0_131]
	at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499) ~[?:1.8.0_131]
	at io.druid.indexer.hadoop.DatasourceInputFormat.getFrequentLocations(DatasourceInputFormat.java:236) ~[?:?]
	at io.druid.indexer.hadoop.DatasourceInputFormat.toDataSourceSplit(DatasourceInputFormat.java:194) ~[?:?]
	at io.druid.indexer.hadoop.DatasourceInputFormat.getSplits(DatasourceInputFormat.java:124) ~[?:?]
	at org.apache.hadoop.mapreduce.lib.input.DelegatingInputFormat.getSplits(DelegatingInputFormat.java:115) ~[?:?]
	at org.apache.hadoop.mapreduce.JobSubmitter.writeNewSplits(JobSubmitter.java:301) ~[?:?]
	at org.apache.hadoop.mapreduce.JobSubmitter.writeSplits(JobSubmitter.java:318) ~[?:?]
	at org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:196) ~[?:?]
	at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1290) ~[?:?]
	at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1287) ~[?:?]
	at java.security.AccessController.doPrivileged(Native Method) ~[?:1.8.0_131]
	at javax.security.auth.Subject.doAs(Subject.java:422) ~[?:1.8.0_131]
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698) ~[?:?]
	at org.apache.hadoop.mapreduce.Job.submit(Job.java:1287) ~[?:?]
	at io.druid.indexer.IndexGeneratorJob.run(IndexGeneratorJob.java:205) ~[druid-indexing-hadoop-0.10.1-iap3.jar:0.10.1-iap3]
	at io.druid.indexer.JobHelper.runJobs(JobHelper.java:372) ~[druid-indexing-hadoop-0.10.1-iap3.jar:0.10.1-iap3]
	at io.druid.indexer.HadoopDruidIndexerJob.run(HadoopDruidIndexerJob.java:95) ~[druid-indexing-hadoop-0.10.1-iap3.jar:0.10.1-iap3]
	at io.druid.indexing.common.task.HadoopIndexTask$HadoopIndexGeneratorInnerProcessing.runTask(HadoopIndexTask.java:277) ~[druid-indexing-service-0.10.1-iap3.jar:0.10.1-iap3]

However, some lines above in the logs, the values are visible, set as per the documentation (and in a way that worked fine pre-upgrade):

{
  "type" : "index_hadoop",
  "spec" : {
    // ...
    "tuningConfig" : {
      "type" : "hadoop",
      "jobProperties" : {
        "fs.s3n.awsAccessKeyId" : "<key id>",        // <- here are the
        "fs.s3n.awsSecretAccessKey" : "<secret key>" // <- properties
      },
      //...
    }
  },
  // ...
}

After some investigation, I found out that the real config is ignored, and what hadoop gets instead is this:

JobConf dummyConf = new JobConf();

In the PR that it was introduced in, the only discussion of this line is that dummyConf should be a local variable instead of a field: #2223 (comment)

I am currently looking for a workaround, but this should be fixed for good. However, l cannot understand how the dummyConf was supposed to ever work. Maybe @navis can help explain?

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type
    No fields configured for issues without a type.

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions