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

Regression 0.9.2 -> 0.10.1 in segments merging #4977

Closed
l15k4 opened this issue Oct 18, 2017 · 21 comments
Closed

Regression 0.9.2 -> 0.10.1 in segments merging #4977

l15k4 opened this issue Oct 18, 2017 · 21 comments
Labels

Comments

@l15k4
Copy link

l15k4 commented Oct 18, 2017

I'm now sure that this a bug :
https://groups.google.com/forum/#!topic/druid-user/2nq9KZlYdt8
There is no other way for me to pass the AWS credentials so we cannot upgrade to druid 0.10.1.

@l15k4
Copy link
Author

l15k4 commented Oct 18, 2017

These are full logs from server https://gist.github.com/l15k4/ff4e61376bf5367e707ddfd1fc15c287

@l15k4
Copy link
Author

l15k4 commented Oct 18, 2017

I compared the logs at 0.9.2 and 0.10.1 and they are identical ... the descriptor.json at s3 too...

Could this be caused by different hadoop-client versions 2.3.0 -> 2.7.3 ?

@l15k4
Copy link
Author

l15k4 commented Oct 18, 2017

The problem here is, that the tuningConfig is ignored in this case :-/ So I added druid/_common/core-site.xml with :

<configuration>
  <property>
    <name>fs.s3.impl</name>
    <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
  </property>

  <property>
    <name>fs.s3n.impl</name>
    <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
  </property>

  <property>
    <name>fs.s3a.access.key</name>
    <value>${COMMON_DRUID_S3_ACCESSKEY}</value>
  </property>

  <property>
    <name>fs.s3a.secret.key</name>
    <value>${COMMON_DRUID_S3_SECRETKEY}</value>
  </property>

  <property>
    <name>fs.s3a.endpoint</name>
    <value>s3.eu-west-1.amazonaws.com</value>
  </property>

  <property>
    <name>com.amazonaws.services.s3.enableV4</name>
    <value>true</value>
  </property>
</configuration>

But I'm getting this now :

java.lang.RuntimeException: java.lang.reflect.InvocationTargetException
	at com.google.common.base.Throwables.propagate(Throwables.java:160) ~[guava-16.0.1.jar:?]
	at io.druid.indexing.common.task.HadoopTask.invokeForeignLoader(HadoopTask.java:218) ~[druid-indexing-service-0.10.1-iap3.jar:0.10.1-iap3]
	at io.druid.indexing.common.task.HadoopIndexTask.run(HadoopIndexTask.java:224) ~[druid-indexing-service-0.10.1-iap3.jar:0.10.1-iap3]
	at io.druid.indexing.overlord.ThreadPoolTaskRunner$ThreadPoolTaskRunnerCallable.call(ThreadPoolTaskRunner.java:436) [druid-indexing-service-0.10.1-iap3.jar:0.10.1-iap3]
	at io.druid.indexing.overlord.ThreadPoolTaskRunner$ThreadPoolTaskRunnerCallable.call(ThreadPoolTaskRunner.java:408) [druid-indexing-service-0.10.1-iap3.jar:0.10.1-iap3]
	at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_101]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [?:1.8.0_101]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [?:1.8.0_101]
	at java.lang.Thread.run(Thread.java:745) [?:1.8.0_101]
Caused by: java.lang.reflect.InvocationTargetException
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ~[?:1.8.0_101]
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) ~[?:1.8.0_101]
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_101]
	at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_101]
	at io.druid.indexing.common.task.HadoopTask.invokeForeignLoader(HadoopTask.java:215) ~[druid-indexing-service-0.10.1-iap3.jar:0.10.1-iap3]
	... 7 more
Caused by: java.lang.NoSuchMethodError: com.amazonaws.services.s3.transfer.TransferManager.<init>(Lcom/amazonaws/services/s3/AmazonS3;Ljava/util/concurrent/ThreadPoolExecutor;)V
	at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:287) ~[?:?]
	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_101]
	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1374) ~[?:1.8.0_101]
	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481) ~[?:1.8.0_101]
	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471) ~[?:1.8.0_101]
	at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) ~[?:1.8.0_101]
	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) ~[?:1.8.0_101]
	at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499) ~[?:1.8.0_101]
	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) ~[?:?]

@l15k4
Copy link
Author

l15k4 commented Oct 18, 2017

Adding core-site.xml with :

<configuration>
  <property>
    <name>fs.s3.impl</name>
    <value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value>
  </property>

  <property>
    <name>fs.s3n.impl</name>
    <value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value>
  </property>

  <property>
    <name>fs.s3n.access.key</name>
    <value>${COMMON_DRUID_S3_ACCESSKEY}</value>
  </property>

  <property>
    <name>fs.s3n.secret.key</name>
    <value>${COMMON_DRUID_S3_SECRETKEY}</value>
  </property>
</configuration>

Doesn't help either, it is not picked up...

@jon-wei
Copy link
Contributor

jon-wei commented Oct 18, 2017

For this S3N configuration:

<configuration>
  <property>
    <name>fs.s3.impl</name>
    <value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value>
  </property>

  <property>
    <name>fs.s3n.impl</name>
    <value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value>
  </property>

  <property>
    <name>fs.s3n.access.key</name>
    <value>${COMMON_DRUID_S3_ACCESSKEY}</value>
  </property>

  <property>
    <name>fs.s3n.secret.key</name>
    <value>${COMMON_DRUID_S3_SECRETKEY}</value>
  </property>
