-
Notifications
You must be signed in to change notification settings - Fork 2.8k
Spark: Encapsulate parquet objects for Comet #13786
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
Open
parthchandra
wants to merge
3
commits into
apache:main
Choose a base branch
from
parthchandra:iceberg-shading-pr
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from 2 commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
222 changes: 222 additions & 0 deletions
222
parquet/src/main/java/org/apache/iceberg/parquet/CometBridge.java
This file contains hidden or 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,222 @@ | ||
/* | ||
* 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.parquet; | ||
|
||
import java.lang.reflect.Constructor; | ||
import java.lang.reflect.InvocationTargetException; | ||
import java.lang.reflect.Method; | ||
import java.util.List; | ||
import java.util.Map; | ||
import org.apache.iceberg.io.InputFile; | ||
import org.apache.parquet.column.ColumnDescriptor; | ||
|
||
/** | ||
* Bridge class that uses reflection to interact with Comet FileReader without adding a direct | ||
* dependency on the comet-spark jar. | ||
*/ | ||
class CometBridge { | ||
private static final String FILE_READER_CLASS = "org.apache.comet.parquet.FileReader"; | ||
private static final String WRAPPED_INPUT_FILE_CLASS = | ||
"org.apache.comet.parquet.WrappedInputFile"; | ||
private static final String READ_OPTIONS_CLASS = "org.apache.comet.parquet.ReadOptions"; | ||
private static final String READ_OPTIONS_BUILDER_CLASS = | ||
"org.apache.comet.parquet.ReadOptions$Builder"; | ||
private static final String ROW_GROUP_READER_CLASS = "org.apache.comet.parquet.RowGroupReader"; | ||
|
||
private static volatile Boolean cometAvailable = null; | ||
private static Class<?> fileReaderClass; | ||
private static Class<?> wrappedInputFileClass; | ||
private static Class<?> readOptionsClass; | ||
private static Class<?> readOptionsBuilderClass; | ||
private static Class<?> rowGroupReaderClass; | ||
|
||
private static Constructor<?> fileReaderConstructor; | ||
private static Constructor<?> wrappedInputFileConstructor; | ||
private static Constructor<?> readOptionsBuilderConstructor; | ||
private static Method readOptionsBuildMethod; | ||
private static Method setRequestedSchemaFromSpecsMethod; | ||
private static Method readNextRowGroupMethod; | ||
private static Method skipNextRowGroupMethod; | ||
private static Method closeMethod; | ||
private static Method getRowCountMethod; | ||
|
||
// prevent instantiation. | ||
private CometBridge() {} | ||
|
||
/** | ||
* Checks if Comet classes are available in the classpath. | ||
* | ||
* @return true if Comet is available, false otherwise | ||
*/ | ||
public static boolean isCometAvailable() { | ||
if (cometAvailable == null) { | ||
synchronized (CometBridge.class) { | ||
if (cometAvailable == null) { | ||
try { | ||
initializeClasses(); | ||
cometAvailable = true; | ||
} catch (ClassNotFoundException | NoSuchMethodException e) { | ||
cometAvailable = false; | ||
} | ||
} | ||
} | ||
} | ||
return cometAvailable; | ||
} | ||
|
||
private static void initializeClasses() throws ClassNotFoundException, NoSuchMethodException { | ||
// Load classes | ||
fileReaderClass = Class.forName(FILE_READER_CLASS); | ||
wrappedInputFileClass = Class.forName(WRAPPED_INPUT_FILE_CLASS); | ||
readOptionsClass = Class.forName(READ_OPTIONS_CLASS); | ||
readOptionsBuilderClass = Class.forName(READ_OPTIONS_BUILDER_CLASS); | ||
rowGroupReaderClass = Class.forName(ROW_GROUP_READER_CLASS); | ||
|
||
// Initialize constructors and methods | ||
fileReaderConstructor = | ||
fileReaderClass.getConstructor( | ||
wrappedInputFileClass, | ||
readOptionsClass, | ||
Map.class, | ||
Long.class, | ||
Long.class, | ||
byte[].class, | ||
byte[].class); | ||
|
||
// WrappedInputFile constructor takes Object parameter | ||
wrappedInputFileConstructor = wrappedInputFileClass.getConstructor(Object.class); | ||
|
||
// ReadOptionsBuilder constructor takes Configuration parameter | ||
readOptionsBuilderConstructor = | ||
readOptionsBuilderClass.getConstructor(org.apache.hadoop.conf.Configuration.class); | ||
readOptionsBuildMethod = readOptionsBuilderClass.getMethod("build"); | ||
|
||
setRequestedSchemaFromSpecsMethod = | ||
fileReaderClass.getMethod("setRequestedSchemaFromSpecs", List.class); | ||
readNextRowGroupMethod = fileReaderClass.getMethod("readNextRowGroup"); | ||
skipNextRowGroupMethod = fileReaderClass.getMethod("skipNextRowGroup"); | ||
closeMethod = fileReaderClass.getMethod("close"); | ||
|
||
getRowCountMethod = rowGroupReaderClass.getMethod("getRowCount"); | ||
} | ||
|
||
/** Wrapper for Comet FileReader that uses reflection. */ | ||
public static class FileReaderWrapper implements AutoCloseable { | ||
private final Object fileReader; | ||
|
||
private FileReaderWrapper(Object fileReader) { | ||
this.fileReader = fileReader; | ||
} | ||
|
||
/** Creates a new FileReader instance using reflection. */ | ||
public static FileReaderWrapper create( | ||
InputFile file, | ||
Object readOptions, | ||
Map<String, String> properties, | ||
Long start, | ||
Long length, | ||
byte[] fileEncryptionKey, | ||
byte[] fileAADPrefix) | ||
throws IllegalAccessException, InstantiationException, InvocationTargetException { | ||
|
||
// Create WrappedInputFile | ||
Object wrappedInputFile = wrappedInputFileConstructor.newInstance(file); | ||
|
||
// Create FileReader | ||
Object fileReader = | ||
fileReaderConstructor.newInstance( | ||
wrappedInputFile, | ||
readOptions, | ||
properties, | ||
start, | ||
length, | ||
fileEncryptionKey, | ||
fileAADPrefix); | ||
|
||
return new FileReaderWrapper(fileReader); | ||
} | ||
|
||
/** Sets the requested schema from ParquetColumnSpec list. */ | ||
public void setRequestedSchemaFromSpecs(List<Object> specs) | ||
throws IllegalAccessException, InvocationTargetException { | ||
setRequestedSchemaFromSpecsMethod.invoke(fileReader, specs); | ||
} | ||
|
||
/** Reads the next row group. */ | ||
public RowGroupReaderWrapper readNextRowGroup() | ||
throws IllegalAccessException, InvocationTargetException { | ||
Object rowGroupReader = readNextRowGroupMethod.invoke(fileReader); | ||
return new RowGroupReaderWrapper(rowGroupReader); | ||
} | ||
|
||
/** Skips the next row group. */ | ||
public void skipNextRowGroup() throws IllegalAccessException, InvocationTargetException { | ||
skipNextRowGroupMethod.invoke(fileReader); | ||
} | ||
|
||
/** Closes the file reader. */ | ||
@Override | ||
public void close() throws IllegalAccessException, InvocationTargetException { | ||
closeMethod.invoke(fileReader); | ||
} | ||
} | ||
|
||
/** Wrapper for Comet RowGroupReader that uses reflection. */ | ||
public static class RowGroupReaderWrapper { | ||
private final Object rowGroupReader; | ||
|
||
private RowGroupReaderWrapper(Object rowGroupReader) { | ||
this.rowGroupReader = rowGroupReader; | ||
} | ||
|
||
/** Gets the row count for this row group. */ | ||
public long getRowCount() throws IllegalAccessException, InvocationTargetException { | ||
return (Long) getRowCountMethod.invoke(rowGroupReader); | ||
} | ||
|
||
/** Returns the underlying row group reader object. */ | ||
public Object getRowGroupReader() { | ||
return rowGroupReader; | ||
} | ||
} | ||
|
||
/** Creates ReadOptions using reflection. */ | ||
public static Object createReadOptions(org.apache.hadoop.conf.Configuration conf) | ||
throws IllegalStateException, | ||
IllegalAccessException, | ||
InstantiationException, | ||
InvocationTargetException { | ||
if (!isCometAvailable()) { | ||
throw new IllegalStateException("Comet is not available in the classpath"); | ||
} | ||
|
||
Object builder = readOptionsBuilderConstructor.newInstance(conf); | ||
return readOptionsBuildMethod.invoke(builder); | ||
} | ||
|
||
/** Creates ParquetColumnSpec from ColumnDescriptor using CometTypeUtils. */ | ||
public static Object createParquetColumnSpec(ColumnDescriptor descriptor) | ||
throws IllegalStateException { | ||
if (!isCometAvailable()) { | ||
throw new IllegalStateException("Comet is not available in the classpath"); | ||
} | ||
|
||
return CometTypeUtils.descriptorToParquetColumnSpec(descriptor); | ||
} | ||
} |
56 changes: 56 additions & 0 deletions
56
parquet/src/main/java/org/apache/iceberg/parquet/CometIOException.java
This file contains hidden or 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,56 @@ | ||
/* | ||
* 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.parquet; | ||
|
||
import java.io.IOException; | ||
|
||
/** | ||
* Exception class for handling both IOException and general Exception cases when working with Comet | ||
* FileReader operations. This provides a unified way to handle exceptions that may occur during | ||
* reflection-based calls to Comet classes. | ||
*/ | ||
public class CometIOException extends RuntimeException { | ||
|
||
public CometIOException(String message) { | ||
super(message); | ||
} | ||
|
||
public CometIOException(String message, Throwable cause) { | ||
super(message, cause); | ||
} | ||
|
||
public CometIOException(Throwable cause) { | ||
super(cause); | ||
} | ||
|
||
/** Creates a CometIOException from an IOException. */ | ||
public static CometIOException fromIOException(IOException cause) { | ||
return new CometIOException("Comet IO operation failed", cause); | ||
} | ||
|
||
/** Creates a CometIOException from a general Exception. */ | ||
public static CometIOException fromException(Exception cause) { | ||
return new CometIOException("Comet operation failed", cause); | ||
} | ||
|
||
/** Creates a CometIOException with a custom message and cause. */ | ||
public static CometIOException fromException(String message, Exception cause) { | ||
return new CometIOException(message, cause); | ||
} | ||
} |
114 changes: 114 additions & 0 deletions
114
parquet/src/main/java/org/apache/iceberg/parquet/CometPageReadStore.java
This file contains hidden or 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,114 @@ | ||
/* | ||
* 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.parquet; | ||
|
||
import java.lang.reflect.Method; | ||
import java.util.Optional; | ||
import java.util.PrimitiveIterator; | ||
import org.apache.parquet.column.ColumnDescriptor; | ||
import org.apache.parquet.column.page.PageReadStore; | ||
import org.apache.parquet.column.page.PageReader; | ||
|
||
/** | ||
* A PageReadStore implementation that wraps Comet's RowGroupReader using reflection to avoid direct | ||
* dependencies on Comet classes. | ||
*/ | ||
public class CometPageReadStore implements PageReadStore { | ||
private final Object cometRowGroupReader; | ||
private static Method getPageReaderMethod; | ||
private static Method getRowCountMethod; | ||
private static Method getRowIndexOffsetMethod; | ||
private static Method getRowIndexesMethod; | ||
|
||
static { | ||
try { | ||
Class<?> rowGroupReaderClass = Class.forName("org.apache.comet.parquet.RowGroupReader"); | ||
getPageReaderMethod = rowGroupReaderClass.getMethod("getPageReader", String[].class); | ||
getRowCountMethod = rowGroupReaderClass.getMethod("getRowCount"); | ||
getRowIndexOffsetMethod = rowGroupReaderClass.getMethod("getRowIndexOffset"); | ||
// getRowIndexes method may not exist in all versions, so make it optional | ||
try { | ||
getRowIndexesMethod = rowGroupReaderClass.getMethod("getRowIndexes"); | ||
} catch (NoSuchMethodException e) { | ||
getRowIndexesMethod = null; | ||
} | ||
} catch (Exception e) { | ||
throw new RuntimeException("Failed to initialize CometPageReadStore reflection methods", e); | ||
} | ||
} | ||
|
||
public CometPageReadStore(Object cometRowGroupReader) { | ||
this.cometRowGroupReader = cometRowGroupReader; | ||
} | ||
|
||
@Override | ||
public PageReader getPageReader(ColumnDescriptor descriptor) { | ||
try { | ||
return (PageReader) | ||
getPageReaderMethod.invoke(cometRowGroupReader, (Object) descriptor.getPath()); | ||
} catch (Exception e) { | ||
throw new RuntimeException("Failed to get page reader for column: " + descriptor, e); | ||
} | ||
} | ||
|
||
@Override | ||
public long getRowCount() { | ||
try { | ||
return (Long) getRowCountMethod.invoke(cometRowGroupReader); | ||
} catch (Exception e) { | ||
throw new RuntimeException("Failed to get row count", e); | ||
} | ||
} | ||
|
||
/** | ||
* Gets the row index offset from the underlying Comet RowGroupReader. This method uses reflection | ||
* to call getRowIndexOffset() which returns Optional<Long>. | ||
*/ | ||
@Override | ||
public Optional<Long> getRowIndexOffset() { | ||
try { | ||
return (Optional<Long>) getRowIndexOffsetMethod.invoke(cometRowGroupReader); | ||
} catch (Exception e) { | ||
throw new RuntimeException("Failed to get row index offset", e); | ||
} | ||
} | ||
|
||
/** | ||
* Gets the row indexes from the underlying Comet RowGroupReader if available. Falls back to the | ||
* default implementation if the method is not available. | ||
*/ | ||
@Override | ||
@SuppressWarnings("unchecked") | ||
public Optional<PrimitiveIterator.OfLong> getRowIndexes() { | ||
if (getRowIndexesMethod != null) { | ||
try { | ||
return (Optional<PrimitiveIterator.OfLong>) getRowIndexesMethod.invoke(cometRowGroupReader); | ||
} catch (Exception e) { | ||
// Fall back to default implementation | ||
} | ||
} | ||
// Use default implementation from PageReadStore interface | ||
return PageReadStore.super.getRowIndexes(); | ||
} | ||
|
||
/** Returns the underlying Comet RowGroupReader object. */ | ||
public Object getCometRowGroupReader() { | ||
return cometRowGroupReader; | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Seems a couple of the methods are not used. Shall we remove the no usages methods?
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.
Removed