Skip to content
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

[Kernel][Defaults] Handle legacy map types in Parquet files #3097

Merged
merged 4 commits into from
May 14, 2024
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -77,8 +77,8 @@ private static Converter createElementConverter(
ArrayType typeFromClient,
GroupType typeFromFile) {

checkArgument(
typeFromFile.getFieldCount() == 1, "Expected exactly one field in the array type");
checkArgument(typeFromFile.getFieldCount() == 1,
"Expected exactly one field in the array type, but got: " + typeFromFile);
GroupType repeatedGroup = typeFromFile.getType(0).asGroupType();

// TODO: handle the legacy 2-level list physical format
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@
import io.delta.kernel.data.ColumnVector;
import io.delta.kernel.types.MapType;

import static io.delta.kernel.internal.util.Preconditions.checkArgument;

import io.delta.kernel.defaults.internal.data.vector.DefaultMapVector;
import static io.delta.kernel.defaults.internal.parquet.ParquetColumnReaders.createConverter;

Expand Down Expand Up @@ -57,10 +59,16 @@ public ColumnVector getDataColumnVector(int batchSize) {
}

private static Converter[] createElementConverters(
int initialBatchSize,
MapType typeFromClient,
GroupType typeFromFile) {
final GroupType innerMapType = (GroupType) typeFromFile.getType("key_value");
int initialBatchSize,
MapType typeFromClient,
GroupType typeFromFile) {
// Repeated element can be any name. Latest Parquet versions use "key_value" as the name,
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we check which version it was written as?

i.e. latest -> use key_value. older? only then allow arbitrary name

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As far as I know, there is no clear or confirmed way to identify which version (i.e., Parquet format version) the file was written with. Each writer followed its own way to add the metadata.

// but legacy versions can use any arbitrary name for the repeated group.
// See https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#maps for details
checkArgument(typeFromFile.getFieldCount() == 1,
"Expected exactly one repeated field in the map type, but got: " + typeFromFile);

GroupType innerMapType = typeFromFile.getType(0).asGroupType();
Converter[] elemConverters = new Converter[2];
elemConverters[0] = createConverter(
initialBatchSize,
Expand Down
Binary file not shown.
Original file line number Diff line number Diff line change
Expand Up @@ -141,4 +141,60 @@ class ParquetFileReaderSuite extends AnyFunSuite

checkAnswer(actResult2, expResult2)
}

/////////////////////////////////////////////////////////////////////////////////////////////////
// Test compatibility with Parquet legacy format files //
/////////////////////////////////////////////////////////////////////////////////////////////////

// Test and the test file are copied from Spark's `ParquetThriftCompatibilitySuite`
test("read parquet file generated by parquet-thrift") {
val parquetFilePath = getTestResourceFilePath("parquet/parquet-thrift-compat.snappy.parquet")

val readSchema = new StructType()
.add("boolColumn", BooleanType.BOOLEAN)
.add("byteColumn", ByteType.BYTE)
.add("shortColumn", ShortType.SHORT)
.add("intColumn", IntegerType.INTEGER)
.add("longColumn", LongType.LONG)
.add("doubleColumn", DoubleType.DOUBLE)
// Thrift `BINARY` values are actually unencoded `STRING` values, and thus are always
// treated as `BINARY (UTF8)` in parquet-thrift, since parquet-thrift always assume
// Thrift `STRING`s are encoded using UTF-8.
.add("binaryColumn", StringType.STRING)
.add("stringColumn", StringType.STRING)
.add("enumColumn", StringType.STRING)
// maybe indicates nullable columns, above ones are non-nullable
.add("maybeBoolColumn", BooleanType.BOOLEAN)
.add("maybeByteColumn", ByteType.BYTE)
.add("maybeShortColumn", ShortType.SHORT)
.add("maybeIntColumn", IntegerType.INTEGER)
.add("maybeLongColumn", LongType.LONG)
.add("maybeDoubleColumn", DoubleType.DOUBLE)
// Thrift `BINARY` values are actually unencoded `STRING` values, and thus are always
// treated as `BINARY (UTF8)` in parquet-thrift, since parquet-thrift always assume
// Thrift `STRING`s are encoded using UTF-8.
.add("maybeBinaryColumn", StringType.STRING)
.add("maybeStringColumn", StringType.STRING)
.add("maybeEnumColumn", StringType.STRING)
// TODO: not working - separate PR to handle 2-level legacy lists
// .add("stringsColumn", new ArrayType(StringType.STRING, true /* containsNull */))
// .add("intSetColumn", new ArrayType(IntegerType.INTEGER, true /* containsNull */))
.add("intToStringColumn",
new MapType(IntegerType.INTEGER, StringType.STRING, true /* valueContainsNull */))
// TODO: not working - separate PR to handle 2-level legacy lists
// .add("complexColumn", new MapType(
// IntegerType.INTEGER,
// new ArrayType(
// new StructType()
// .add("nestedIntsColumn", new ArrayType(IntegerType.INTEGER, true /* containsNull */))
// .add("nestedStringColumn", StringType.STRING)
// .add("stringColumn", StringType.STRING),
// true /* containsNull */),
// true /* valueContainsNull */))

assert(parquetFileRowCount(parquetFilePath) === 10)
checkAnswer(
readParquetFilesUsingKernel(parquetFilePath, readSchema), /* actual */
readParquetFilesUsingSpark(parquetFilePath, readSchema) /* expected */)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -207,27 +207,24 @@ trait ParquetSuiteBase extends TestUtils {
}

def readParquetUsingKernelAsColumnarBatches(
actualFileDir: String,
inputFileOrDir: String,
readSchema: StructType,
predicate: Optional[Predicate] = Optional.empty()): Seq[ColumnarBatch] = {
val parquetFiles = Files.list(Paths.get(actualFileDir))
.iterator().asScala
.map(_.toString)
.filter(path => path.endsWith(".parquet"))
.map(path => FileStatus.of(path, 0L, 0L))
val parquetFileList = parquetFiles(inputFileOrDir)
.map(FileStatus.of(_, 0, 0))

val data = defaultEngine.getParquetHandler.readParquetFiles(
toCloseableIterator(parquetFiles.asJava),
toCloseableIterator(parquetFileList.asJava.iterator()),
readSchema,
predicate)

data.asScala.toSeq
}

def parquetFileCount(path: String): Long = parquetFiles(path).size
def parquetFileCount(fileOrDir: String): Long = parquetFiles(fileOrDir).size

def parquetFileRowCount(path: String): Long = {
val files = parquetFiles(path)
def parquetFileRowCount(fileOrDir: String): Long = {
val files = parquetFiles(fileOrDir)

var rowCount = 0L
files.foreach { file =>
Expand All @@ -238,12 +235,17 @@ trait ParquetSuiteBase extends TestUtils {
rowCount
}

def parquetFiles(path: String): Seq[String] = {
Files.list(Paths.get(path))
.iterator().asScala
.map(_.toString)
.filter(path => path.endsWith(".parquet"))
.toSeq
def parquetFiles(fileOrDir: String): Seq[String] = {
val fileOrDirPath = Paths.get(fileOrDir)
if (Files.isDirectory(fileOrDirPath)) {
Files.list(fileOrDirPath)
.iterator().asScala
.map(_.toString)
.filter(path => path.endsWith(".parquet"))
.toSeq
} else {
Seq(fileOrDir)
}
}

def footer(path: String): ParquetMetadata = {
Expand Down
Loading