-
Notifications
You must be signed in to change notification settings - Fork 3k
Flink 1.19: Run without Hadoop #7369
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
Changes from all commits
b4d3887
c088745
b4df442
9debe7d
fc52d5a
bf64429
c546668
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,35 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
| package org.apache.iceberg.hadoop; | ||
|
|
||
| /** Responsible telling if specific Hadoop dependencies are on classpath. */ | ||
| public class HadoopDependency { | ||
|
|
||
| private HadoopDependency() {} | ||
|
|
||
| public static boolean isHadoopCommonOnClasspath(ClassLoader classLoader) { | ||
| try { | ||
| Class.forName("org.apache.hadoop.conf.Configuration", false, classLoader); | ||
| Class.forName("org.apache.hadoop.security.UserGroupInformation", false, classLoader); | ||
| return true; | ||
| } catch (ClassNotFoundException e) { | ||
| return false; | ||
| } | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,13 +26,18 @@ | |
|
|
||
| /** Wraps a {@link Configuration} object in a {@link Serializable} layer. */ | ||
| public class SerializableConfiguration implements Serializable { | ||
| private static final long serialVersionUID = -8840976521081151175L; | ||
pvary marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| private transient Configuration hadoopConf; | ||
|
|
||
| public SerializableConfiguration(Configuration hadoopConf) { | ||
| this.hadoopConf = hadoopConf; | ||
| } | ||
|
|
||
| public SerializableConfiguration(Object hadoopConf) { | ||
| this.hadoopConf = (Configuration) hadoopConf; | ||
| } | ||
|
Comment on lines
+37
to
+39
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What's the point here? When there's no Hadoop one classpath then it will blow up no matter what, right?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am also interested in the answer to Gabor's question. also wondering if we can get overload ambiguity from the two constructors? |
||
|
|
||
| private void writeObject(ObjectOutputStream out) throws IOException { | ||
| out.defaultWriteObject(); | ||
| hadoopConf.write(out); | ||
|
|
@@ -47,4 +52,8 @@ private void readObject(ObjectInputStream in) throws ClassNotFoundException, IOE | |
| public Configuration get() { | ||
| return hadoopConf; | ||
| } | ||
|
|
||
| public Configuration getClone() { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. maybe name this method just as
|
||
| return new Configuration(hadoopConf); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,22 +18,18 @@ | |
| */ | ||
| package org.apache.iceberg.flink; | ||
|
|
||
| import java.net.URL; | ||
| import java.nio.file.Files; | ||
| import java.nio.file.Paths; | ||
| import java.util.List; | ||
| import java.util.Locale; | ||
| import java.util.Map; | ||
| import org.apache.flink.configuration.GlobalConfiguration; | ||
| import org.apache.flink.runtime.hadoop.HadoopDependency; | ||
| import org.apache.flink.runtime.util.HadoopUtils; | ||
| import org.apache.flink.table.catalog.Catalog; | ||
| import org.apache.flink.table.factories.CatalogFactory; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.iceberg.CatalogProperties; | ||
| import org.apache.iceberg.catalog.Namespace; | ||
| import org.apache.iceberg.flink.util.HadoopUtil; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Strings; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Maps; | ||
| import org.apache.iceberg.util.PropertyUtil; | ||
|
|
@@ -56,7 +52,7 @@ | |
| * </ul> | ||
| * | ||
| * <p>To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override | ||
| * {@link #createCatalogLoader(String, Map, Configuration)}. | ||
| * {@link #createCatalogLoader(String, Map, Object)}. | ||
| */ | ||
| public class FlinkCatalogFactory implements CatalogFactory { | ||
|
|
||
|
|
@@ -85,7 +81,7 @@ public class FlinkCatalogFactory implements CatalogFactory { | |
| * @return an Iceberg catalog loader | ||
| */ | ||
| static CatalogLoader createCatalogLoader( | ||
| String name, Map<String, String> properties, Configuration hadoopConf) { | ||
| String name, Map<String, String> properties, Object hadoopConf) { | ||
| String catalogImpl = properties.get(CatalogProperties.CATALOG_IMPL); | ||
| if (catalogImpl != null) { | ||
| String catalogType = properties.get(ICEBERG_CATALOG_TYPE); | ||
|
|
@@ -103,10 +99,10 @@ static CatalogLoader createCatalogLoader( | |
| case ICEBERG_CATALOG_TYPE_HIVE: | ||
| // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in | ||
| // that case it will | ||
| // fallback to parse those values from hadoop configuration which is loaded from classpath. | ||
| // fall back to parse those values from hadoop configuration which is loaded from classpath. | ||
| String hiveConfDir = properties.get(HIVE_CONF_DIR); | ||
| String hadoopConfDir = properties.get(HADOOP_CONF_DIR); | ||
| Configuration newHadoopConf = mergeHiveConf(hadoopConf, hiveConfDir, hadoopConfDir); | ||
| Object newHadoopConf = HadoopUtil.mergeHiveConf(hadoopConf, hiveConfDir, hadoopConfDir); | ||
| return CatalogLoader.hive(name, newHadoopConf, properties); | ||
|
|
||
| case ICEBERG_CATALOG_TYPE_HADOOP: | ||
|
|
@@ -139,8 +135,7 @@ public Catalog createCatalog(String name, Map<String, String> properties) { | |
| return createCatalog(name, properties, clusterHadoopConf()); | ||
| } | ||
|
|
||
| protected Catalog createCatalog( | ||
| String name, Map<String, String> properties, Configuration hadoopConf) { | ||
| protected Catalog createCatalog(String name, Map<String, String> properties, Object hadoopConf) { | ||
| CatalogLoader catalogLoader = createCatalogLoader(name, properties, hadoopConf); | ||
| String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, DEFAULT_DATABASE_NAME); | ||
|
|
||
|
|
@@ -172,42 +167,11 @@ protected Catalog createCatalog( | |
| cacheExpirationIntervalMs); | ||
| } | ||
|
|
||
| private static Configuration mergeHiveConf( | ||
| Configuration hadoopConf, String hiveConfDir, String hadoopConfDir) { | ||
| Configuration newConf = new Configuration(hadoopConf); | ||
| if (!Strings.isNullOrEmpty(hiveConfDir)) { | ||
| Preconditions.checkState( | ||
| Files.exists(Paths.get(hiveConfDir, "hive-site.xml")), | ||
| "There should be a hive-site.xml file under the directory %s", | ||
| hiveConfDir); | ||
| newConf.addResource(new Path(hiveConfDir, "hive-site.xml")); | ||
| public static Object clusterHadoopConf() { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this better in the |
||
| if (HadoopDependency.isHadoopCommonOnClasspath(CatalogLoader.class.getClassLoader())) { | ||
| return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration()); | ||
| } else { | ||
| // If don't provide the hive-site.xml path explicitly, it will try to load resource from | ||
| // classpath. If still | ||
| // couldn't load the configuration file, then it will throw exception in HiveCatalog. | ||
| URL configFile = CatalogLoader.class.getClassLoader().getResource("hive-site.xml"); | ||
| if (configFile != null) { | ||
| newConf.addResource(configFile); | ||
| } | ||
| return null; | ||
| } | ||
|
|
||
| if (!Strings.isNullOrEmpty(hadoopConfDir)) { | ||
| Preconditions.checkState( | ||
| Files.exists(Paths.get(hadoopConfDir, "hdfs-site.xml")), | ||
| "Failed to load Hadoop configuration: missing %s", | ||
| Paths.get(hadoopConfDir, "hdfs-site.xml")); | ||
| newConf.addResource(new Path(hadoopConfDir, "hdfs-site.xml")); | ||
| Preconditions.checkState( | ||
| Files.exists(Paths.get(hadoopConfDir, "core-site.xml")), | ||
| "Failed to load Hadoop configuration: missing %s", | ||
| Paths.get(hadoopConfDir, "core-site.xml")); | ||
| newConf.addResource(new Path(hadoopConfDir, "core-site.xml")); | ||
| } | ||
|
|
||
| return newConf; | ||
| } | ||
|
|
||
| public static Configuration clusterHadoopConf() { | ||
| return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration()); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -69,7 +69,7 @@ private FlinkConfigOptions() {} | |
| public static final ConfigOption<Boolean> TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO = | ||
| ConfigOptions.key("table.exec.iceberg.expose-split-locality-info") | ||
| .booleanType() | ||
| .noDefaultValue() | ||
| .defaultValue(false) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is a backwards incompatible change. I'm not sure how widely it's used, but we need to bdo some research and be more vocal about this change, if we decide to go ahead with it.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. agree with @pvary that this changes the default behavior, which calls would it work if we add the |
||
| .withDescription( | ||
| "Expose split host information to use Flink's locality aware split assigner."); | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -163,7 +163,8 @@ private static TableLoader createTableLoader( | |
| String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName); | ||
| Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null"); | ||
|
|
||
| org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf(); | ||
| org.apache.hadoop.conf.Configuration hadoopConf = | ||
| (org.apache.hadoop.conf.Configuration) FlinkCatalogFactory.clusterHadoopConf(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. So FlinkSQL will still need hadoop on the classpath?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. agree with Peter that this seems problematic, as the return value can be null |
||
| FlinkCatalogFactory factory = new FlinkCatalogFactory(); | ||
| FlinkCatalog flinkCatalog = | ||
| (FlinkCatalog) factory.createCatalog(catalogName, tableProps, hadoopConf); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What happens when the hadoop is not on the classpath, but the table is
HadoopConfigurable?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No configuration will be passed through.