Skip to content

Lazy initialisation of Parquet extensions module#12827

Merged
kfaraz merged 2 commits intoapache:masterfrom
tejaswini-imply:fix-lazy_initialisation_of_parquet_extensions
Aug 2, 2022
Merged

Lazy initialisation of Parquet extensions module#12827
kfaraz merged 2 commits intoapache:masterfrom
tejaswini-imply:fix-lazy_initialisation_of_parquet_extensions

Conversation

@tejaswini-imply
Copy link
Member

@tejaswini-imply tejaswini-imply commented Jul 27, 2022

Fixes Historicals and MiddleManagers failing with UnknownHostException when working with the druid-parquet-extensions module and Ephemeral Hadoop cluster.

Description:

Druid services are failing with the following error during loading and configuring modules of the initialization phase in the case when fs.defaultFS URI value is specified while operating with an Ephemeral Hadoop cluster. While configuration HDFS DFS object is getting initialized with fs.defaultFS URI if present, which isn't getting resolved in case of Ephemeral cluster. Hence routed the initialization from ParquetExtensionsModule to ParquetInputFormat just before creating the ParquetReader object.

This fix is similar to the one done for OrcExtensionsModule --> #12663, #12738

An exception was caught and reported. Message: java.net.UnknownHostException: druid-dataproc-hadoop-cluster-us-east4-m
  at com.google.inject.util.Modules$OverrideModule.configure(Modules.java:198)
Caused by: java.lang.IllegalArgumentException: java.net.UnknownHostException: druid-dataproc-hadoop-cluster-us-east4-m
	at org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:417)
	at org.apache.hadoop.hdfs.NameNodeProxiesClient.createProxyWithClientProtocol(NameNodeProxiesClient.java:132)
	at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:351)
	at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:285)
	at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:160)
	at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2812)
	at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:100)
	at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2849)
	at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2831)
	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:389)
	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:181)
	at org.apache.druid.data.input.parquet.ParquetExtensionsModule.configure(ParquetExtensionsModule.java:85)
	at com.google.inject.spi.Elements$RecordingBinder.install(Elements.java:340)
	at com.google.inject.spi.Elements.getElements(Elements.java:110)
	at com.google.inject.util.Modules$OverrideModule.configure(Modules.java:198)
	at com.google.inject.AbstractModule.configure(AbstractModule.java:62)
	at com.google.inject.spi.Elements$RecordingBinder.install(Elements.java:340)
	at com.google.inject.spi.Elements.getElements(Elements.java:110)
	at com.google.inject.internal.InjectorShell$Builder.build(InjectorShell.java:138)
	at com.google.inject.internal.InternalInjectorCreator.build(InternalInjectorCreator.java:104)
	at com.google.inject.Guice.createInjector(Guice.java:99)
	at com.google.inject.Guice.createInjector(Guice.java:73)
	at com.google.inject.Guice.createInjector(Guice.java:62)
	at org.apache.druid.initialization.Initialization.makeInjectorWithModules(Initialization.java:451)
	at org.apache.druid.cli.GuiceRunnable.makeInjector(GuiceRunnable.java:93)
	at org.apache.druid.cli.GuiceRunnable.makeInjector(GuiceRunnable.java:86)
	at org.apache.druid.cli.CliPeon.run(CliPeon.java:302)
	at org.apache.druid.cli.Main.main(Main.java:113)
Caused by: java.net.UnknownHostException: druid-dataproc-hadoop-cluster-us-east4-m
	... 28 more

Key classes changed:

  • ParquetExtensionsModule
  • ParquetInputFormat

This PR has:

  • been self-reviewed.
  • added documentation for new or modified features or behaviors.
  • added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
  • added or updated version, license, or notice information in licenses.yaml
  • added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
  • added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
  • added integration tests.
  • been tested in a test Druid cluster.

Copy link
Contributor

@paul-rogers paul-rogers left a comment

Choose a reason for hiding this comment

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

Thanks for the fix! A few questions/comments.

}
finally {
Thread.currentThread().setContextClassLoader(currCtxCl);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

This seems a step in the right direction. Is it enough?

I the NN is ephemeral, isn't there still a race condition between here and when we actually resolve a file name using the NN? Do we have to repeat this for each NN change? Should it occur just prior to the first file reference? Should we then retry in case of a NN failover at that moment?

Also, this way of initializing with the target class loader works, but is odd. First, what does the class loader do? By calling getClass(), we're getting the class loader for this class. But, we already have that since we are in this class. Did we mean the class loader for the extensions module?

Can we have an Initialization class loaded in that class loader (assuming Druid code runs in that class loader), and invoke that so we get the class loader set up automagically?

Copy link
Member Author

Choose a reason for hiding this comment

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

Thanks, @paul-rogers, for the review. I have provided my thoughts and understanding of the concerns.

Also, this way of initializing with the target class loader works, but is odd. First, what does the class loader do? By calling getClass(), we're getting the class loader for this class. But, we already have that since we are in this class. Did we mean the class loader for the extensions module?

Seems like setting Thread context with the required classloader is needed for Hadoop versions before this patch https://issues.apache.org/jira/browse/HADOOP-12596. With this patch, ServiceLoader#load(..) is using classloader from Configuration.

Can we have an Initialization class loaded in that class loader (assuming Druid code runs in that class loader), and invoke that so we get the class loader set up automagically?

IMO I don't think this is necessary. The problem is that when FIleSystem.get(..) is called, hadoop-hdfs-client.jar is not in the classpath for classloader (Thread.currentThread().getContextClassLoader()) used because it wasn't in the original classpath of the druid process but was a transitive dependency of druid-hdfs-storage extension. So using the right classloader containing this jar should be good enough.

I the NN is ephemeral, isn't there still a race condition between here and when we actually resolve a file name using the NN? Do we have to repeat this for each NN change? Should it occur just prior to the first file reference? Should we then retry in case of a NN failover at that moment?

I'm not sure about the race condition between NN change and resolving the file name, but the problem case is, as per my understanding, for example, when users are operating Druid along with GCP workflows (which spins up ephemeral Dataproc cluster when ingestion data arrives, and NN gets resolved then) and they shouldn't have to restart Druid services every time an ingestion task is submitted. Moreover, FileSystem instantiated here is ignored, and I had to assume it's a fail first mechanism in case of conf disparity before ParquetReader even tries to attempt creating a HadoopInputFile from the config.

@paul-rogers
Copy link
Contributor

Thanks for the explanation! LGTM (non-binding).

Copy link
Contributor

@kfaraz kfaraz left a comment

Choose a reason for hiding this comment

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

Thanks for the fix, @tejaswini-imply ! LGTM.

@kfaraz
Copy link
Contributor

kfaraz commented Aug 2, 2022

Merging as failure is unrelated.

@kfaraz kfaraz merged commit cceb2e8 into apache:master Aug 2, 2022
@abhishekagarwal87 abhishekagarwal87 added this to the 24.0.0 milestone Aug 26, 2022
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.

4 participants