Lazy initialisation of Parquet extensions module#12827
Conversation
paul-rogers
left a comment
There was a problem hiding this comment.
Thanks for the fix! A few questions/comments.
...parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetInputFormat.java
Outdated
Show resolved
Hide resolved
| } | ||
| finally { | ||
| Thread.currentThread().setContextClassLoader(currCtxCl); | ||
| } |
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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.
|
Thanks for the explanation! LGTM (non-binding). |
kfaraz
left a comment
There was a problem hiding this comment.
Thanks for the fix, @tejaswini-imply ! LGTM.
|
Merging as failure is unrelated. |
Fixes Historicals and MiddleManagers failing with
UnknownHostExceptionwhen working with thedruid-parquet-extensionsmodule 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.defaultFSURI value is specified while operating with an Ephemeral Hadoop cluster. While configuration HDFS DFS object is getting initialized withfs.defaultFSURI if present, which isn't getting resolved in case of Ephemeral cluster. Hence routed the initialization fromParquetExtensionsModuletoParquetInputFormatjust before creating theParquetReaderobject.This fix is similar to the one done for
OrcExtensionsModule--> #12663, #12738Key classes changed:
ParquetExtensionsModuleParquetInputFormatThis PR has: