tejaswini-imply commented on code in PR #12827:
URL: https://github.com/apache/druid/pull/12827#discussion_r933928321


##########
extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetInputFormat.java:
##########
@@ -51,6 +53,26 @@ public ParquetInputFormat(
     this.conf = conf;
   }
 
+  private void initialize(Configuration conf)
+  {
+    //Initializing seperately since during eager initialization, resolving
+    //namenode hostname throws an error if nodes are ephemeral
+
+    // Ensure that FileSystem class level initialization happens with correct 
CL
+    // See https://github.com/apache/druid/issues/1714
+    ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
+    try {
+      
Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
+      FileSystem.get(conf);
+    }
+    catch (IOException ex) {
+      throw new RuntimeException(ex);
+    }
+    finally {
+      Thread.currentThread().setContextClassLoader(currCtxCl);
+    }

Review 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.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to