Skip to content
Open
Show file tree
Hide file tree
Changes from all 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 @@ -2449,7 +2449,10 @@ public static class Blob extends HoodieSchema {
public static final String TYPE_DESCRIPTOR = "BLOB";
private static final String DEFAULT_NAME = "blob";
private static final List<Schema.Field> BLOB_FIELDS = createBlobFields();
private static final int REFERENCE_FIELD_COUNT = AvroSchemaUtils.getNonNullTypeFromUnion(BLOB_FIELDS.get(2).schema()).getFields().size();

public static final String INLINE = "INLINE";
public static final String OUT_OF_LINE = "OUT_OF_LINE";
public static final String TYPE = "type";
public static final String INLINE_DATA_FIELD = "data";
public static final String EXTERNAL_REFERENCE = "reference";
Expand All @@ -2465,7 +2468,7 @@ public static int getFieldCount() {
}

public static int getReferenceFieldCount() {
return AvroSchemaUtils.getNonNullTypeFromUnion(BLOB_FIELDS.get(2).schema()).getFields().size();
return REFERENCE_FIELD_COUNT;
}

/**
Expand Down Expand Up @@ -2523,7 +2526,7 @@ private static List<Schema.Field> createBlobFields() {
referenceField.setFields(referenceFields);

return Arrays.asList(
new Schema.Field(TYPE, Schema.createEnum("blob_storage_type", null, null, Arrays.asList("INLINE", "OUT_OF_LINE")), null, null),
new Schema.Field(TYPE, Schema.createEnum("blob_storage_type", null, null, Arrays.asList(INLINE, OUT_OF_LINE)), null, null),
new Schema.Field(INLINE_DATA_FIELD, AvroSchemaUtils.createNullableSchema(bytesField), null, Schema.Field.NULL_DEFAULT_VALUE),
new Schema.Field(EXTERNAL_REFERENCE, AvroSchemaUtils.createNullableSchema(referenceField), null, Schema.Field.NULL_DEFAULT_VALUE)
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -221,6 +221,7 @@ private Type convertField(String fieldName, HoodieSchema schema, Type.Repetition
}
break;
case RECORD:
case BLOB:
return new GroupType(repetition, fieldName, convertFields(schema.getFields(), schemaPath));
case ENUM:
builder = Types.primitive(BINARY, repetition).as(enumType());
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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.spark.sql.types

import org.apache.hudi.common.schema.HoodieSchema

import org.apache.spark.sql.avro.HoodieSparkSchemaConverters

/**
* Factory object for creating Spark StructType representation of {@link HoodieSchema.Blob}.
*
* BLOB is a logical type in Hudi that represents binary large objects. The data can be
* stored inline (as bytes) or out-of-line (as a reference to a file location).
*/
object BlobType {

/**
* Creates a StructType representing a {@link HoodieSchema.Blob}.
*
* @return StructType with blob structure
*/
def apply(): DataType = {
HoodieSparkSchemaConverters.toSqlType(HoodieSchema.createBlob())._1
Copy link
Collaborator

Choose a reason for hiding this comment

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

[nit] During sql parsing visitPrimitiveDataType would always be invoked and call this later. I think we might want to cache the call at L40, maybe something like this?

object BlobType {
  val dataType: DataType = HoodieSparkSchemaConverters.toSqlType(HoodieSchema.createBlob())._1
  def apply(): DataType = dataType
}

}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,129 @@
/*
* 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.hudi.blob

import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaType}

import org.apache.spark.sql.{Column, SparkSession}
import org.apache.spark.sql.functions.{lit, struct}
import org.apache.spark.sql.types._
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}

import java.io.File
import java.nio.file.{Files, Path}

object BlobTestHelpers {
def blobMetadata: Metadata = {
new MetadataBuilder()
.putString(HoodieSchema.TYPE_METADATA_FIELD, HoodieSchemaType.BLOB.name())
.build()
}

def inlineBlobStructCol(name: String, bytesCol: Column): Column = {
struct(
lit(HoodieSchema.Blob.INLINE).as(HoodieSchema.Blob.TYPE),
bytesCol.as(HoodieSchema.Blob.INLINE_DATA_FIELD),
lit(null).cast("struct<externalPath:string,offset:bigint,length:bigint,managed:boolean>")
.as(HoodieSchema.Blob.EXTERNAL_REFERENCE)
).as(name, blobMetadata)
}

def wholeFileBlobStructCol(name: String, filePathCol: Column): Column = {
struct(
lit(HoodieSchema.Blob.OUT_OF_LINE).as(HoodieSchema.Blob.TYPE),
lit(null).cast("binary").as(HoodieSchema.Blob.INLINE_DATA_FIELD),
struct(
filePathCol.as(HoodieSchema.Blob.EXTERNAL_REFERENCE_PATH),
lit(null).cast("bigint").as(HoodieSchema.Blob.EXTERNAL_REFERENCE_OFFSET),
lit(null).cast("bigint").as(HoodieSchema.Blob.EXTERNAL_REFERENCE_LENGTH),
lit(false).as(HoodieSchema.Blob.EXTERNAL_REFERENCE_IS_MANAGED)
).as(HoodieSchema.Blob.EXTERNAL_REFERENCE)
).as(name, blobMetadata)
}

def blobStructCol(name: String, filePathCol: Column, offsetCol: Column, lengthCol: Column): Column = {
struct(
lit(HoodieSchema.Blob.OUT_OF_LINE).as(HoodieSchema.Blob.TYPE),
lit(null).cast("binary").as(HoodieSchema.Blob.INLINE_DATA_FIELD),
struct(
filePathCol.as(HoodieSchema.Blob.EXTERNAL_REFERENCE_PATH),
offsetCol.as(HoodieSchema.Blob.EXTERNAL_REFERENCE_OFFSET),
lengthCol.as(HoodieSchema.Blob.EXTERNAL_REFERENCE_LENGTH),
lit(false).as(HoodieSchema.Blob.EXTERNAL_REFERENCE_IS_MANAGED)
).as(HoodieSchema.Blob.EXTERNAL_REFERENCE)
).as(name, blobMetadata)
}

def createTestFile(tempDir: Path, name: String, size: Int): String = {
val file = new File(tempDir.toString, name)
val bytes = (0 until size).map(i => (i % 256).toByte).toArray
Files.write(file.toPath, bytes)
file.getAbsolutePath
}

/**
* Assert that byte array contains expected pattern (i % 256) at given offset.
*
* @param data Array of bytes to verify
* @param expectedOffset Starting offset for pattern (default 0)
*/
def assertBytesContent(data: Array[Byte], expectedOffset: Int = 0): Unit = {
for (i <- 0 until data.length) {
assertEquals((expectedOffset + i) % 256, data(i) & 0xFF,
s"Mismatch at byte $i (global offset ${expectedOffset + i})")
}
}

/**
* Execute code block with temporary Spark configuration.
* Automatically restores previous values after execution.
*
* @param spark SparkSession instance
* @param configs Configuration key-value pairs to set
* @param fn Code block to execute with configs
*/
def withSparkConfig[T](spark: SparkSession, configs: Map[String, String])(fn: => T): T = {
val oldValues = configs.keys.map(k => (k, spark.conf.getOption(k))).toMap
try {
configs.foreach { case (k, v) => spark.conf.set(k, v) }
fn
} finally {
oldValues.foreach { case (k, oldValue) =>
oldValue match {
case Some(v) => spark.conf.set(k, v)
case None => spark.conf.unset(k)
}
}
}
}

/**
* Assert that DataFrame contains all specified columns.
*
* @param df DataFrame to check
* @param columnNames Variable number of column names
*/
def assertColumnsExist(df: org.apache.spark.sql.DataFrame, columnNames: String*): Unit = {
columnNames.foreach { colName =>
assertTrue(df.columns.contains(colName),
s"DataFrame missing expected column: $colName. Available: ${df.columns.mkString(", ")}")
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
/*
* 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.hudi.blob

import org.apache.hudi.SparkDatasetMixin
import org.apache.hudi.blob.BlobTestHelpers._
import org.apache.hudi.client.SparkRDDWriteClient
import org.apache.hudi.common.model.{HoodieAvroIndexedRecord, HoodieFileFormat, HoodieKey, HoodieRecord, HoodieTableType}
import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaField, HoodieSchemaType}
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.hudi.config.HoodieIndexConfig
import org.apache.hudi.index.HoodieIndex
import org.apache.hudi.testutils.HoodieClientTestBase

import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord}
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.EnumSource

import java.util.{Arrays, Properties}

import scala.collection.JavaConverters._

/**
* End-to-end test for blob support including schema creation, multi-commit writes,
* DataFrame reads, and SQL blob retrieval.
*
* This test validates:
* <ul>
* <li>Creating tables with blob columns</li>
* <li>Writing blob records with out-of-line references</li>
* <li>Multi-commit operations (insert + upsert)</li>
* <li>Reading blob references via DataFrame</li>
* <li>SQL read_blob() function integration</li>
* </ul>
*/
class TestBlobSupport extends HoodieClientTestBase with SparkDatasetMixin {
val SCHEMA: HoodieSchema = HoodieSchema.createRecord("test_blobs", null, null, Arrays.asList(
HoodieSchemaField.of("id", HoodieSchema.create(HoodieSchemaType.STRING), null, null),
HoodieSchemaField.of("value", HoodieSchema.create(HoodieSchemaType.INT), null, null),
HoodieSchemaField.of("data", HoodieSchema.createBlob(), null, null)
))

@ParameterizedTest
@EnumSource(classOf[HoodieTableType])
def testEndToEnd(tableType: HoodieTableType): Unit = {
val filePath1 = createTestFile(tempDir, "file1.bin", 1000)
val filePath2 = createTestFile(tempDir, "file2.bin", 1000)

val properties = new Properties()
properties.put("hoodie.datasource.write.recordkey.field", "id")
properties.put("hoodie.datasource.write.partitionpath.field", "")
properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "id")
properties.put(HoodieTableConfig.PARTITION_FIELDS.key(), "")
properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieFileFormat.PARQUET.toString)

// Initialize table
HoodieTableMetaClient.newTableBuilder()
.setTableName("test_blob_table")
.setTableType(tableType)
.fromProperties(properties)
.initTable(storageConf, basePath)

var client: SparkRDDWriteClient[IndexedRecord] = null
val config = getConfigBuilder(SCHEMA.toString())
.withIndexConfig(HoodieIndexConfig.newBuilder.withIndexType(HoodieIndex.IndexType.SIMPLE).build)
.build()
try {
client = getHoodieWriteClient(config).asInstanceOf[SparkRDDWriteClient[IndexedRecord]]

// First commit - insert
val commit1 = client.startCommit()
val firstBatch = createTestRecords(filePath1)
val statuses1 = client.insert(jsc.parallelize(firstBatch.asJava, 1), commit1).collect()
client.commit(commit1, jsc.parallelize(statuses1, 1))

// Second commit - upsert
val commit2 = client.startCommit()
val secondBatch = createTestRecords(filePath2)
val statuses2 = client.upsert(jsc.parallelize(secondBatch.asJava, 1), commit2).collect()
client.commit(commit2, jsc.parallelize(statuses2, 1))
} finally {
if (client != null) client.close()
}

// Read and verify DataFrame results
val table = sparkSession.read.format("hudi").load(basePath)
val rows = table.collectAsList()
assertEquals(10, rows.size())

rows.asScala.foreach { row =>
val data = row.getStruct(row.fieldIndex("data"))
val reference = data.getStruct(data.fieldIndex(HoodieSchema.Blob.EXTERNAL_REFERENCE))
val filePath = reference.getString(reference.fieldIndex(HoodieSchema.Blob.EXTERNAL_REFERENCE_PATH))
assertTrue(filePath.endsWith("file2.bin"))
}
}

private def createTestRecords(filePath: String): Seq[HoodieRecord[IndexedRecord]] = {
(0 until 10).map { i =>
val id = s"id_$i"
val key = new HoodieKey(id, "")

val dataSchema = SCHEMA.getField("data").get.schema
val fileReference = new GenericData.Record(dataSchema.getField(HoodieSchema.Blob.EXTERNAL_REFERENCE)
.get.getNonNullSchema.toAvroSchema)
fileReference.put(HoodieSchema.Blob.EXTERNAL_REFERENCE_PATH, filePath)
fileReference.put(HoodieSchema.Blob.EXTERNAL_REFERENCE_OFFSET, i * 100L)
fileReference.put(HoodieSchema.Blob.EXTERNAL_REFERENCE_LENGTH, 100L)
fileReference.put(HoodieSchema.Blob.EXTERNAL_REFERENCE_IS_MANAGED, false)


val blobRecord = new GenericData.Record(dataSchema.toAvroSchema)
blobRecord.put(HoodieSchema.Blob.TYPE, new GenericData.EnumSymbol(dataSchema.getField(HoodieSchema.Blob.TYPE)
.get.schema.toAvroSchema, HoodieSchema.Blob.OUT_OF_LINE))
blobRecord.put(HoodieSchema.Blob.EXTERNAL_REFERENCE, fileReference)

val record = new GenericData.Record(SCHEMA.toAvroSchema)
record.put("id", id)
record.put("value", i)
record.put("data", blobRecord)

new HoodieAvroIndexedRecord(key, record)
}
}
}
Loading
Loading