This is an automated email from the ASF dual-hosted git repository.
kfaraz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git
The following commit(s) were added to refs/heads/master by this push:
new cceb2e849e Perform lazy initialization of parquet extensions module
(#12827)
cceb2e849e is described below
commit cceb2e849ee837beb1228adf018b9a1aec79e226
Author: Tejaswini Bandlamudi <[email protected]>
AuthorDate: Tue Aug 2 13:41:12 2022 +0530
Perform lazy initialization of parquet extensions module (#12827)
Historicals and middle managers crash with an `UnknownHostException` on
trying
to load `druid-parquet-extensions` with an ephemeral Hadoop cluster. This
happens
because the `fs.defaultFS` URI value cannot be resolved at start up time as
the
hadoop cluster may not exist at startup time.
This commit fixes the error by performing initialization of the filesystem
in
`ParquetInputFormat.createReader()` whenever a new reader is requested.
---
.../input/parquet/ParquetExtensionsModule.java | 16 ---------------
.../data/input/parquet/ParquetInputFormat.java | 23 ++++++++++++++++++++++
2 files changed, 23 insertions(+), 16 deletions(-)
diff --git
a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java
b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java
index 31a1e90f8e..84b1f562af 100644
---
a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java
+++
b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java
@@ -30,9 +30,7 @@ import
org.apache.druid.data.input.parquet.simple.ParquetHadoopInputRowParser;
import org.apache.druid.data.input.parquet.simple.ParquetParseSpec;
import org.apache.druid.initialization.DruidModule;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Properties;
@@ -77,20 +75,6 @@ public class ParquetExtensionsModule implements DruidModule
// Set explicit CL. Otherwise it'll try to use thread context CL, which
may not have all of our dependencies.
conf.setClassLoader(getClass().getClassLoader());
- // 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);
- }
-
if (props != null) {
for (String propName : props.stringPropertyNames()) {
if (propName.startsWith("hadoop.")) {
diff --git
a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetInputFormat.java
b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetInputFormat.java
index 1de4c552ac..766638c97a 100644
---
a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetInputFormat.java
+++
b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetInputFormat.java
@@ -29,9 +29,11 @@ import org.apache.druid.data.input.impl.NestedInputFormat;
import org.apache.druid.data.input.parquet.guice.Parquet;
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
import javax.annotation.Nullable;
import java.io.File;
+import java.io.IOException;
import java.util.Objects;
public class ParquetInputFormat extends NestedInputFormat
@@ -51,6 +53,26 @@ public class ParquetInputFormat extends NestedInputFormat
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);
+ }
+ }
+
@JsonProperty
public boolean getBinaryAsString()
{
@@ -70,6 +92,7 @@ public class ParquetInputFormat extends NestedInputFormat
File temporaryDirectory
)
{
+ initialize(conf);
return new ParquetReader(conf, inputRowSchema, source, temporaryDirectory,
getFlattenSpec(), binaryAsString);
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]