</configuration>

I think you'd need to set "fs.s3n.awsAccessKeyId" instead of "fs.s3n.access.key" and "fs.s3n.awsSecretAccessKey" instead of "fs.s3n.secret.key"

@l15k4
Copy link
Author

l15k4 commented Oct 19, 2017

Thanks @jon-wei, I will try it out. But why doesn't it pick up the tuningConfig as it does in 0.9.2 ? We don't use hadoop at all, we just use it for s3 in/out .

@l15k4
Copy link
Author

l15k4 commented Oct 19, 2017

@jon-wei it works, although it is not grabbed from the jobProperties for some reason which it was in 0.9.2. So I'd let this issue open as other people may hit this issue too I believe.

@jon-wei
Copy link
Contributor

jon-wei commented Oct 19, 2017

@l15k4 Cool, good to hear that's working for you at least.

Thanks for the report on the tuningConfig issue.

To confirm, there is no remote hadoop cluster in your use case?

I remember testing 0.10.1 against several hadoop clusters, using S3 deep storage, and didn't see this issue, but don't think I've tried a case where there was no hadoop cluster, I wonder if something changed there.

@l15k4
Copy link
Author

l15k4 commented Oct 20, 2017

@jon-wei No hadoop cluster, no. We use s3 only + built-in druid hadoop for hadoop-indexing-tasks ...

@l15k4 l15k4 closed this as completed Oct 21, 2017
@l15k4
Copy link
Author

l15k4 commented Oct 21, 2017

Sorry I miss clicked and closed the issue ...

@l15k4 l15k4 reopened this Oct 21, 2017
@artmoskvin
Copy link

+1 observing this error

@ywilkof
Copy link

ywilkof commented Oct 25, 2017

I experience it as well.

@artmoskvin
Copy link

artmoskvin commented Oct 31, 2017

To sum up the problem, looks like the above configuration works only with v2 buckets. I can't make v4 buckets work with s3n and switching to s3a leads to java.lang.NoSuchMethodError: com.amazonaws.services.s3.transfer.TransferManager.<init>(Lcom/amazonaws/services/s3/AmazonS3;Ljava/util/concurrent/ThreadPoolExecutor;)V

@gianm
Copy link
Contributor

gianm commented Oct 31, 2017

@moscowart thanks for the summary -- sounds like the explanation of why some people have this issue and others don't is that some people are in aws v4 regions and some are not.

Has anyone in this thread had a chance to try recompiling Druid with adjusted versions of aws-java-sdk and/or Hadoop? Hadoop 2.8.2 is out now, and it is built against aws-java-sdk 1.10.6 so it may link better with the 1.10.77 that Druid is built with.

If this works, then upgrading Druid to Hadoop 2.8.2 could be a viable option.

@artmoskvin
Copy link

@gianm I'm going to try as I have a strict requirement having bucket in v4 region (Frankfurt). I'll let you know once I have any results. Btw did it work in previous versions? Like 0.9.2?

@artmoskvin
Copy link

So I was not able to update Hadoop in Druid due to resources constraint. I found a workaround, using Hadoop 2.7.3 from HDP 2.5.3 distribution. It is patched with aws-java-sdk-*:1.10.6 and works perfectly with it. So I replaced hadoop and aws jars and was able to use S3 v4 as a deep storage. I know it's a dirty hack but this is all I can do now.

@amalakar
Copy link
Contributor

amalakar commented Dec 6, 2017

I am facing the same issue with the latest release of druid-0.11 as well.

@amalakar
Copy link
Contributor

I was able to get it working by doing the following:

Following error was gone:

Caused by: java.lang.NoSuchMethodError: com.amazonaws.services.s3.transfer.TransferManager.<init>(Lcom/amazonaws/services/s3/AmazonS3;Ljava/util/concurrent/ThreadPoolExecutor;)V
        at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:287) ~[?:?]
        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) ~[?:?]

The code change can be seen here - https://github.com/amalakar/druid/pull/1/files

@hoesler
Copy link
Contributor

hoesler commented Jan 11, 2018

@gianm I compiled druid 0.11.0 against hadoop 2.8.3 and both hdfs deep storage and hadoop indexing (local) finally seem to work fine with s3a using a v4 endpoint. Here are my changes: 0.11.0...hoesler:feature/hadoop2.8

Do you think that it is feasible to upgrade hadoop dependencies in 0.12.0? Do you want me to send a pull request? If so, which branch should I use?

@stale
Copy link

stale bot commented Jun 21, 2019

This issue has been marked as stale due to 280 days of inactivity. It will be closed in 2 weeks if no further activity occurs. If this issue is still relevant, please simply write any comment. Even if closed, you can still revive the issue at any time or discuss it on the dev@druid.apache.org list. Thank you for your contributions.

@stale stale bot added the stale label Jun 21, 2019
@stale
Copy link

stale bot commented Jul 5, 2019

This issue has been closed due to lack of activity. If you think that is incorrect, or the issue requires additional review, you can revive the issue at any time.

@stale stale bot closed this as completed Jul 5, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

No branches or pull requests

7 participants