[
https://issues.apache.org/jira/browse/DRILL-5365?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16550042#comment-16550042
]
ASF GitHub Bot commented on DRILL-5365:
---------------------------------------
ilooner commented on a change in pull request #1296: DRILL-5365: Prevent plugin
config from changing default fs. Make DrillFileSystem Immutable.
URL: https://github.com/apache/drill/pull/1296#discussion_r203907020
##########
File path:
exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFileSystemCache.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.drill.exec.store.dfs;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * <h4>Motivation</h4>
+ * <p>
+ * This cache is intended to work around the bugs in the {@link
org.apache.hadoop.fs.FileSystem} static cache (DRILL-5365). Specifically, as of
Hadoop 2.7.x the
+ * {@link org.apache.hadoop.fs.FileSystem} cache has the following bad
behavior:
+ * </p>
+ * <ul>
+ * <li>
+ * The {@link org.apache.hadoop.conf.Configuration} object is not
considered when constructing keys for the {@link
org.apache.hadoop.fs.FileSystem} cache of
+ * {@link org.apache.hadoop.fs.FileSystem} objects.
+ * </li>
+ * <li>
+ * The {@link org.apache.hadoop.fs.FileSystem} cache does not honor the
<b>fs.default.name</b> property when constructing keys, only
<b>fs.defaultFS</b> is used to construct
+ * keys in the cache.
+ * </li>
+ * </ul>
+ *
+ * <h4>Usage</h4>
+ *
+ * <ul>
+ * <li>
+ * A prerequisite for usage is that all {@link
org.apache.hadoop.conf.Configuration} objects are normalized with
+ * {@link
org.apache.drill.exec.store.dfs.DrillFileSystem#normalize(Configuration)}.
+ * </li>
+ * <li>
+ * This cache should only be used from {@link
org.apache.drill.exec.store.dfs.DrillFileSystem}.
+ * </li>
+ * </ul>
+ *
+ * <h4>TODO</h4>
+ *
+ * <ul>
+ * <li>
+ * Drill currently keeps a {@link
org.apache.drill.exec.store.dfs.DrillFileSystem} open indefinitely. This will
be corrected
+ * in DRILL-6608. As a result this cache currently has no methods to
remove {@link org.apache.hadoop.fs.FileSystem} objects
+ * after they are created.
+ * </li>
+ * </ul>
+ */
+class DrillFileSystemCache {
+ private Map<Map<String, String>, FileSystem> cache = new HashMap<>();
+
+ /**
+ * If a {@link org.apache.hadoop.fs.FileSystem} object corresponding to the
given {@link org.apache.hadoop.conf.Configuration}
+ * exists in the cache, then it is returned. If no corresponding {@link
org.apache.hadoop.fs.FileSystem} exist, then it is created,
+ * added to the cache, and returned.
+ * @param configuration The {@link org.apache.hadoop.conf.Configuration}
corresponding to the desired {@link org.apache.hadoop.fs.FileSystem}
+ * object. It is expected that this configuration is
first normalized with {@link
org.apache.drill.exec.store.dfs.DrillFileSystem#normalize(Configuration)}.
+ * @return The {@link org.apache.hadoop.fs.FileSystem} object corresponding
to he given {@link org.apache.hadoop.conf.Configuration}.
+ * @throws IOException An error when creating the desired {@link
org.apache.hadoop.fs.FileSystem} object.
+ */
+ protected synchronized FileSystem get(final Configuration configuration)
throws IOException {
+ final Map<String, String> map = new HashMap<>(configToMap(configuration));
+
+ if (!cache.containsKey(map)) {
+ cache.put(map, FileSystem.newInstance(configuration));
+ }
+
+ return cache.get(map);
+ }
+
+ static Map<String, String> configToMap(final Configuration configuration) {
+ Preconditions.checkNotNull(configuration);
+ final Map<String, String> map = new HashMap<>();
+
+ for (Map.Entry<String, String> entry: configuration) {
+ map.put(entry.getKey().trim(), entry.getValue());
Review comment:
- The UGI object is only created and used in the Key when
FileSystem.getUnique is called. The call DrillFileSystem was using previously
was FileSystem.get, so the cache never used UGI objects as part of the key for
our case. The behavior is pre-existing and I'm not sure if it's a bug or now.
We should determine if it is a bug or not, if it is I would like to fix that as
a separate Jira. What are your thoughts?
- For your second point, you are right. However, there is a bug in the logic
of FileSystem.getDefaultUri (which is used to construct the key) that if both
fs.defaultFS and fs.default.name are defined in a configuration the deprecated
option (fs.defaultFS) wins. Also it does not take into account the rest of the
configuration object in the key. So if you require two different configurations
for the same file system you are out of luck, only the first one wins.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> FileNotFoundException when reading a parquet file
> -------------------------------------------------
>
> Key: DRILL-5365
> URL: https://issues.apache.org/jira/browse/DRILL-5365
> Project: Apache Drill
> Issue Type: Bug
> Components: Storage - Hive
> Affects Versions: 1.10.0
> Reporter: Chun Chang
> Assignee: Timothy Farkas
> Priority: Major
> Fix For: 1.14.0
>
>
> The parquet file is generated through the following CTAS.
> To reproduce the issue: 1) two or more nodes cluster; 2) enable
> impersonation; 3) set "fs.default.name": "file:///" in hive storage plugin;
> 4) restart drillbits; 5) as a regular user, on node A, drop the table/file;
> 6) ctas from a large enough hive table as source to recreate the table/file;
> 7) query the table from node A should work; 8) query from node B as same user
> should reproduce the issue.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)