-
Notifications
You must be signed in to change notification settings - Fork 3.1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Fix HDFS impersonation in Iceberg Connector
- Loading branch information
Showing
8 changed files
with
182 additions
and
34 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
73 changes: 73 additions & 0 deletions
73
presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/HdfsInputFile.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,73 @@ | ||
/* | ||
* Licensed 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 io.prestosql.plugin.iceberg; | ||
|
||
import io.prestosql.plugin.hive.HdfsEnvironment; | ||
import io.prestosql.plugin.hive.HdfsEnvironment.HdfsContext; | ||
import io.prestosql.spi.PrestoException; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.iceberg.hadoop.HadoopInputFile; | ||
import org.apache.iceberg.io.InputFile; | ||
import org.apache.iceberg.io.SeekableInputStream; | ||
|
||
import java.io.IOException; | ||
|
||
import static io.prestosql.plugin.iceberg.IcebergErrorCode.ICEBERG_FILESYSTEM_ERROR; | ||
import static java.util.Objects.requireNonNull; | ||
|
||
public class HdfsInputFile | ||
implements InputFile | ||
{ | ||
private final InputFile delegate; | ||
private final HdfsEnvironment environment; | ||
private final String user; | ||
|
||
public HdfsInputFile(Path path, HdfsEnvironment environment, HdfsContext context) | ||
{ | ||
requireNonNull(path, "path is null"); | ||
this.environment = requireNonNull(environment, "environment is null"); | ||
requireNonNull(context, "context is null"); | ||
try { | ||
this.delegate = HadoopInputFile.fromPath(path, environment.getFileSystem(context, path), environment.getConfiguration(context, path)); | ||
} | ||
catch (IOException e) { | ||
throw new PrestoException(ICEBERG_FILESYSTEM_ERROR, "Failed to create input file: " + path, e); | ||
} | ||
this.user = context.getIdentity().getUser(); | ||
} | ||
|
||
@Override | ||
public long getLength() | ||
{ | ||
return environment.doAs(user, delegate::getLength); | ||
} | ||
|
||
@Override | ||
public SeekableInputStream newStream() | ||
{ | ||
return environment.doAs(user, delegate::newStream); | ||
} | ||
|
||
@Override | ||
public String location() | ||
{ | ||
return delegate.location(); | ||
} | ||
|
||
@Override | ||
public boolean exists() | ||
{ | ||
return environment.doAs(user, delegate::exists); | ||
} | ||
} |
76 changes: 76 additions & 0 deletions
76
presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/HdfsOutputFile.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,76 @@ | ||
/* | ||
* Licensed 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 io.prestosql.plugin.iceberg; | ||
|
||
import io.prestosql.plugin.hive.HdfsEnvironment; | ||
import io.prestosql.plugin.hive.HdfsEnvironment.HdfsContext; | ||
import io.prestosql.spi.PrestoException; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.iceberg.hadoop.HadoopOutputFile; | ||
import org.apache.iceberg.io.InputFile; | ||
import org.apache.iceberg.io.OutputFile; | ||
import org.apache.iceberg.io.PositionOutputStream; | ||
|
||
import java.io.IOException; | ||
|
||
import static io.prestosql.plugin.iceberg.IcebergErrorCode.ICEBERG_FILESYSTEM_ERROR; | ||
import static java.util.Objects.requireNonNull; | ||
|
||
public class HdfsOutputFile | ||
implements OutputFile | ||
{ | ||
private final OutputFile delegate; | ||
private final Path path; | ||
private final HdfsEnvironment environment; | ||
private final HdfsContext context; | ||
private final String user; | ||
|
||
public HdfsOutputFile(Path path, HdfsEnvironment environment, HdfsContext context) | ||
{ | ||
this.path = requireNonNull(path, "path is null"); | ||
this.environment = requireNonNull(environment, "environment is null"); | ||
this.context = requireNonNull(context, "context is null"); | ||
try { | ||
this.delegate = HadoopOutputFile.fromPath(path, environment.getFileSystem(context, path), environment.getConfiguration(context, path)); | ||
} | ||
catch (IOException e) { | ||
throw new PrestoException(ICEBERG_FILESYSTEM_ERROR, "Failed to create output file: " + path.toString(), e); | ||
} | ||
this.user = context.getIdentity().getUser(); | ||
} | ||
|
||
@Override | ||
public PositionOutputStream create() | ||
{ | ||
return environment.doAs(user, delegate::create); | ||
} | ||
|
||
@Override | ||
public PositionOutputStream createOrOverwrite() | ||
{ | ||
return environment.doAs(user, delegate::createOrOverwrite); | ||
} | ||
|
||
@Override | ||
public String location() | ||
{ | ||
return delegate.location(); | ||
} | ||
|
||
@Override | ||
public InputFile toInputFile() | ||
{ | ||
return new HdfsInputFile(path, environment, context); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.